You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by zh...@apache.org on 2017/08/18 23:10:22 UTC

[01/11] geode git commit: GEODE-3169: Decoupling of DiskStore and backups This closes #715 * move backup logic away from DiskStore and into BackupManager * refactor code into smaller methods * improve test code clarity [Forced Update!]

Repository: geode
Updated Branches:
  refs/heads/feature/GEM-1601 291dac019 -> 360c3d536 (forced update)


http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/test/java/org/apache/geode/internal/cache/BackupJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/BackupJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/BackupJUnitTest.java
index caa2ce5..28dc662 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/BackupJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/BackupJUnitTest.java
@@ -23,18 +23,15 @@ import static org.junit.Assert.*;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.filefilter.DirectoryFileFilter;
 import org.apache.commons.io.filefilter.RegexFileFilter;
+
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.DiskStore;
 import org.apache.geode.cache.DiskStoreFactory;
-import org.apache.geode.cache.DiskWriteAttributesFactory;
 import org.apache.geode.cache.EvictionAction;
 import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.internal.cache.persistence.BackupManager;
-import org.apache.geode.internal.cache.persistence.RestoreScript;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.Before;
@@ -54,16 +51,17 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Properties;
 import java.util.Random;
+import java.util.concurrent.CompletableFuture;
 
 @Category(IntegrationTest.class)
 public class BackupJUnitTest {
 
-  protected GemFireCacheImpl cache = null;
+  private static final String DISK_STORE_NAME = "diskStore";
+  private GemFireCacheImpl cache = null;
   private File tmpDir;
-  protected File cacheXmlFile;
+  private File cacheXmlFile;
 
-  protected DistributedSystem ds = null;
-  protected Properties props = new Properties();
+  private Properties props = new Properties();
 
   private File backupDir;
   private File[] diskDirs;
@@ -103,7 +101,6 @@ public class BackupJUnitTest {
 
   private void createCache() throws IOException {
     cache = (GemFireCacheImpl) new CacheFactory(props).create();
-    ds = cache.getDistributedSystem();
   }
 
   @After
@@ -123,33 +120,26 @@ public class BackupJUnitTest {
 
   @Test
   public void testBackupAndRecover() throws IOException, InterruptedException {
-    backupAndRecover(new RegionCreator() {
-      public Region createRegion() {
-        DiskStoreImpl ds = createDiskStore();
-        return BackupJUnitTest.this.createRegion();
-      }
+    backupAndRecover(() -> {
+      createDiskStore();
+      return BackupJUnitTest.this.createRegion();
     });
   }
 
   @Test
   public void testBackupAndRecoverOldConfig() throws IOException, InterruptedException {
-    backupAndRecover(new RegionCreator() {
-      public Region createRegion() {
-        DiskStoreImpl ds = createDiskStore();
-        RegionFactory rf = new RegionFactory();
-        rf.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
-        rf.setDiskDirs(diskDirs);
-        DiskWriteAttributesFactory daf = new DiskWriteAttributesFactory();
-        daf.setMaxOplogSize(1);
-        rf.setDiskWriteAttributes(daf.create());
-        return rf.create("region");
-      }
+    backupAndRecover(() -> {
+      createDiskStore();
+      RegionFactory regionFactory = cache.createRegionFactory();
+      regionFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+      regionFactory.setDiskStoreName(DISK_STORE_NAME);
+      return regionFactory.create("region");
     });
   }
 
-  public void backupAndRecover(RegionCreator regionFactory)
+  private void backupAndRecover(RegionCreator regionFactory)
       throws IOException, InterruptedException {
-    Region region = regionFactory.createRegion();
+    Region<Object, Object> region = regionFactory.createRegion();
 
     // Put enough data to roll some oplogs
     for (int i = 0; i < 1024; i++) {
@@ -193,8 +183,8 @@ public class BackupJUnitTest {
 
     BackupManager backup =
         cache.startBackup(cache.getInternalDistributedSystem().getDistributedMember());
-    backup.prepareBackup();
-    backup.finishBackup(backupDir, null, false);
+    backup.prepareForBackup();
+    backup.doBackup(backupDir, null, false);
 
     // Put another key to make sure we restore
     // from a backup that doesn't contain this key
@@ -238,19 +228,19 @@ public class BackupJUnitTest {
 
   @Test
   public void testBackupEmptyDiskStore() throws IOException, InterruptedException {
-    DiskStoreImpl ds = createDiskStore();
+    createDiskStore();
 
     BackupManager backup =
         cache.startBackup(cache.getInternalDistributedSystem().getDistributedMember());
-    backup.prepareBackup();
-    backup.finishBackup(backupDir, null, false);
+    backup.prepareForBackup();
+    backup.doBackup(backupDir, null, false);
     assertEquals("No backup files should have been created", Collections.emptyList(),
         Arrays.asList(backupDir.list()));
   }
 
   @Test
   public void testBackupOverflowOnlyDiskStore() throws IOException, InterruptedException {
-    DiskStoreImpl ds = createDiskStore();
+    createDiskStore();
     Region region = createOverflowRegion();
     // Put another key to make sure we restore
     // from a backup that doesn't contain this key
@@ -258,8 +248,8 @@ public class BackupJUnitTest {
 
     BackupManager backup =
         cache.startBackup(cache.getInternalDistributedSystem().getDistributedMember());
-    backup.prepareBackup();
-    backup.finishBackup(backupDir, null, false);
+    backup.prepareForBackup();
+    backup.doBackup(backupDir, null, false);
 
 
     assertEquals("No backup files should have been created", Collections.emptyList(),
@@ -275,51 +265,54 @@ public class BackupJUnitTest {
     dsf.setAutoCompact(false);
     dsf.setAllowForceCompaction(true);
     dsf.setCompactionThreshold(20);
-    String name = "diskStore";
-    DiskStoreImpl ds = (DiskStoreImpl) dsf.create(name);
+    DiskStoreImpl ds = (DiskStoreImpl) dsf.create(DISK_STORE_NAME);
 
-    Region region = createRegion();
+    Region<Object, Object> region = createRegion();
 
     // Put enough data to roll some oplogs
     for (int i = 0; i < 1024; i++) {
       region.put(i, getBytes(i));
     }
 
-    RestoreScript script = new RestoreScript();
-    ds.startBackup(backupDir, null, script);
-
-    for (int i = 2; i < 1024; i++) {
-      assertTrue(region.destroy(i) != null);
-    }
-    assertTrue(ds.forceCompaction());
-    // Put another key to make sure we restore
-    // from a backup that doesn't contain this key
-    region.put("A", "A");
-
-    ds.finishBackup(
-        new BackupManager(cache.getInternalDistributedSystem().getDistributedMember(), cache));
-    script.generate(backupDir);
+    BackupManager backupManager =
+        cache.startBackup(cache.getInternalDistributedSystem().getDistributedMember());
+    backupManager.validateRequestingAdmin();
+    backupManager.prepareForBackup();
+    final Region theRegion = region;
+    final DiskStore theDiskStore = ds;
+    CompletableFuture.runAsync(() -> destroyAndCompact(theRegion, theDiskStore));
+    backupManager.doBackup(backupDir, null, false);
 
     cache.close();
     destroyDiskDirs();
     restoreBackup(false);
     createCache();
-    ds = createDiskStore();
+    createDiskStore();
     region = createRegion();
     validateEntriesExist(region, 0, 1024);
 
     assertNull(region.get("A"));
   }
 
+  private void destroyAndCompact(Region<Object, Object> region, DiskStore diskStore) {
+    for (int i = 2; i < 1024; i++) {
+      assertTrue(region.destroy(i) != null);
+    }
+    assertTrue(diskStore.forceCompaction());
+    // Put another key to make sure we restore
+    // from a backup that doesn't contain this key
+    region.put("A", "A");
+  }
+
   @Test
   public void testBackupCacheXml() throws Exception {
-    DiskStoreImpl ds = createDiskStore();
+    createDiskStore();
     createRegion();
 
     BackupManager backup =
         cache.startBackup(cache.getInternalDistributedSystem().getDistributedMember());
-    backup.prepareBackup();
-    backup.finishBackup(backupDir, null, false);
+    backup.prepareForBackup();
+    backup.doBackup(backupDir, null, false);
     Collection<File> fileCollection = FileUtils.listFiles(backupDir,
         new RegexFileFilter("cache.xml"), DirectoryFileFilter.DIRECTORY);
     assertEquals(1, fileCollection.size());
@@ -337,12 +330,9 @@ public class BackupJUnitTest {
     // The cache xml file should be small enough to fit in one byte array
     int size = (int) file.length();
     byte[] contents = new byte[size];
-    FileInputStream fis = new FileInputStream(file);
-    try {
+    try (FileInputStream fis = new FileInputStream(file)) {
       assertEquals(size, fis.read(contents));
       assertEquals(-1, fis.read());
-    } finally {
-      fis.close();
     }
     return contents;
   }
@@ -406,36 +396,35 @@ public class BackupJUnitTest {
 
   }
 
-  protected Region createRegion() {
-    RegionFactory rf = new RegionFactory();
-    rf.setDiskStoreName("diskStore");
-    rf.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
-    return rf.create("region");
+  private Region createRegion() {
+    RegionFactory regionFactory = cache.createRegionFactory();
+    regionFactory.setDiskStoreName(DISK_STORE_NAME);
+    regionFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+    return regionFactory.create("region");
   }
 
   private Region createOverflowRegion() {
-    RegionFactory rf = new RegionFactory();
-    rf.setDiskStoreName("diskStore");
-    rf.setEvictionAttributes(
+    RegionFactory regionFactory = cache.createRegionFactory();
+    regionFactory.setDiskStoreName(DISK_STORE_NAME);
+    regionFactory.setEvictionAttributes(
         EvictionAttributes.createLIFOEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK));
-    rf.setDataPolicy(DataPolicy.NORMAL);
-    return rf.create("region");
+    regionFactory.setDataPolicy(DataPolicy.NORMAL);
+    return regionFactory.create("region");
   }
 
   private DiskStore findDiskStore() {
-    return cache.findDiskStore("diskStore");
+    return cache.findDiskStore(DISK_STORE_NAME);
   }
 
-  private DiskStoreImpl createDiskStore() {
-    DiskStoreFactory dsf = cache.createDiskStoreFactory();
-    dsf.setDiskDirs(diskDirs);
-    dsf.setMaxOplogSize(1);
-    String name = "diskStore";
-    return (DiskStoreImpl) dsf.create(name);
+  private void createDiskStore() {
+    DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+    diskStoreFactory.setDiskDirs(diskDirs);
+    diskStoreFactory.setMaxOplogSize(1);
+    diskStoreFactory.create(DISK_STORE_NAME);
   }
 
   private interface RegionCreator {
-    Region createRegion();
+    Region<Object, Object> createRegion();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
index ee3d7f7..f31f17b 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
@@ -55,7 +55,6 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.ClassBuilder;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.DeployedJar;
-import org.apache.geode.internal.cache.persistence.BackupManager;
 import org.apache.geode.internal.util.IOUtils;
 import org.apache.geode.internal.util.TransformUtils;
 import org.apache.geode.test.dunit.Host;
@@ -615,7 +614,7 @@ public class IncrementalBackupDUnitTest extends JUnit4CacheTestCase {
     File backupDir = getBackupDirForMember(getBaselineDir(), getMemberId(vm));
     assertTrue(backupDir.exists());
 
-    File incomplete = new File(backupDir, BackupManager.INCOMPLETE_BACKUP);
+    File incomplete = new File(backupDir, BackupManager.INCOMPLETE_BACKUP_FILE);
     incomplete.createNewFile();
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupPrepareAndFinishMsgDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupPrepareAndFinishMsgDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupPrepareAndFinishMsgDUnitTest.java
index 39c5c3c..e0fea77 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupPrepareAndFinishMsgDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupPrepareAndFinishMsgDUnitTest.java
@@ -22,11 +22,18 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
 
 import org.apache.geode.admin.internal.FinishBackupRequest;
 import org.apache.geode.admin.internal.PrepareBackupRequest;
@@ -46,490 +53,151 @@ import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.internal.cache.BackupLock;
 import org.apache.geode.internal.cache.DiskStoreImpl;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.awaitility.Awaitility;
-import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({DistributedTest.class})
-public class BackupPrepareAndFinishMsgDUnitTest extends CacheTestCase {
+public abstract class BackupPrepareAndFinishMsgDUnitTest extends CacheTestCase {
   // Although this test does not make use of other members, the current member needs to be
   // a distributed member (rather than local) because it sends prepare and finish backup messages
-  File[] diskDirs = null;
+  private static final String TEST_REGION_NAME = "TestRegion";
+  private File[] diskDirs = null;
   private int waitingForBackupLockCount = 0;
+  private Region<Integer, Integer> region;
 
-  @After
-  public void after() throws Exception {
-    waitingForBackupLockCount = 0;
-    diskDirs = null;
-  }
-
-  @Test
-  public void testCreateWithParReg() throws Throwable {
-    doCreate(RegionShortcut.PARTITION_PERSISTENT, true);
-  }
-
-  @Test
-  public void testCreateWithReplicate() throws Throwable {
-    doCreate(RegionShortcut.REPLICATE_PERSISTENT, true);
-  }
-
-  @Test
-  public void testPutAsCreateWithParReg() throws Throwable {
-    doCreate(RegionShortcut.PARTITION_PERSISTENT, false);
-  }
-
-  @Test
-  public void testPutAsCreateWithReplicate() throws Throwable {
-    doCreate(RegionShortcut.REPLICATE_PERSISTENT, false);
-  }
-
-  @Test
-  public void testUpdateWithParReg() throws Throwable {
-    doUpdate(RegionShortcut.PARTITION_PERSISTENT);
-  }
-
-  @Test
-  public void testUpdateWithReplicate() throws Throwable {
-    doUpdate(RegionShortcut.REPLICATE_PERSISTENT);
-  }
-
-  @Test
-  public void testInvalidateWithParReg() throws Throwable {
-    doInvalidate(RegionShortcut.PARTITION_PERSISTENT);
-  }
-
-  @Test
-  public void testInvalidateWithReplicate() throws Throwable {
-    doInvalidate(RegionShortcut.REPLICATE_PERSISTENT);
-  }
-
-  @Test
-  public void testDestroyWithParReg() throws Throwable {
-    doDestroy(RegionShortcut.PARTITION_PERSISTENT);
-  }
-
-  @Test
-  public void testDestroyWithReplicate() throws Throwable {
-    doDestroy(RegionShortcut.REPLICATE_PERSISTENT);
-  }
-
-  @Test
-  public void testGetWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "get");
-  }
-
-  @Test
-  public void testGetWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "get");
-  }
-
-  @Test
-  public void testContainsKeyWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "containsKey");
-  }
-
-  @Test
-  public void testContainsKeyWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "containsKey");
-  }
-
-  @Test
-  public void testContainsValueWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "containsValue");
-  }
-
-  @Test
-  public void testContainsValueWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "containsValue");
-  }
-
-  @Test
-  public void testContainsValueForKeyWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "containsValueForKey");
-  }
-
-  @Test
-  public void testContainsValueForKeyWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "containsValueForKey");
-  }
-
-  @Test
-  public void testEntrySetWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "entrySet");
-  }
-
-  @Test
-  public void testEntrySetWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "entrySet");
-  }
-
-  @Test
-  public void testGetAllWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "getAll");
-  }
-
-  @Test
-  public void testGetAllWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "getAll");
-  }
-
-  @Test
-  public void testGetEntryWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "getEntry");
-  }
-
-  @Test
-  public void testGetEntryWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "getEntry");
-  }
-
-  @Test
-  public void testIsEmptyWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "isEmpty");
-  }
-
-  @Test
-  public void testIsEmptyWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "isEmpty");
-  }
-
-  @Test
-  public void testKeySetWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "keySet");
-  }
-
-  @Test
-  public void testKeySetWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "keySet");
-  }
-
-  @Test
-  public void testSizeWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "size");
-  }
+  protected abstract Region<Integer, Integer> createRegion();
 
-  @Test
-  public void testSizeWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "size");
+  @Before
+  public void setup() {
+    region = createRegion();
   }
 
   @Test
-  public void testValuesWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "values");
+  public void createWaitsForBackupTest() throws Throwable {
+    doActionAndVerifyWaitForBackup(() -> region.create(1, 1));
+    verifyKeyValuePair(1, 1);
   }
 
   @Test
-  public void testValuesWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "values");
+  public void putThatCreatesWaitsForBackupTest() throws Throwable {
+    doActionAndVerifyWaitForBackup(() -> region.put(1, 1));
+    verifyKeyValuePair(1, 1);
   }
 
   @Test
-  public void testQueryWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "query");
+  public void putWaitsForBackupTest() throws Throwable {
+    region.put(1, 1);
+    doActionAndVerifyWaitForBackup(() -> region.put(1, 2));
+    verifyKeyValuePair(1, 2);
   }
 
   @Test
-  public void testQueryWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "query");
+  public void invalidateWaitsForBackupTest() throws Throwable {
+    region.put(1, 1);
+    doActionAndVerifyWaitForBackup(() -> region.invalidate(1));
+    verifyKeyValuePair(1, null);
   }
 
   @Test
-  public void testExistsValueWithParReg() throws Throwable {
-    doRead(RegionShortcut.PARTITION_PERSISTENT, "existsValue");
+  public void destroyWaitsForBackupTest() throws Throwable {
+    region.put(1, 1);
+    doActionAndVerifyWaitForBackup(() -> region.destroy(1));
+    assertFalse(region.containsKey(1));
   }
 
   @Test
-  public void testExistsValueWithReplicate() throws Throwable {
-    doRead(RegionShortcut.REPLICATE_PERSISTENT, "existsValue");
-  }
+  public void putAllWaitsForBackupTest() throws Throwable {
+    Map<Integer, Integer> entries = new HashMap<>();
+    entries.put(1, 1);
+    entries.put(2, 2);
 
-  @Test
-  public void testPutAllWithParReg() throws Throwable {
-    doPutAll(RegionShortcut.PARTITION_PERSISTENT);
+    doActionAndVerifyWaitForBackup(() -> region.putAll(entries));
+    verifyKeyValuePair(1, 1);
+    verifyKeyValuePair(2, 2);
   }
 
   @Test
-  public void testPutAllWithReplicate() throws Throwable {
-    doPutAll(RegionShortcut.REPLICATE_PERSISTENT);
-  }
+  public void removeAllWaitsForBackupTest() throws Throwable {
+    region.put(1, 1);
+    region.put(2, 2);
 
-  @Test
-  public void testRemoveAllWithParReg() throws Throwable {
-    doRemoveAll(RegionShortcut.PARTITION_PERSISTENT);
+    List<Integer> keys = Arrays.asList(1, 2);
+    doActionAndVerifyWaitForBackup(() -> region.removeAll(keys));
+    assertTrue(region.isEmpty());
   }
 
   @Test
-  public void testRemoveAllWithReplicate() throws Throwable {
-    doRemoveAll(RegionShortcut.REPLICATE_PERSISTENT);
-  }
-
-  /**
-   * Test that a create waits for backup
-   * 
-   * @param shortcut The region shortcut to use to create the region
-   * @throws InterruptedException
-   */
-  private void doCreate(RegionShortcut shortcut, boolean useCreate) throws InterruptedException {
-    Region aRegion = createRegion(shortcut);
-    Runnable runnable = new Runnable() {
-      public void run() {
-        if (useCreate) {
-          aRegion.create(1, 1);
-        } else {
-          aRegion.put(1, 1);
-        }
-      }
-    };
-
-    verifyWaitForBackup(runnable);
-    assertTrue(aRegion.containsKey(1));
-    assertEquals(aRegion.get(1), 1);
-  }
-
-  /**
-   * Test that an update waits for backup
-   * 
-   * @param shortcut The region shortcut to use to create the region
-   * @throws InterruptedException
-   */
-  private void doUpdate(RegionShortcut shortcut) throws InterruptedException {
-    Region aRegion = createRegion(shortcut);
-    aRegion.put(1, 1);
-
-    Runnable runnable = new Runnable() {
-      public void run() {
-        aRegion.put(1, 2);
-      }
-    };
-
-    verifyWaitForBackup(runnable);
-    assertTrue(aRegion.containsKey(1));
-    assertEquals(aRegion.get(1), 2);
-  }
-
-  /**
-   * Test that an invalidate waits for backup
-   * 
-   * @param shortcut The region shortcut to use to create the region
-   * @throws InterruptedException
-   */
-  private void doInvalidate(RegionShortcut shortcut) throws InterruptedException {
-    Region aRegion = createRegion(shortcut);
-    aRegion.put(1, 1);
-
-    Runnable runnable = (new Runnable() {
-      public void run() {
-        aRegion.invalidate(1);
-      }
-    });
-
-    verifyWaitForBackup(runnable);
-    assertTrue(aRegion.containsKey(1));
-    assertNull(aRegion.get(1));
+  public void readActionsDoNotBlockDuringBackup() {
+    region.put(1, 1);
+    doReadActionsAndVerifyCompletion();
   }
 
-  /**
-   * Test that a destroy waits for backup
-   * 
-   * @param shortcut The region shortcut to use to create the region
-   * @throws InterruptedException
-   */
-  private void doDestroy(RegionShortcut shortcut) throws InterruptedException {
-    Region aRegion = createRegion(shortcut);
-    aRegion.put(1, 1);
-
-    Runnable runnable = new Runnable() {
-      public void run() {
-        aRegion.destroy(1);
-      }
-    };
-
-    verifyWaitForBackup(runnable);
-    assertFalse(aRegion.containsKey(1));
-  }
-
-  /**
-   * Test that a read op does NOT wait for backup
-   * 
-   * @param shortcut The region shortcut to use to create the region
-   * @throws InterruptedException
-   */
-  private void doRead(RegionShortcut shortcut, String op) throws Exception {
-    Region aRegion = createRegion(shortcut);
-    aRegion.put(1, 1);
-
-    Runnable runnable = new Runnable() {
-      public void run() {
-        switch (op) {
-          case "get": {
-            aRegion.get(1);
-            break;
-          }
-          case "containsKey": {
-            aRegion.containsKey(1);
-            break;
-          }
-          case "containsValue": {
-            aRegion.containsValue(1);
-            break;
-          }
-          case "containsValueForKey": {
-            aRegion.containsValue(1);
-            break;
-          }
-          case "entrySet": {
-            aRegion.entrySet();
-            break;
-          }
-          case "existsValue": {
-            try {
-              aRegion.existsValue("value = 1");
-            } catch (FunctionDomainException | TypeMismatchException | NameResolutionException
-                | QueryInvocationTargetException e) {
-              fail(e.toString());
-            }
-            break;
-          }
-          case "getAll": {
-            aRegion.getAll(new ArrayList());
-            break;
-          }
-          case "getEntry": {
-            aRegion.getEntry(1);
-            break;
-          }
-          case "isEmpty": {
-            aRegion.isEmpty();
-            break;
-          }
-          case "keySet": {
-            aRegion.keySet();
-            break;
-          }
-          case "query": {
-            try {
-              aRegion.query("select *");
-            } catch (FunctionDomainException | TypeMismatchException | NameResolutionException
-                | QueryInvocationTargetException e) {
-              fail(e.toString());
-            }
-            break;
-          }
-          case "size": {
-            aRegion.size();
-            break;
-          }
-          case "values": {
-            aRegion.values();
-            break;
-          }
-          default: {
-            fail("Unknown operation " + op);
-          }
-        }
-      }
-    };
-
-    verifyNoWaitForBackup(runnable);
-  }
-
-  /**
-   * Test that a putAll waits for backup
-   * 
-   * @param shortcut The region shortcut to use to create the region
-   * @throws InterruptedException
-   */
-  private void doPutAll(RegionShortcut shortcut) throws InterruptedException {
-    Region aRegion = createRegion(shortcut);
-    Runnable runnable = new Runnable() {
-      public void run() {
-        Map<Object, Object> putAllMap = new HashMap<Object, Object>();
-        putAllMap.put(1, 1);
-        putAllMap.put(2, 2);
-        aRegion.putAll(putAllMap);
-      }
-    };
-
-    verifyWaitForBackup(runnable);
-    assertTrue(aRegion.containsKey(1));
-    assertEquals(aRegion.get(1), 1);
-    assertTrue(aRegion.containsKey(2));
-    assertEquals(aRegion.get(2), 2);
-  }
-
-  /**
-   * Test that a removeAll waits for backup
-   * 
-   * @param shortcut The region shortcut to use to create the region
-   * @throws InterruptedException
-   */
-  private void doRemoveAll(RegionShortcut shortcut) throws InterruptedException {
-    Region aRegion = createRegion(shortcut);
-    aRegion.put(1, 2);
-    aRegion.put(2, 3);
-
-    Runnable runnable = new Runnable() {
-      public void run() {
-        List<Object> keys = new ArrayList();
-        keys.add(1);
-        keys.add(2);
-        aRegion.removeAll(keys);
-      }
-    };
-
-    verifyWaitForBackup(runnable);
-    assertEquals(aRegion.size(), 0);
+  private void doActionAndVerifyWaitForBackup(Runnable function)
+      throws InterruptedException, TimeoutException, ExecutionException {
+    DM dm = GemFireCacheImpl.getInstance().getDistributionManager();
+    Set recipients = dm.getOtherDistributionManagerIds();
+    Future<Void> future = null;
+    PrepareBackupRequest.send(dm, recipients);
+    waitingForBackupLockCount = 0;
+    future = CompletableFuture.runAsync(function);
+    Awaitility.await().atMost(5, TimeUnit.SECONDS)
+        .until(() -> assertTrue(waitingForBackupLockCount == 1));
+    FinishBackupRequest.send(dm, recipients, diskDirs[0], null, false);
+    future.get(5, TimeUnit.SECONDS);
   }
 
-  /**
-   * Test that executing the given runnable waits for backup completion to proceed
-   * 
-   * @param runnable The code that should wait for backup.
-   * @throws InterruptedException
-   */
-  private void verifyWaitForBackup(Runnable runnable) throws InterruptedException {
-    DM dm = ((InternalCache) GemFireCacheImpl.getInstance()).getDistributionManager();
+  private void doReadActionsAndVerifyCompletion() {
+    DM dm = GemFireCacheImpl.getInstance().getDistributionManager();
     Set recipients = dm.getOtherDistributionManagerIds();
-    boolean abort = true;
-    Thread aThread = new Thread(runnable);
+    PrepareBackupRequest.send(dm, recipients);
+    waitingForBackupLockCount = 0;
+    List<CompletableFuture<?>> futureList = doReadActions();
+    CompletableFuture.allOf(futureList.toArray(new CompletableFuture<?>[futureList.size()]));
+    assertTrue(waitingForBackupLockCount == 0);
+    FinishBackupRequest.send(dm, recipients, diskDirs[0], null, false);
+  }
+
+  private void verifyKeyValuePair(Integer key, Integer expectedValue) {
+    assertTrue(region.containsKey(key));
+    assertEquals(expectedValue, region.get(key));
+  }
+
+  private List<CompletableFuture<?>> doReadActions() {
+    List<Runnable> actions = new ArrayList<>();
+    actions.add(() -> region.get(1));
+    actions.add(() -> region.containsKey(1));
+    actions.add(() -> region.containsValue(1));
+    actions.add(region::entrySet);
+    actions.add(this::valueExistsCheck);
+    actions.add(() -> region.getAll(Collections.emptyList()));
+    actions.add(() -> region.getEntry(1));
+    actions.add(region::isEmpty);
+    actions.add(region::keySet);
+    actions.add(region::size);
+    actions.add(region::values);
+    actions.add(this::queryCheck);
+    return actions.stream().map(runnable -> CompletableFuture.runAsync(runnable))
+        .collect(Collectors.toList());
+  }
+
+  private void valueExistsCheck() {
     try {
-      PrepareBackupRequest.send(dm, recipients);
-      abort = false;
-      waitingForBackupLockCount = 0;
-      aThread.start();
-      Awaitility.await().atMost(30, TimeUnit.SECONDS)
-          .until(() -> assertTrue(waitingForBackupLockCount == 1));
-    } finally {
-      FinishBackupRequest.send(dm, recipients, diskDirs[0], null, abort);
-      aThread.join(30000);
-      assertFalse(aThread.isAlive());
+      region.existsValue("value = 1");
+    } catch (FunctionDomainException | TypeMismatchException | NameResolutionException
+        | QueryInvocationTargetException e) {
+      throw new RuntimeException(e);
     }
   }
 
-  /**
-   * Test that executing the given runnable does NOT wait for backup completion to proceed
-   * 
-   * @param runnable The code that should not wait for backup.
-   * @throws InterruptedException
-   */
-  private void verifyNoWaitForBackup(Runnable runnable) throws InterruptedException {
-    DM dm = ((InternalCache) GemFireCacheImpl.getInstance()).getDistributionManager();
-    Set recipients = dm.getOtherDistributionManagerIds();
-    boolean abort = true;
-    Thread aThread = new Thread(runnable);
+  private void queryCheck() {
     try {
-      PrepareBackupRequest.send(dm, recipients);
-      abort = false;
-      waitingForBackupLockCount = 0;
-      aThread.start();
-      aThread.join(30000);
-      assertFalse(aThread.isAlive());
-      assertTrue(waitingForBackupLockCount == 0);
-    } finally {
-      FinishBackupRequest.send(dm, recipients, diskDirs[0], null, abort);
+      region.query("select * from /" + TEST_REGION_NAME);
+    } catch (FunctionDomainException | TypeMismatchException | NameResolutionException
+        | QueryInvocationTargetException e) {
+      throw new RuntimeException(e);
     }
   }
 
@@ -549,7 +217,7 @@ public class BackupPrepareAndFinishMsgDUnitTest extends CacheTestCase {
    * @param shortcut The region shortcut to use to create the region
    * @return The newly created region.
    */
-  private Region<?, ?> createRegion(RegionShortcut shortcut) {
+  protected Region<Integer, Integer> createRegion(RegionShortcut shortcut) {
     Cache cache = getCache();
     DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
     diskDirs = getDiskDirs();
@@ -557,7 +225,7 @@ public class BackupPrepareAndFinishMsgDUnitTest extends CacheTestCase {
     DiskStore diskStore = diskStoreFactory.create(getUniqueName());
     ((DiskStoreImpl) diskStore).getBackupLock().setBackupLockTestHook(new BackupLockHook());
 
-    RegionFactory<String, String> regionFactory = cache.createRegionFactory(shortcut);
+    RegionFactory<Integer, Integer> regionFactory = cache.createRegionFactory(shortcut);
     regionFactory.setDiskStoreName(diskStore.getName());
     regionFactory.setDiskSynchronous(true);
     if (shortcut.equals(RegionShortcut.PARTITION_PERSISTENT)) {
@@ -565,7 +233,7 @@ public class BackupPrepareAndFinishMsgDUnitTest extends CacheTestCase {
       prFactory.setTotalNumBuckets(1);
       regionFactory.setPartitionAttributes(prFactory.create());
     }
-    return regionFactory.create("TestRegion");
+    return regionFactory.create(TEST_REGION_NAME);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PartitionedBackupPrepareAndFinishMsgDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PartitionedBackupPrepareAndFinishMsgDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PartitionedBackupPrepareAndFinishMsgDUnitTest.java
new file mode 100644
index 0000000..4b42c21
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PartitionedBackupPrepareAndFinishMsgDUnitTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.cache.persistence;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+
+public class PartitionedBackupPrepareAndFinishMsgDUnitTest
+    extends BackupPrepareAndFinishMsgDUnitTest {
+  private static final RegionShortcut REGION_TYPE = RegionShortcut.PARTITION_PERSISTENT;
+
+  @Override
+  public Region<Integer, Integer> createRegion() {
+    return createRegion(REGION_TYPE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
new file mode 100644
index 0000000..3f0ba7d
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.cache.persistence;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+
+public class ReplicateBackupPrepareAndFinishMsgDUnitTest
+    extends BackupPrepareAndFinishMsgDUnitTest {
+  private static final RegionShortcut REGION_TYPE = RegionShortcut.REPLICATE_PERSISTENT;
+
+  @Override
+  public Region<Integer, Integer> createRegion() {
+    return createRegion(REGION_TYPE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java
index bdf097e..60fb859 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java
@@ -32,7 +32,7 @@ import org.apache.geode.admin.internal.PrepareBackupRequest;
 import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.locks.DLockService;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.persistence.BackupManager;
+import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.test.fake.Fakes;
 import org.apache.geode.test.junit.categories.UnitTest;
@@ -74,9 +74,9 @@ public class DistributedSystemBridgeJUnitTest {
 
     InOrder inOrder = inOrder(dm, backupManager);
     inOrder.verify(dm).putOutgoing(isA(PrepareBackupRequest.class));
-    inOrder.verify(backupManager).prepareBackup();
+    inOrder.verify(backupManager).prepareForBackup();
     inOrder.verify(dm).putOutgoing(isA(FinishBackupRequest.class));
-    inOrder.verify(backupManager).finishBackup(any(), any(), eq(false));
+    inOrder.verify(backupManager).doBackup(any(), any(), eq(false));
   }
 
   @Test
@@ -99,6 +99,6 @@ public class DistributedSystemBridgeJUnitTest {
     }
 
     verify(dm).putOutgoing(isA(FinishBackupRequest.class));
-    verify(backupManager).finishBackup(any(), any(), eq(true));
+    verify(backupManager).doBackup(any(), any(), eq(true));
   }
 }


[07/11] geode git commit: GEODE-3395 Variable-ize product version and name in user guide - Developing

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/distributed_regions/how_region_versioning_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/distributed_regions/how_region_versioning_works.html.md.erb b/geode-docs/developing/distributed_regions/how_region_versioning_works.html.md.erb
index 9911d31..83fedc1 100644
--- a/geode-docs/developing/distributed_regions/how_region_versioning_works.html.md.erb
+++ b/geode-docs/developing/distributed_regions/how_region_versioning_works.html.md.erb
@@ -21,32 +21,32 @@ limitations under the License.
 
 <a id="topic_7A4B6C6169BD4B1ABD356294F744D236"></a>
 
-Geode performs different consistency checks depending on the type of region you have configured.
+<%=vars.product_name%> performs different consistency checks depending on the type of region you have configured.
 
 ## <a id="topic_7A4B6C6169BD4B1ABD356294F744D236__section_B090F5FB87D84104A7BE4BCEA6BAE6B7" class="no-quick-link"></a>Partitioned Region Consistency
 
-For a partitioned region, Geode maintains consistency by routing all updates on a given key to the Geode member that holds the primary copy of that key. That member holds a lock on the key while distributing updates to other members that host a copy of the key. Because all updates to a partitioned region are serialized on the primary Geode member, all members apply the updates in the same order and consistency is maintained at all times. See [Understanding Partitioning](../partitioned_regions/how_partitioning_works.html).
+For a partitioned region, <%=vars.product_name%> maintains consistency by routing all updates on a given key to the <%=vars.product_name%> member that holds the primary copy of that key. That member holds a lock on the key while distributing updates to other members that host a copy of the key. Because all updates to a partitioned region are serialized on the primary <%=vars.product_name%> member, all members apply the updates in the same order and consistency is maintained at all times. See [Understanding Partitioning](../partitioned_regions/how_partitioning_works.html).
 
 ## <a id="topic_7A4B6C6169BD4B1ABD356294F744D236__section_72DFB366C8F14ADBAF2A136669ECAB1E" class="no-quick-link"></a>Replicated Region Consistency
 
-For a replicated region, any member that hosts the region can update a key and distribute that update to other members without locking the key. It is possible that two members can update the same key at the same time (a concurrent update). It is also possible that, due to network latency, an update in one member is distributed to other members at a later time, after those members have already applied more recent updates to the key (an out-of-order update). By default, Geode members perform conflict checking before applying region updates in order to detect and consistently resolve concurrent and out-of-order updates. Conflict checking ensures that region data eventually becomes consistent on all members that host the region. The conflict checking behavior for replicated regions is summarized as follows:
+For a replicated region, any member that hosts the region can update a key and distribute that update to other members without locking the key. It is possible that two members can update the same key at the same time (a concurrent update). It is also possible that, due to network latency, an update in one member is distributed to other members at a later time, after those members have already applied more recent updates to the key (an out-of-order update). By default, <%=vars.product_name%> members perform conflict checking before applying region updates in order to detect and consistently resolve concurrent and out-of-order updates. Conflict checking ensures that region data eventually becomes consistent on all members that host the region. The conflict checking behavior for replicated regions is summarized as follows:
 
 -   If two members update the same key at the same time, conflict checking ensures that all members eventually apply the same value, which is the value of one of the two concurrent updates.
 -   If a member receives an out-of-order update (an update that is received after one or more recent updates were applied), conflict checking ensures that the out-of-order update is discarded and not applied to the cache.
 
-[How Consistency Checking Works for Replicated Regions](#topic_C5B74CCDD909403C815639339AA03758) and [How Destroy and Clear Operations Are Resolved](#topic_321B05044B6641FCAEFABBF5066BD399) provide more details about how Geode performs conflict checking when applying an update.
+[How Consistency Checking Works for Replicated Regions](#topic_C5B74CCDD909403C815639339AA03758) and [How Destroy and Clear Operations Are Resolved](#topic_321B05044B6641FCAEFABBF5066BD399) provide more details about how <%=vars.product_name%> performs conflict checking when applying an update.
 
 ## <a id="topic_7A4B6C6169BD4B1ABD356294F744D236__section_313045F430EE459CB411CAAE7B00F3D8" class="no-quick-link"></a>Non-Replicated Regions and Client Cache Consistency
 
 When a member receives an update for an entry in a non-replicated region and applies an update, it performs conflict checking in the same way as for a replicated region. However, if the member initiates an operation on an entry that is not present in the region, it first passes that operation to a member that hosts a replicate. The member that hosts the replica generates and provides the version information necessary for subsequent conflict checking. See [How Consistency Checking Works for Replicated Regions](#topic_C5B74CCDD909403C815639339AA03758).
 
-Client caches also perform consistency checking in the same way when they receive an update for a region entry. However, all region operations that originate in the client cache are first passed onto an available Geode server, which generates the version information necessary for subsequent conflict checking.
+Client caches also perform consistency checking in the same way when they receive an update for a region entry. However, all region operations that originate in the client cache are first passed onto an available <%=vars.product_name%> server, which generates the version information necessary for subsequent conflict checking.
 
 ## <a id="topic_B64891585E7F4358A633C792F10FA23E" class="no-quick-link"></a>Configuring Consistency Checking
 
-Geode enables consistency checking by default. You cannot disable consistency checking for persistent regions. For all other regions, you can explicitly enable or disable consistency checking by setting the `concurrency-checks-enabled` region attribute in `cache.xml` to "true" or "false."
+<%=vars.product_name%> enables consistency checking by default. You cannot disable consistency checking for persistent regions. For all other regions, you can explicitly enable or disable consistency checking by setting the `concurrency-checks-enabled` region attribute in `cache.xml` to "true" or "false."
 
-All Geode members that host a region must use the same `concurrency-checks-enabled` setting for that region.
+All <%=vars.product_name%> members that host a region must use the same `concurrency-checks-enabled` setting for that region.
 
 A client cache can disable consistency checking for a region even if server caches enable consistency checking for the same region. This configuration ensures that the client sees all events for the region, but it does not prevent the client cache region from becoming out-of-sync with the server cache.
 
@@ -65,21 +65,21 @@ If you cannot support the additional overhead in your deployment, you can disabl
 
 ## <a id="topic_C5B74CCDD909403C815639339AA03758" class="no-quick-link"></a>How Consistency Checking Works for Replicated Regions
 
-Each region stores version and timestamp information for use in conflict detection. Geode members use the recorded information to detect and resolve conflicts consistently before applying a distributed update.
+Each region stores version and timestamp information for use in conflict detection. <%=vars.product_name%> members use the recorded information to detect and resolve conflicts consistently before applying a distributed update.
 
 <a id="topic_C5B74CCDD909403C815639339AA03758__section_763B071061C94D1E82E8883325294547"></a>
-By default, each entry in a region stores the ID of the Geode member that last updated the entry, as well as a version stamp for the entry that is incremented each time an update occurs. The version information is stored in each local entry, and the version stamp is distributed to other Geode members when the local entry is updated.
+By default, each entry in a region stores the ID of the <%=vars.product_name%> member that last updated the entry, as well as a version stamp for the entry that is incremented each time an update occurs. The version information is stored in each local entry, and the version stamp is distributed to other <%=vars.product_name%> members when the local entry is updated.
 
-A Geode member or client that receives an update message first compares the update version stamp with the version stamp recorded in its local cache. If the update version stamp is larger, it represents a newer version of the entry, so the receiving member applies the update locally and updates the version information. A smaller update version stamp indicates an out-of-order update, which is discarded.
+A <%=vars.product_name%> member or client that receives an update message first compares the update version stamp with the version stamp recorded in its local cache. If the update version stamp is larger, it represents a newer version of the entry, so the receiving member applies the update locally and updates the version information. A smaller update version stamp indicates an out-of-order update, which is discarded.
 
-An identical version stamp indicates that multiple Geode members updated the same entry at the same time. To resolve a concurrent update, a Geode member always applies (or keeps) the region entry that has the highest membership ID; the region entry having the lower membership ID is discarded.
+An identical version stamp indicates that multiple <%=vars.product_name%> members updated the same entry at the same time. To resolve a concurrent update, a <%=vars.product_name%> member always applies (or keeps) the region entry that has the highest membership ID; the region entry having the lower membership ID is discarded.
 
 **Note:**
-When a Geode member discards an update message (either for an out-of-order update or when resolving a concurrent update), it does not pass the discarded event to an event listener for the region. You can track the number of discarded updates for each member using the `conflatedEvents` statistic. See [Geode Statistics List](../../reference/statistics_list.html#statistics_list). Some members may discard an update while other members apply the update, depending on the order in which each member receives the update. For this reason, the `conflatedEvents` statistic differs for each Geode member. The example below describes this behavior in more detail.
+When a <%=vars.product_name%> member discards an update message (either for an out-of-order update or when resolving a concurrent update), it does not pass the discarded event to an event listener for the region. You can track the number of discarded updates for each member using the `conflatedEvents` statistic. See [<%=vars.product_name%> Statistics List](../../reference/statistics_list.html#statistics_list). Some members may discard an update while other members apply the update, depending on the order in which each member receives the update. For this reason, the `conflatedEvents` statistic differs for each <%=vars.product_name%> member. The example below describes this behavior in more detail.
 
-The following example shows how a concurrent update is handled in a distributed system of three Geode members. Assume that Members A, B, and C have membership IDs of 1, 2, and 3, respectively. Each member currently stores an entry, X, in their caches at version C2 (the entry was last updated by member C):
+The following example shows how a concurrent update is handled in a distributed system of three <%=vars.product_name%> members. Assume that Members A, B, and C have membership IDs of 1, 2, and 3, respectively. Each member currently stores an entry, X, in their caches at version C2 (the entry was last updated by member C):
 
-**Step 1:** An application updates entry X on Geode member A at the same time another application updates entry X on member C. Each member increments the version stamp for the entry and records the version stamp with their member ID in their local caches. In this case the entry was originally at version C2, so each member updates the version to 3 (A3 and C3, respectively) in their local caches.
+**Step 1:** An application updates entry X on <%=vars.product_name%> member A at the same time another application updates entry X on member C. Each member increments the version stamp for the entry and records the version stamp with their member ID in their local caches. In this case the entry was originally at version C2, so each member updates the version to 3 (A3 and C3, respectively) in their local caches.
 
 <img src="../../images_svg/region_entry_versions_1.svg" id="topic_C5B74CCDD909403C815639339AA03758__image_nt5_ptw_4r" class="image" />
 
@@ -101,27 +101,27 @@ At this point, all members that host the region have achieved a consistent state
 
 ## <a id="topic_321B05044B6641FCAEFABBF5066BD399" class="no-quick-link"></a>How Destroy and Clear Operations Are Resolved
 
-When consistency checking is enabled for a region, a Geode member does not immediately remove an entry from the region when an application destroys the entry. Instead, the member retains the entry with its current version stamp for a period of time in order to detect possible conflicts with operations that have occurred. The retained entry is referred to as a *tombstone*. Geode retains tombstones for partitioned regions and non-replicated regions as well as for replicated regions, in order to provide consistency.
+When consistency checking is enabled for a region, a <%=vars.product_name%> member does not immediately remove an entry from the region when an application destroys the entry. Instead, the member retains the entry with its current version stamp for a period of time in order to detect possible conflicts with operations that have occurred. The retained entry is referred to as a *tombstone*. <%=vars.product_name%> retains tombstones for partitioned regions and non-replicated regions as well as for replicated regions, in order to provide consistency.
 
 A tombstone in a client cache or a non-replicated region expires after 8 minutes, at which point the tombstone is immediately removed from the cache.
 
-A tombstone for a replicated or partitioned region expires after 10 minutes. Expired tombstones are eligible for garbage collection by the Geode member. Garbage collection is automatically triggered after 100,000 tombstones of any type have timed out in the local Geode member. You can optionally set the `gemfire.tombstone-gc-threshold` property to a value smaller than 100000 to perform garbage collection more frequently.
+A tombstone for a replicated or partitioned region expires after 10 minutes. Expired tombstones are eligible for garbage collection by the <%=vars.product_name%> member. Garbage collection is automatically triggered after 100,000 tombstones of any type have timed out in the local <%=vars.product_name%> member. You can optionally set the `gemfire.tombstone-gc-threshold` property to a value smaller than 100000 to perform garbage collection more frequently.
 
 **Note:**
-To avoid out-of-memory errors, a Geode member also initiates garbage collection for tombstones when the amount of free memory drops below 30 percent of total memory.
+To avoid out-of-memory errors, a <%=vars.product_name%> member also initiates garbage collection for tombstones when the amount of free memory drops below 30 percent of total memory.
 
-You can monitor the total number of tombstones in a cache using the `tombstoneCount` statistic in `CachePerfStats`. The `tombstoneGCCount` statistic records the total number of tombstone garbage collection cycles that a member has performed. `replicatedTombstonesSize` and `nonReplicatedTombstonesSize` show the approximate number of bytes that are currently consumed by tombstones in replicated or partitioned regions, and in non-replicated regions, respectively. See [Geode Statistics List](../../reference/statistics_list.html#statistics_list).
+You can monitor the total number of tombstones in a cache using the `tombstoneCount` statistic in `CachePerfStats`. The `tombstoneGCCount` statistic records the total number of tombstone garbage collection cycles that a member has performed. `replicatedTombstonesSize` and `nonReplicatedTombstonesSize` show the approximate number of bytes that are currently consumed by tombstones in replicated or partitioned regions, and in non-replicated regions, respectively. See [<%=vars.product_name%> Statistics List](../../reference/statistics_list.html#statistics_list).
 
 ## <a id="topic_321B05044B6641FCAEFABBF5066BD399__section_4D0140E96A3141EB8D983D0A43464097" class="no-quick-link"></a>About Region.clear() Operations
 
-Region entry version stamps and tombstones ensure consistency only when individual entries are destroyed. A `Region.clear()` operation, however, operates on all entries in a region at once. To provide consistency for `Region.clear()` operations, Geode obtains a distributed read/write lock for the region, which blocks all concurrent updates to the region. Any updates that were initiated before the clear operation are allowed to complete before the region is cleared.
+Region entry version stamps and tombstones ensure consistency only when individual entries are destroyed. A `Region.clear()` operation, however, operates on all entries in a region at once. To provide consistency for `Region.clear()` operations, <%=vars.product_name%> obtains a distributed read/write lock for the region, which blocks all concurrent updates to the region. Any updates that were initiated before the clear operation are allowed to complete before the region is cleared.
 
 ## <a id="topic_32ACFA5542C74F3583ECD30467F352B0" class="no-quick-link"></a>Transactions with Consistent Regions
 
 A transaction that modifies a region having consistency checking enabled generates all necessary version information for region updates when the transaction commits.
 
-If a transaction modifies a normal, preloaded or empty region, the transaction is first delegated to a Geode member that holds a replicate for the region. This behavior is similar to the transactional behavior for partitioned regions, where the partitioned region transaction is forwarded to a member that hosts the primary for the partitioned region update.
+If a transaction modifies a normal, preloaded or empty region, the transaction is first delegated to a <%=vars.product_name%> member that holds a replicate for the region. This behavior is similar to the transactional behavior for partitioned regions, where the partitioned region transaction is forwarded to a member that hosts the primary for the partitioned region update.
 
-The limitation for transactions on normal, preloaded or or empty regions is that, when consistency checking is enabled, a transaction cannot perform a `localDestroy` or `localInvalidate` operation against the region. Geode throws an `UnsupportedOperationInTransactionException` exception in such cases. An application should use a `Destroy` or `Invalidate` operation in place of a `localDestroy` or `localInvalidate` when consistency checks are enabled.
+The limitation for transactions on normal, preloaded or or empty regions is that, when consistency checking is enabled, a transaction cannot perform a `localDestroy` or `localInvalidate` operation against the region. <%=vars.product_name%> throws an `UnsupportedOperationInTransactionException` exception in such cases. An application should use a `Destroy` or `Invalidate` operation in place of a `localDestroy` or `localInvalidate` when consistency checks are enabled.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/distributed_regions/how_region_versioning_works_wan.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/distributed_regions/how_region_versioning_works_wan.html.md.erb b/geode-docs/developing/distributed_regions/how_region_versioning_works_wan.html.md.erb
index 275d496..b939ea8 100644
--- a/geode-docs/developing/distributed_regions/how_region_versioning_works_wan.html.md.erb
+++ b/geode-docs/developing/distributed_regions/how_region_versioning_works_wan.html.md.erb
@@ -19,21 +19,21 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-When two or more Geode systems are configured to distribute events over a WAN, each system performs local consistency checking before it distributes an event to a configured gateway sender. Discarded events are not distributed across the WAN.
+When two or more <%=vars.product_name%> systems are configured to distribute events over a WAN, each system performs local consistency checking before it distributes an event to a configured gateway sender. Discarded events are not distributed across the WAN.
 
-Regions can also be configured to distribute updates to other Geode clusters over a WAN. With a distributed WAN configuration, multiple gateway senders asynchronously queue and send region updates to another Geode cluster. It is possible for multiple sites to send updates to the same region entry at the same time. It is also possible that, due to a slow WAN connection, a cluster might receive region updates after a considerable delay, and after it has applied more recent updates to a region. To ensure that WAN-replicated regions eventually reach a consistent state, Geode first ensures that each cluster performs consistency checking to regions before queuing updates to a gateway sender for WAN distribution. In order words, region conflicts are first detected and resolved in the local cluster, using the techniques described in the previous sections.
+Regions can also be configured to distribute updates to other <%=vars.product_name%> clusters over a WAN. With a distributed WAN configuration, multiple gateway senders asynchronously queue and send region updates to another <%=vars.product_name%> cluster. It is possible for multiple sites to send updates to the same region entry at the same time. It is also possible that, due to a slow WAN connection, a cluster might receive region updates after a considerable delay, and after it has applied more recent updates to a region. To ensure that WAN-replicated regions eventually reach a consistent state, <%=vars.product_name%> first ensures that each cluster performs consistency checking to regions before queuing updates to a gateway sender for WAN distribution. In order words, region conflicts are first detected and resolved in the local cluster, using the techniques described in the previous sections.
 
-When a Geode cluster in a WAN configuration receives a distributed update, conflict checking is performed to ensure that all sites apply updates in the same way. This ensures that regions eventually reach a consistent state across all Geode clusters. The default conflict checking behavior for WAN-replicated regions is summarized as follows:
+When a <%=vars.product_name%> cluster in a WAN configuration receives a distributed update, conflict checking is performed to ensure that all sites apply updates in the same way. This ensures that regions eventually reach a consistent state across all <%=vars.product_name%> clusters. The default conflict checking behavior for WAN-replicated regions is summarized as follows:
 
--   If an update is received from the same Geode cluster that last updated the region entry, then there is no conflict and the update is applied.
--   If an update is received from a different Geode cluster than the one that last updated the region entry, then a potential conflict exists. A cluster applies the update only when the update has a timestamp that is later than the timestamp currently recorded in the cache.
+-   If an update is received from the same <%=vars.product_name%> cluster that last updated the region entry, then there is no conflict and the update is applied.
+-   If an update is received from a different <%=vars.product_name%> cluster than the one that last updated the region entry, then a potential conflict exists. A cluster applies the update only when the update has a timestamp that is later than the timestamp currently recorded in the cache.
 
 **Note:**
-If you use the default conflict checking feature for WAN deployments, you must ensure that all Geode members in all clusters synchronize their system clocks. For example, use a common NTP server for all Geode members that participate in a WAN deployment.
+If you use the default conflict checking feature for WAN deployments, you must ensure that all <%=vars.product_name%> members in all clusters synchronize their system clocks. For example, use a common NTP server for all <%=vars.product_name%> members that participate in a WAN deployment.
 
 As an alternative to the default conflict checking behavior for WAN deployments, you can develop and deploy a custom conflict resolver for handling region events that are distributed over a WAN. Using a custom resolver enables you to handle conflicts using criteria other than, or in addition to, timestamp information. For example, you might always prioritize updates that originate from a particular site, given that the timestamp value is within a certain range.
 
-When a gateway sender distributes an event to another Geode site, it adds the distributed system ID of the local cluster, as well as a timestamp for the event. In a default configuration, the cluster that receives the event examines the timestamp to determine whether or not the event should be applied. If the timestamp of the update is earlier than the local timestamp, the cluster discards the event. If the timestamp is the same as the local timestamp, then the entry having the highest distributed system ID is applied (or kept).
+When a gateway sender distributes an event to another <%=vars.product_name%> site, it adds the distributed system ID of the local cluster, as well as a timestamp for the event. In a default configuration, the cluster that receives the event examines the timestamp to determine whether or not the event should be applied. If the timestamp of the update is earlier than the local timestamp, the cluster discards the event. If the timestamp is the same as the local timestamp, then the entry having the highest distributed system ID is applied (or kept).
 
 You can override the default consistency checking for WAN events by installing a conflict resolver plug-in for the region. If a conflict resolver is installed, then any event that can potentially cause a conflict (any event that originated from a different distributed system ID than the ID that last modified the entry) is delivered to the conflict resolver. The resolver plug-in then makes the sole determination for which update to apply or keep.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/distributed_regions/locking_in_global_regions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/distributed_regions/locking_in_global_regions.html.md.erb b/geode-docs/developing/distributed_regions/locking_in_global_regions.html.md.erb
index 2dc0c8a..313a19b 100644
--- a/geode-docs/developing/distributed_regions/locking_in_global_regions.html.md.erb
+++ b/geode-docs/developing/distributed_regions/locking_in_global_regions.html.md.erb
@@ -26,7 +26,7 @@ In regions with global scope, locking helps ensure cache consistency.
 
 Locking of regions and entries is done in two ways:
 
-1.  **Implicit**. Geode automatically locks global regions and their data entries during most operations. Region invalidation and destruction do not acquire locks.
+1.  **Implicit**. <%=vars.product_name%> automatically locks global regions and their data entries during most operations. Region invalidation and destruction do not acquire locks.
 2.  **Explicit**. You can use the API to explicitly lock the region and its entries. Do this to guarantee atomicity in tasks with multi-step distributed operations. The `Region` methods `org.apache.geode.cache.Region.getDistributedLock` and `org.apache.geode.cache.Region.getRegionDistributedLock` return instances of `java.util.concurrent.locks.Lock` for a region and a specified key.
 
     **Note:**

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/distributed_regions/managing_distributed_regions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/distributed_regions/managing_distributed_regions.html.md.erb b/geode-docs/developing/distributed_regions/managing_distributed_regions.html.md.erb
index 7d4cf37..0149d96 100644
--- a/geode-docs/developing/distributed_regions/managing_distributed_regions.html.md.erb
+++ b/geode-docs/developing/distributed_regions/managing_distributed_regions.html.md.erb
@@ -57,7 +57,7 @@ Before you begin, understand [Basic Configuration and Programming](../../basic_c
     </region-attributes>
     ```
 
-4.  If you are using `global` scope, program any explicit locking you need in addition to the automated locking provided by Geode.
+4.  If you are using `global` scope, program any explicit locking you need in addition to the automated locking provided by <%=vars.product_name%>.
 
 ## <a id="configure_distributed_region__section_6F53FB58B8A84D0F8086AFDB08A649F9" class="no-quick-link"></a>Local Destroy and Invalidate in the Replicated Region
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/distributed_regions/region_entry_versions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/distributed_regions/region_entry_versions.html.md.erb b/geode-docs/developing/distributed_regions/region_entry_versions.html.md.erb
index 9aa8dde..53d1483 100644
--- a/geode-docs/developing/distributed_regions/region_entry_versions.html.md.erb
+++ b/geode-docs/developing/distributed_regions/region_entry_versions.html.md.erb
@@ -22,29 +22,29 @@ limitations under the License.
 <a id="topic_CF2798D3E12647F182C2CEC4A46E2045"></a>
 
 
-Geode ensures that all copies of a region eventually reach a consistent state on all members and clients that host the region, including Geode members that distribute region events.
+<%=vars.product_name%> ensures that all copies of a region eventually reach a consistent state on all members and clients that host the region, including <%=vars.product_name%> members that distribute region events.
 
--   **[Consistency Checking by Region Type](../../developing/distributed_regions/how_region_versioning_works.html#topic_7A4B6C6169BD4B1ABD356294F744D236)**
+-   **[Consistency Checking by Region Type](how_region_versioning_works.html#topic_7A4B6C6169BD4B1ABD356294F744D236)**
 
-    Geode performs different consistency checks depending on the type of region you have configured.
+    <%=vars.product_name%> performs different consistency checks depending on the type of region you have configured.
 
--   **[Configuring Consistency Checking](../../developing/distributed_regions/how_region_versioning_works.html#topic_B64891585E7F4358A633C792F10FA23E)**
+-   **[Configuring Consistency Checking](how_region_versioning_works.html#topic_B64891585E7F4358A633C792F10FA23E)**
 
-    Geode enables consistency checking by default. You cannot disable consistency checking for persistent regions. For all other regions, you can explicitly enable or disable consistency checking by setting the `concurrency-checks-enabled` region attribute in `cache.xml` to "true" or "false."
+    <%=vars.product_name%> enables consistency checking by default. You cannot disable consistency checking for persistent regions. For all other regions, you can explicitly enable or disable consistency checking by setting the `concurrency-checks-enabled` region attribute in `cache.xml` to "true" or "false."
 
--   **[Overhead for Consistency Checks](../../developing/distributed_regions/how_region_versioning_works.html#topic_0BDACA590B2C4974AC9C450397FE70B2)**
+-   **[Overhead for Consistency Checks](how_region_versioning_works.html#topic_0BDACA590B2C4974AC9C450397FE70B2)**
 
     Consistency checking requires additional overhead for storing and distributing version and timestamp information, as well as for maintaining destroyed entries for a period of time to meet consistency requirements.
 
--   **[How Consistency Checking Works for Replicated Regions](../../developing/distributed_regions/how_region_versioning_works.html#topic_C5B74CCDD909403C815639339AA03758)**
+-   **[How Consistency Checking Works for Replicated Regions](how_region_versioning_works.html#topic_C5B74CCDD909403C815639339AA03758)**
 
-    Each region stores version and timestamp information for use in conflict detection. Geode members use the recorded information to detect and resolve conflicts consistently before applying a distributed update.
+    Each region stores version and timestamp information for use in conflict detection. <%=vars.product_name%> members use the recorded information to detect and resolve conflicts consistently before applying a distributed update.
 
--   **[How Destroy and Clear Operations Are Resolved](../../developing/distributed_regions/how_region_versioning_works.html#topic_321B05044B6641FCAEFABBF5066BD399)**
+-   **[How Destroy and Clear Operations Are Resolved](how_region_versioning_works.html#topic_321B05044B6641FCAEFABBF5066BD399)**
 
-    When consistency checking is enabled for a region, a Geode member does not immediately remove an entry from the region when an application destroys the entry. Instead, the member retains the entry with its current version stamp for a period of time in order to detect possible conflicts with operations that have occurred. The retained entry is referred to as a *tombstone*. Geode retains tombstones for partitioned regions and non-replicated regions as well as for replicated regions, in order to provide consistency.
+    When consistency checking is enabled for a region, a <%=vars.product_name%> member does not immediately remove an entry from the region when an application destroys the entry. Instead, the member retains the entry with its current version stamp for a period of time in order to detect possible conflicts with operations that have occurred. The retained entry is referred to as a *tombstone*. <%=vars.product_name%> retains tombstones for partitioned regions and non-replicated regions as well as for replicated regions, in order to provide consistency.
 
--   **[Transactions with Consistent Regions](../../developing/distributed_regions/how_region_versioning_works.html#topic_32ACFA5542C74F3583ECD30467F352B0)**
+-   **[Transactions with Consistent Regions](how_region_versioning_works.html#topic_32ACFA5542C74F3583ECD30467F352B0)**
 
     A transaction that modifies a region having consistency checking enabled generates all necessary version information for region updates when the transaction commits.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/chapter_overview.html.md.erb b/geode-docs/developing/events/chapter_overview.html.md.erb
index f5b46f4..1a26c08 100644
--- a/geode-docs/developing/events/chapter_overview.html.md.erb
+++ b/geode-docs/developing/events/chapter_overview.html.md.erb
@@ -19,26 +19,26 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode provides versatile and reliable event distribution and handling for your cached data and system member events.
+<%=vars.product_name%> provides versatile and reliable event distribution and handling for your cached data and system member events.
 
--   **[How Events Work](../../developing/events/how_events_work.html)**
+-   **[How Events Work](how_events_work.html)**
 
-    Members in your Geode distributed system receive cache updates from other members through cache events. The other members can be peers to the member, clients or servers or other distributed systems.
+    Members in your <%=vars.product_name%> distributed system receive cache updates from other members through cache events. The other members can be peers to the member, clients or servers or other distributed systems.
 
--   **[Implementing Geode Event Handlers](../../developing/events/event_handler_overview.html)**
+-   **[Implementing <%=vars.product_name%> Event Handlers](event_handler_overview.html)**
 
     You can specify event handlers for region and region entry operations and for administrative events.
 
--   **[Configuring Peer-to-Peer Event Messaging](../../developing/events/configure_p2p_event_messaging.html)**
+-   **[Configuring Peer-to-Peer Event Messaging](configure_p2p_event_messaging.html)**
 
     You can receive events from distributed system peers for any region that is not a local region. Local regions receive only local cache events.
 
--   **[Configuring Client/Server Event Messaging](../../developing/events/configure_client_server_event_messaging.html)**
+-   **[Configuring Client/Server Event Messaging](configure_client_server_event_messaging.html)**
 
     You can receive events from your servers for server-side cache events and query result changes.
 
--   **[Configuring Multi-Site (WAN) Event Queues](../../developing/events/configure_multisite_event_messaging.html)**
+-   **[Configuring Multi-Site (WAN) Event Queues](configure_multisite_event_messaging.html)**
 
-    In a multi-site (WAN) installation, Geode uses gateway sender queues to distribute events for regions that are configured with a gateway sender. AsyncEventListeners also use an asynchronous event queue to distribute events for configured regions. This section describes additional options for configuring the event queues that are used by gateway senders or AsyncEventListener implementations.
+    In a multi-site (WAN) installation, <%=vars.product_name%> uses gateway sender queues to distribute events for regions that are configured with a gateway sender. AsyncEventListeners also use an asynchronous event queue to distribute events for configured regions. This section describes additional options for configuring the event queues that are used by gateway senders or AsyncEventListener implementations.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/configure_client_server_event_messaging.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/configure_client_server_event_messaging.html.md.erb b/geode-docs/developing/events/configure_client_server_event_messaging.html.md.erb
index ebd4a3a..77701bd 100644
--- a/geode-docs/developing/events/configure_client_server_event_messaging.html.md.erb
+++ b/geode-docs/developing/events/configure_client_server_event_messaging.html.md.erb
@@ -72,10 +72,10 @@ To receive entry events in the client from the server:
     4.  To have events enqueued for your clients during client downtime, configure durable client/server messaging.
     5.  Write any continuous queries (CQs) that you want to run to receive continuously streaming updates to client queries. CQ events do not update the client cache. If you have dependencies between CQs and/or interest registrations, so that you want the two types of subscription events to arrive as closely together on the client, use a single server pool for everything. Using different pools can lead to time differences in the delivery of events because the pools might use different servers to process and deliver the event messages.
 
--   **[Configuring Highly Available Servers](../../developing/events/configuring_highly_available_servers.html)**
+-   **[Configuring Highly Available Servers](configuring_highly_available_servers.html)**
 
--   **[Implementing Durable Client/Server Messaging](../../developing/events/implementing_durable_client_server_messaging.html)**
+-   **[Implementing Durable Client/Server Messaging](implementing_durable_client_server_messaging.html)**
 
--   **[Tuning Client/Server Event Messaging](../../developing/events/tune_client_server_event_messaging.html)**
+-   **[Tuning Client/Server Event Messaging](tune_client_server_event_messaging.html)**
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/configure_multisite_event_messaging.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/configure_multisite_event_messaging.html.md.erb b/geode-docs/developing/events/configure_multisite_event_messaging.html.md.erb
index 5756652..9fb887a 100644
--- a/geode-docs/developing/events/configure_multisite_event_messaging.html.md.erb
+++ b/geode-docs/developing/events/configure_multisite_event_messaging.html.md.erb
@@ -19,20 +19,20 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-In a multi-site (WAN) installation, Geode uses gateway sender queues to distribute events for regions that are configured with a gateway sender. AsyncEventListeners also use an asynchronous event queue to distribute events for configured regions. This section describes additional options for configuring the event queues that are used by gateway senders or AsyncEventListener implementations.
+In a multi-site (WAN) installation, <%=vars.product_name%> uses gateway sender queues to distribute events for regions that are configured with a gateway sender. AsyncEventListeners also use an asynchronous event queue to distribute events for configured regions. This section describes additional options for configuring the event queues that are used by gateway senders or AsyncEventListener implementations.
 
 <a id="configure_multisite_event_messaging__section_1BBF77E166E84F7CA110385FD03D8453"></a>
 Before you begin, set up your multi-site (WAN) installation or configure asynchronous event queues and AsyncEventListener implementations. See [Configuring a Multi-site (WAN) System](../../topologies_and_comm/multi_site_configuration/setting_up_a_multisite_system.html#setting_up_a_multisite_system) or [Implementing an AsyncEventListener for Write-Behind Cache Event Handling](implementing_write_behind_event_handler.html#implementing_write_behind_cache_event_handling).
 
--   **[Persisting an Event Queue](../../developing/events/configuring_highly_available_gateway_queues.html)**
+-   **[Persisting an Event Queue](configuring_highly_available_gateway_queues.html)**
 
     You can configure a gateway sender queue or an asynchronous event queue to persist data to disk similar to the way in which replicated regions are persisted.
 
--   **[Configuring Dispatcher Threads and Order Policy for Event Distribution](../../developing/events/configuring_gateway_concurrency_levels.html)**
+-   **[Configuring Dispatcher Threads and Order Policy for Event Distribution](configuring_gateway_concurrency_levels.html)**
 
-    By default, Geode uses multiple dispatcher threads to process region events simultaneously in a gateway sender queue for distribution between sites, or in an asynchronous event queue for distributing events for write-behind caching. With serial queues, you can also configure the ordering policy for dispatching those events.
+    By default, <%=vars.product_name%> uses multiple dispatcher threads to process region events simultaneously in a gateway sender queue for distribution between sites, or in an asynchronous event queue for distributing events for write-behind caching. With serial queues, you can also configure the ordering policy for dispatching those events.
 
--   **[Conflating Events in a Queue](../../developing/events/conflate_multisite_gateway_queue.html)**
+-   **[Conflating Events in a Queue](conflate_multisite_gateway_queue.html)**
 
     Conflating a queue improves distribution performance. When conflation is enabled, only the latest queued value is sent for a particular key.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/configuring_gateway_concurrency_levels.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/configuring_gateway_concurrency_levels.html.md.erb b/geode-docs/developing/events/configuring_gateway_concurrency_levels.html.md.erb
index e59d3b4..f637064 100644
--- a/geode-docs/developing/events/configuring_gateway_concurrency_levels.html.md.erb
+++ b/geode-docs/developing/events/configuring_gateway_concurrency_levels.html.md.erb
@@ -19,9 +19,9 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-By default, Geode uses multiple dispatcher threads to process region events simultaneously in a gateway sender queue for distribution between sites, or in an asynchronous event queue for distributing events for write-behind caching. With serial queues, you can also configure the ordering policy for dispatching those events.
+By default, <%=vars.product_name%> uses multiple dispatcher threads to process region events simultaneously in a gateway sender queue for distribution between sites, or in an asynchronous event queue for distributing events for write-behind caching. With serial queues, you can also configure the ordering policy for dispatching those events.
 
-By default, a gateway sender queue or asynchronous event queue uses 5 dispatcher threads per queue. This provides support for applications that have the ability to process queued events concurrently for distribution to another Geode site or listener. If your application does not require concurrent distribution, or if you do not have enough resources to support the requirements of multiple dispatcher threads, then you can configure a single dispatcher thread to process a queue.
+By default, a gateway sender queue or asynchronous event queue uses 5 dispatcher threads per queue. This provides support for applications that have the ability to process queued events concurrently for distribution to another <%=vars.product_name%> site or listener. If your application does not require concurrent distribution, or if you do not have enough resources to support the requirements of multiple dispatcher threads, then you can configure a single dispatcher thread to process a queue.
 
 -   [Using Multiple Dispatcher Threads to Process a Queue](configuring_gateway_concurrency_levels.html#concept_6C52A037E39E4FD6AE4C6A982A4A1A85__section_20E8EFCE89EB4DC7AA822D03C8E0F470)
 -   [Performance and Memory Considerations](configuring_gateway_concurrency_levels.html#concept_6C52A037E39E4FD6AE4C6A982A4A1A85__section_C4C83B5C0FDD4913BA128365EE7E4E35)
@@ -30,9 +30,9 @@ By default, a gateway sender queue or asynchronous event queue uses 5 dispatcher
 
 ## <a id="concept_6C52A037E39E4FD6AE4C6A982A4A1A85__section_20E8EFCE89EB4DC7AA822D03C8E0F470" class="no-quick-link"></a>Using Multiple Dispatcher Threads to Process a Queue
 
-When multiple dispatcher threads are configured for a parallel queue, Geode simply uses multiple threads to process the contents of each individual queue. The total number of queues that are created is still determined by the number of Geode members that host the region.
+When multiple dispatcher threads are configured for a parallel queue, <%=vars.product_name%> simply uses multiple threads to process the contents of each individual queue. The total number of queues that are created is still determined by the number of <%=vars.product_name%> members that host the region.
 
-When multiple dispatcher threads are configured for a serial queue, Geode creates an additional copy of the queue for each thread on each member that hosts the queue. To obtain the maximum throughput, increase the number of dispatcher threads until your network is saturated.
+When multiple dispatcher threads are configured for a serial queue, <%=vars.product_name%> creates an additional copy of the queue for each thread on each member that hosts the queue. To obtain the maximum throughput, increase the number of dispatcher threads until your network is saturated.
 
 The following diagram illustrates a serial gateway sender queue that is configured with multiple dispatcher threads.
 <img src="../../images/MultisiteConcurrency_WAN_Gateway.png" id="concept_6C52A037E39E4FD6AE4C6A982A4A1A85__image_093DAC58EBEE456485562C92CA79899F" class="image" width="624" />
@@ -48,8 +48,8 @@ When a serial gateway sender or an asynchronous event queue uses multiple dispat
 
 When using multiple `dispatcher-threads` (greater than 1) with a serial event queue, you can also configure the `order-policy` that those threads use to distribute events from the queue. The valid order policy values are:
 
--   **key (default)**. All updates to the same key are distributed in order. Geode preserves key ordering by placing all updates to the same key in the same dispatcher thread queue. You typically use key ordering when updates to entries have no relationship to each other, such as for an application that uses a single feeder to distribute stock updates to several other systems.
--   **thread**. All region updates from a given thread are distributed in order. Geode preserves thread ordering by placing all region updates from the same thread into the same dispatcher thread queue. In general, use thread ordering when updates to one region entry affect updates to another region entry.
+-   **key (default)**. All updates to the same key are distributed in order. <%=vars.product_name%> preserves key ordering by placing all updates to the same key in the same dispatcher thread queue. You typically use key ordering when updates to entries have no relationship to each other, such as for an application that uses a single feeder to distribute stock updates to several other systems.
+-   **thread**. All region updates from a given thread are distributed in order. <%=vars.product_name%> preserves thread ordering by placing all region updates from the same thread into the same dispatcher thread queue. In general, use thread ordering when updates to one region entry affect updates to another region entry.
 -   **partition**. All region events that share the same partitioning key are distributed in order. Specify partition ordering when applications use a [PartitionResolver](/releases/latest/javadoc/org/apache/geode/cache/PartitionResolver.html) to implement [custom partitioning](../partitioned_regions/using_custom_partition_resolvers.html). With partition ordering, all entries that share the same "partitioning key" (RoutingObject) are placed into the same dispatcher thread queue.
 
 You cannot configure the `order-policy` for a parallel event queue, because parallel queues cannot preserve event ordering for regions. Only the ordering of events for a given partition (or in a given queue of a distributed region) can be preserved.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/configuring_highly_available_gateway_queues.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/configuring_highly_available_gateway_queues.html.md.erb b/geode-docs/developing/events/configuring_highly_available_gateway_queues.html.md.erb
index 28339e2..3f570ef 100644
--- a/geode-docs/developing/events/configuring_highly_available_gateway_queues.html.md.erb
+++ b/geode-docs/developing/events/configuring_highly_available_gateway_queues.html.md.erb
@@ -23,7 +23,7 @@ You can configure a gateway sender queue or an asynchronous event queue to persi
 
 <a id="configuring_highly_available_gateway_queues__section_7EB2A7E38B074AAAA06D22C59687CB8A"></a>
 Persisting a queue provides high availability for the event messaging that the sender performs. For example, if a persistent gateway sender queue exits for any reason, when the member that hosts the sender restarts it automatically reloads the queue and resumes sending messages. If an asynchronous event queue exits for any reason, write-back caching can resume where it left off when the queue is brought back online.
-Geode persists an event queue if you set the `enable-persistence` attribute to true. The queue is persisted to the disk store specified in the queue's `disk-store-name` attribute, or to the default disk store if you do not specify a store name.
+<%=vars.product_name%> persists an event queue if you set the `enable-persistence` attribute to true. The queue is persisted to the disk store specified in the queue's `disk-store-name` attribute, or to the default disk store if you do not specify a store name.
 
 You must configure the event queue to use persistence if you are using persistent regions. The use of non-persistent event queues with persistent regions is not supported.
 
@@ -69,7 +69,7 @@ In the example below the gateway sender queue uses "diskStoreA" for persistence
     --maximum-queue-memory=100
     ```
 
-If you were to configure 10 dispatcher threads for the serial gateway sender, then the total maximum memory for the gateway sender queue would be 1000MB on each Geode member that hosted the sender, because Geode creates a separate copy of the queue per thread..
+If you were to configure 10 dispatcher threads for the serial gateway sender, then the total maximum memory for the gateway sender queue would be 1000MB on each <%=vars.product_name%> member that hosted the sender, because <%=vars.product_name%> creates a separate copy of the queue per thread..
 
 The following example shows a similar configuration for an asynchronous event queue:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/configuring_highly_available_servers.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/configuring_highly_available_servers.html.md.erb b/geode-docs/developing/events/configuring_highly_available_servers.html.md.erb
index 48cd174..d78221d 100644
--- a/geode-docs/developing/events/configuring_highly_available_servers.html.md.erb
+++ b/geode-docs/developing/events/configuring_highly_available_servers.html.md.erb
@@ -50,6 +50,6 @@ The following table describes the different values for the subscription-redundan
 | &gt; 0                  | Sets the precise number of secondary servers to use for backup to the primary. |
 | -1                      | Every server that is not the primary is to be used as a secondary.             |
 
--   **[Highly Available Client/Server Event Messaging](../../developing/events/ha_event_messaging_whats_next.html)**
+-   **[Highly Available Client/Server Event Messaging](ha_event_messaging_whats_next.html)**
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/event_handler_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/event_handler_overview.html.md.erb b/geode-docs/developing/events/event_handler_overview.html.md.erb
index 22a053f..868a063 100644
--- a/geode-docs/developing/events/event_handler_overview.html.md.erb
+++ b/geode-docs/developing/events/event_handler_overview.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Implementing Geode Event Handlers
----
+<% set_title("Implementing", product_name, "Event Handlers") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/filtering_multisite_events.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/filtering_multisite_events.html.md.erb b/geode-docs/developing/events/filtering_multisite_events.html.md.erb
index 505cd9c..3ee6ac5 100644
--- a/geode-docs/developing/events/filtering_multisite_events.html.md.erb
+++ b/geode-docs/developing/events/filtering_multisite_events.html.md.erb
@@ -19,21 +19,21 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-You can optionally create gateway sender and/or gateway receiver filters to control which events are queued and distributed to a remote site, or to modify the data stream that is transmitted between Geode sites.
+You can optionally create gateway sender and/or gateway receiver filters to control which events are queued and distributed to a remote site, or to modify the data stream that is transmitted between <%=vars.product_name%> sites.
 
 You can implement and deploy two different types of filter for multi-site events:
 
 -   `GatewayEventFilter`. A `GatewayEventFilter` implementation determines whether a region event is placed in a gateway sender queue and/or whether an event in a gateway queue is distributed to a remote site. You can optionally add one or more `GatewayEventFilter` implementations to a gateway sender, etiher in the `cache.xml` configuration file or using the Java API.
 
-    Geode makes a synchronous call to the filter's `beforeEnqueue` method before it places a region event in the gateway sender queue. The filter returns a boolean value that specifies whether the event should be added to the queue.
+    <%=vars.product_name%> makes a synchronous call to the filter's `beforeEnqueue` method before it places a region event in the gateway sender queue. The filter returns a boolean value that specifies whether the event should be added to the queue.
 
-    Geode asynchronously calls the filter's `beforeTransmit` method to determine whether the gateway sender dispatcher thread should distribute the event to a remote gateway receiver.
+    <%=vars.product_name%> asynchronously calls the filter's `beforeTransmit` method to determine whether the gateway sender dispatcher thread should distribute the event to a remote gateway receiver.
 
-    For events that are distributed to another site, Geode calls the listener's `afterAcknowledgement` method to indicate that is has received an ack from the remote site after the event was received.
+    For events that are distributed to another site, <%=vars.product_name%> calls the listener's `afterAcknowledgement` method to indicate that is has received an ack from the remote site after the event was received.
 
--   GatewayTransportFilter. Use a `GatewayTransportFilter` implementation to process the TCP stream that sends a batch of events that is distributed from one Geode cluster to another over a WAN. A `GatewayTransportFilter` is typically used to perform encryption or compression on the data that distributed. You install the same `GatewayTransportFilter` implementation on both a gateway sender and gateway receiver.
+-   GatewayTransportFilter. Use a `GatewayTransportFilter` implementation to process the TCP stream that sends a batch of events that is distributed from one <%=vars.product_name%> cluster to another over a WAN. A `GatewayTransportFilter` is typically used to perform encryption or compression on the data that distributed. You install the same `GatewayTransportFilter` implementation on both a gateway sender and gateway receiver.
 
-    When a gateway sender processes a batch of events for distribution, Geode delivers the stream to the `getInputStream` method of a configured `GatewayTransportFilter` implementation. The filter processes and returns the stream, which is then transmitted to the gateway receiver. When the gateway receiver receives the batch, Geode calls the `getOutputStream` method of a configured filter, which again processes and returns the stream so that the events can be applied in the local cluster.
+    When a gateway sender processes a batch of events for distribution, <%=vars.product_name%> delivers the stream to the `getInputStream` method of a configured `GatewayTransportFilter` implementation. The filter processes and returns the stream, which is then transmitted to the gateway receiver. When the gateway receiver receives the batch, <%=vars.product_name%> calls the `getOutputStream` method of a configured filter, which again processes and returns the stream so that the events can be applied in the local cluster.
 
 ## <a id="topic_E97BB68748F14987916CD1A50E4B4542__section_E20B4A8A98FD4EDAAA8C14B8059AA7F7" class="no-quick-link"></a>Configuring Multi-Site Event Filters
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/how_cache_events_work.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/how_cache_events_work.html.md.erb b/geode-docs/developing/events/how_cache_events_work.html.md.erb
index e54371a..4492092 100644
--- a/geode-docs/developing/events/how_cache_events_work.html.md.erb
+++ b/geode-docs/developing/events/how_cache_events_work.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-When a region or entry operation is performed, Geode distributes the associated events in the distributed system according to system and cache configurations.
+When a region or entry operation is performed, <%=vars.product_name%> distributes the associated events in the distributed system according to system and cache configurations.
 
 <a id="how_cache_events_work__section_7864A275FDB549FD8E2D046DD59CB9F4"></a>
 Install a cache listener for a region in each system member that needs to receive notification of region and entry changes.
@@ -68,4 +68,4 @@ In the following figure:
 
 ## <a id="how_cache_events_work__section_B4DCA51DDF7F44699E7355277172BEF0" class="no-quick-link"></a>Managing Events in Multi-threaded Applications
 
-For partitioned regions, Geode guarantees ordering of events across threads, but for distributed regions it doesn’t. For multi-threaded applications that create distributed regions, you need to use your application synchronization to make sure that one operation completes before the next one begins. Distribution through the distributed-no-ack queue can work with multiple threads if you set the `conserve-sockets` attribute to true. Then the threads share one queue, preserving the order of the events in distributed regions. Different threads can invoke the same listener, so if you allow different threads to send events, it can result in concurrent invocations of the listener. This is an issue only if the threads have some shared state - if they are incrementing a serial number, for example, or adding their events to a log queue. Then you need to make your code thread safe.
+For partitioned regions, <%=vars.product_name%> guarantees ordering of events across threads, but for distributed regions it doesn’t. For multi-threaded applications that create distributed regions, you need to use your application synchronization to make sure that one operation completes before the next one begins. Distribution through the distributed-no-ack queue can work with multiple threads if you set the `conserve-sockets` attribute to true. Then the threads share one queue, preserving the order of the events in distributed regions. Different threads can invoke the same listener, so if you allow different threads to send events, it can result in concurrent invocations of the listener. This is an issue only if the threads have some shared state - if they are incrementing a serial number, for example, or adding their events to a log queue. Then you need to make your code thread safe.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/how_client_server_distribution_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/how_client_server_distribution_works.html.md.erb b/geode-docs/developing/events/how_client_server_distribution_works.html.md.erb
index be7e4f2..88e7698 100644
--- a/geode-docs/developing/events/how_client_server_distribution_works.html.md.erb
+++ b/geode-docs/developing/events/how_client_server_distribution_works.html.md.erb
@@ -122,7 +122,7 @@ Once interest is registered, the server continually monitors region activities a
 
 ## <a id="how_client_server_distribution_works__section_928BB60066414BEB9FAA7FB3120334A3" class="no-quick-link"></a>Server Failover
 
-When a server hosting a subscription queue fails, the queueing responsibilities pass to another server. How this happens depends on whether the new server is a secondary server. In any case, all failover activities are carried out automatically by the Geode system.
+When a server hosting a subscription queue fails, the queueing responsibilities pass to another server. How this happens depends on whether the new server is a secondary server. In any case, all failover activities are carried out automatically by the <%=vars.product_name%> system.
 
 -   **Non-HA failover:** The client fails over without high availability if it is not configured for redundancy or if all secondaries also fail before new secondaries can be initialized. As soon as it can attach to a server, the client goes through an automatic reinitialization process. In this process, the failover code on the client side silently destroys all entries of interest to the client and refetches them from the new server, essentially reinitializing the client cache from the new server’s cache. For the notify all configuration, this clears and reloads all of the entries for the client regions that are connected to the server. For notify by subscription, it clears and reloads only the entries in the region interest lists. To reduce failover noise, the events caused by the local entry destruction and refetching are blocked by the failover code and do not reach the client cache listeners. Because of this, your clients could receive some out-of-sequence events during and af
 ter a server failover. For example, entries that exist on the failed server and not on its replacement are destroyed and never recreated during a failover. Because the destruction events are blocked, the client ends up with entries removed from its cache with no associated destroy events.
 -   **HA failover:** If your client pool is configured with redundancy and a secondary server is available at the time the primary fails, the failover is invisible to the client. The secondary server resumes queueing activities as soon as the primary loss is detected. The secondary might resend a few events, which are discarded automatically by the client message tracking activities.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/how_events_work.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/how_events_work.html.md.erb b/geode-docs/developing/events/how_events_work.html.md.erb
index 2ac899a..291bbb2 100644
--- a/geode-docs/developing/events/how_events_work.html.md.erb
+++ b/geode-docs/developing/events/how_events_work.html.md.erb
@@ -19,11 +19,11 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Members in your Geode distributed system receive cache updates from other members through cache events. The other members can be peers to the member, clients or servers or other distributed systems.
+Members in your <%=vars.product_name%> distributed system receive cache updates from other members through cache events. The other members can be peers to the member, clients or servers or other distributed systems.
 
 ## <a id="how_events_work__section_6C75098DDBB84944ADE57F2088330D5A" class="no-quick-link"></a>Events Features
 
-These are the primary features of Geode events:
+These are the primary features of <%=vars.product_name%> events:
 
 -   Content-based events
 -   Asynchronous event notifications with conflation
@@ -46,7 +46,7 @@ Both kinds of events can be generated by a single member operation.
 **Note:**
 You can handle one of these categories of events in a single system member. You cannot handle both cache and administrative events in a single member.
 
-Because Geode maintains the order of administrative events and the order of cache events separately, using cache events and administrative events in a single process can cause unexpected results.
+Because <%=vars.product_name%> maintains the order of administrative events and the order of cache events separately, using cache events and administrative events in a single process can cause unexpected results.
 
 ## <a id="how_events_work__section_4BCDB22AB927478EBF1035B0DE230DD3" class="no-quick-link"></a>Event Cycle
 
@@ -92,20 +92,20 @@ During a cache operation, event handlers are called at various stages of the ope
 **Note:**
 An `EntryEvent` contains both the old value and the new value of the entry, which helps to indicate the value that was replaced by the cache operation on a particular key.
 
--   **[Peer-to-Peer Event Distribution](../../developing/events/how_cache_events_work.html)**
+-   **[Peer-to-Peer Event Distribution](how_cache_events_work.html)**
 
-    When a region or entry operation is performed, Geode distributes the associated events in the distributed system according to system and cache configurations.
+    When a region or entry operation is performed, <%=vars.product_name%> distributes the associated events in the distributed system according to system and cache configurations.
 
--   **[Client-to-Server Event Distribution](../../developing/events/how_client_server_distribution_works.html)**
+-   **[Client-to-Server Event Distribution](how_client_server_distribution_works.html)**
 
     Clients and servers distribute events according to client activities and according to interest registered by the client in server-side cache changes.
 
--   **[Multi-Site (WAN) Event Distribution](../../developing/events/how_multisite_distribution_works.html)**
+-   **[Multi-Site (WAN) Event Distribution](how_multisite_distribution_works.html)**
 
-    Geode distributes a subset of cache events between distributed systems, with a minimum impact on each system's performance. Events are distributed only for regions that you configure to use a gateway sender for distribution.
+    <%=vars.product_name%> distributes a subset of cache events between distributed systems, with a minimum impact on each system's performance. Events are distributed only for regions that you configure to use a gateway sender for distribution.
 
--   **[List of Event Handlers and Events](../../developing/events/list_of_event_handlers_and_events.html)**
+-   **[List of Event Handlers and Events](list_of_event_handlers_and_events.html)**
 
-    Geode provides many types of events and event handlers to help you manage your different data and application needs.
+    <%=vars.product_name%> provides many types of events and event handlers to help you manage your different data and application needs.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/how_multisite_distribution_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/how_multisite_distribution_works.html.md.erb b/geode-docs/developing/events/how_multisite_distribution_works.html.md.erb
index 7678e7a..ee518e7 100644
--- a/geode-docs/developing/events/how_multisite_distribution_works.html.md.erb
+++ b/geode-docs/developing/events/how_multisite_distribution_works.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode distributes a subset of cache events between distributed systems, with a minimum impact on each system's performance. Events are distributed only for regions that you configure to use a gateway sender for distribution.
+<%=vars.product_name%> distributes a subset of cache events between distributed systems, with a minimum impact on each system's performance. Events are distributed only for regions that you configure to use a gateway sender for distribution.
 
 ## <a id="how_multisite_distribution_works__section_A16562611E094C88B12BC149D5EEEEBA" class="no-quick-link"></a>Queuing Events for Distribution
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/implementing_cache_event_handlers.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/implementing_cache_event_handlers.html.md.erb b/geode-docs/developing/events/implementing_cache_event_handlers.html.md.erb
index 0e639e7..7afcc4d 100644
--- a/geode-docs/developing/events/implementing_cache_event_handlers.html.md.erb
+++ b/geode-docs/developing/events/implementing_cache_event_handlers.html.md.erb
@@ -22,7 +22,7 @@ limitations under the License.
 Depending on your installation and configuration, cache events can come from local operations, peers, servers, and remote sites. Event handlers register their interest in one or more events and are notified when the events occur.
 
 <a id="implementing_cache_event_handlers__section_9286E8C6B3C54089888E1680B4F43692"></a>
-For each type of handler, Geode provides a convenience class with empty stubs for the interface callback methods.
+For each type of handler, <%=vars.product_name%> provides a convenience class with empty stubs for the interface callback methods.
 
 **Note:**
 Write-behind cache listeners are created by extending the `AsyncEventListener` interface, and they are configured with an `AsyncEventQueue` that you assign to one or more regions.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/implementing_durable_client_server_messaging.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/implementing_durable_client_server_messaging.html.md.erb b/geode-docs/developing/events/implementing_durable_client_server_messaging.html.md.erb
index 150a79d..a1d1792 100644
--- a/geode-docs/developing/events/implementing_durable_client_server_messaging.html.md.erb
+++ b/geode-docs/developing/events/implementing_durable_client_server_messaging.html.md.erb
@@ -54,7 +54,7 @@ Use one of the following methods:
 
 The `durable-client-id` indicates that the client is durable and gives the server an identifier to correlate the client to its durable messages. For a non-durable client, this id is an empty string. The ID can be any number that is unique among the clients attached to servers in the same distributed system.
 
-The `durable-client-timeout` tells the server how long to wait for client reconnect. When this timeout is reached, the server stops storing to the client's message queue and discards any stored messages. The default is 300 seconds. This is a tuning parameter. If you change it, take into account the normal activity of your application, the average size of your messages, and the level of risk you can handle, both in lost messages and in the servers' capacity to store enqueued messages. Assuming that no messages are being removed from the queue, how long can the server run before the queue reaches the maximum capacity? How many durable clients can the server handle? To assist with tuning, use the Geode message queue statistics for durable clients through the disconnect and reconnect cycles.
+The `durable-client-timeout` tells the server how long to wait for client reconnect. When this timeout is reached, the server stops storing to the client's message queue and discards any stored messages. The default is 300 seconds. This is a tuning parameter. If you change it, take into account the normal activity of your application, the average size of your messages, and the level of risk you can handle, both in lost messages and in the servers' capacity to store enqueued messages. Assuming that no messages are being removed from the queue, how long can the server run before the queue reaches the maximum capacity? How many durable clients can the server handle? To assist with tuning, use the <%=vars.product_name%> message queue statistics for durable clients through the disconnect and reconnect cycles.
 
 ## <a id="implementing_durable_client_server_messaging__section_BB5DCCE0582E4FE8B62DE473512FC704" class="no-quick-link"></a>Configure Durable Subscriptions and Continuous Queries
 
@@ -169,7 +169,7 @@ During initialization, the client cache is not blocked from doing operations, so
 -   Client cache operations by the application.
 -   Callbacks triggered by replaying old events from the queue
 
-Geode handles the conflicts between the application and interest registrations so they do not create cache update conflicts. But you must program your event handlers so they don't conflict with current operations. This is true for all event handlers, but it is especially important for those used in durable clients. Your handlers may receive events well after the fact and you must ensure your programming takes that into account.
+<%=vars.product_name%> handles the conflicts between the application and interest registrations so they do not create cache update conflicts. But you must program your event handlers so they don't conflict with current operations. This is true for all event handlers, but it is especially important for those used in durable clients. Your handlers may receive events well after the fact and you must ensure your programming takes that into account.
 
 This figure shows the three concurrent procedures during the initialization process. The application begins operations immediately on the client (step 1), while the client’s cache ready message (also step 1) triggers a series of queue operations on the servers (starting with step 2 on the primary server). At the same time, the client registers interest (step 2 on the client) and receives a response from the server. Message B2 applies to an entry in Region A, so the cache listener handles B2’s event. Because B2 comes before the marker, the client does not apply the update to the cache.
 


[02/11] geode git commit: GEODE-3169: Decoupling of DiskStore and backups This closes #715 * move backup logic away from DiskStore and into BackupManager * refactor code into smaller methods * improve test code clarity

Posted by zh...@apache.org.
GEODE-3169: Decoupling of DiskStore and backups
This closes #715
  * move backup logic away from DiskStore and into BackupManager
  * refactor code into smaller methods
  * improve test code clarity


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

Branch: refs/heads/feature/GEM-1601
Commit: 3bb6a2214d02fcb339ecba0d0645457d3926ab12
Parents: f38dff9
Author: Nick Reich <nr...@pivotal.io>
Authored: Tue Aug 8 11:30:17 2017 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Fri Aug 18 09:52:24 2017 -0700

----------------------------------------------------------------------
 .../admin/internal/FinishBackupRequest.java     |   2 +-
 .../admin/internal/PrepareBackupRequest.java    |   4 +-
 .../geode/internal/cache/BackupManager.java     | 603 +++++++++++++++++++
 .../geode/internal/cache/DiskStoreBackup.java   |   9 +-
 .../internal/cache/DiskStoreFactoryImpl.java    |   1 -
 .../geode/internal/cache/DiskStoreImpl.java     | 224 +------
 .../geode/internal/cache/GemFireCacheImpl.java  |   5 +-
 .../geode/internal/cache/InternalCache.java     |   1 -
 .../org/apache/geode/internal/cache/Oplog.java  |   1 +
 .../cache/PartitionedRegionDataStore.java       |   1 -
 .../cache/persistence/BackupManager.java        | 389 ------------
 .../internal/cache/xmlcache/CacheCreation.java  |   2 +-
 .../internal/beans/MemberMBeanBridge.java       |   6 +-
 .../geode/internal/cache/BackupDUnitTest.java   | 176 +++---
 .../geode/internal/cache/BackupJUnitTest.java   | 145 +++--
 .../cache/IncrementalBackupDUnitTest.java       |   3 +-
 .../BackupPrepareAndFinishMsgDUnitTest.java     | 548 ++++-------------
 ...ionedBackupPrepareAndFinishMsgDUnitTest.java |  28 +
 ...icateBackupPrepareAndFinishMsgDUnitTest.java |  28 +
 .../beans/DistributedSystemBridgeJUnitTest.java |   8 +-
 20 files changed, 935 insertions(+), 1249 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java b/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java
index f01666d..88f67bd 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java
@@ -99,7 +99,7 @@ public class FinishBackupRequest extends CliLegacyMessage {
       persistentIds = new HashSet<PersistentID>();
     } else {
       try {
-        persistentIds = cache.getBackupManager().finishBackup(targetDir, baselineDir, abort);
+        persistentIds = cache.getBackupManager().doBackup(targetDir, baselineDir, abort);
       } catch (IOException e) {
         logger.error(
             LocalizedMessage.create(LocalizedStrings.CliLegacyMessage_ERROR, this.getClass()), e);

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java b/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
index 0c096f9..ede70c1 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
@@ -37,7 +37,7 @@ import org.apache.geode.internal.admin.remote.AdminResponse;
 import org.apache.geode.internal.admin.remote.CliLegacyMessage;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.persistence.BackupManager;
+import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
@@ -87,7 +87,7 @@ public class PrepareBackupRequest extends CliLegacyMessage {
     } else {
       try {
         BackupManager manager = cache.startBackup(getSender());
-        persistentIds = manager.prepareBackup();
+        persistentIds = manager.prepareForBackup();
       } catch (IOException e) {
         logger.error(
             LocalizedMessage.create(LocalizedStrings.CliLegacyMessage_ERROR, this.getClass()), e);

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/BackupManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BackupManager.java b/geode-core/src/main/java/org/apache/geode/internal/cache/BackupManager.java
new file mode 100644
index 0000000..b7e0e47
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/BackupManager.java
@@ -0,0 +1,603 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.cache;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URL;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.InternalGemFireError;
+import org.apache.geode.cache.DiskStore;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.MembershipListener;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.internal.DeployedJar;
+import org.apache.geode.internal.JarDeployer;
+import org.apache.geode.internal.cache.persistence.BackupInspector;
+import org.apache.geode.internal.cache.persistence.RestoreScript;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.logging.LogService;
+
+/**
+ * This class manages the state an logic to backup a single cache.
+ */
+public class BackupManager implements MembershipListener {
+  private static final Logger logger = LogService.getLogger(BackupManager.class);
+
+  static final String INCOMPLETE_BACKUP_FILE = "INCOMPLETE_BACKUP_FILE";
+
+  private static final String BACKUP_DIR_PREFIX = "dir";
+  private static final String README_FILE = "README_FILE.txt";
+  private static final String DATA_STORES_DIRECTORY = "diskstores";
+  private static final String USER_FILES = "user";
+  private static final String CONFIG_DIRECTORY = "config";
+
+  private final Map<DiskStoreImpl, DiskStoreBackup> backupByDiskStore = new HashMap<>();
+  private final RestoreScript restoreScript = new RestoreScript();
+  private final InternalDistributedMember sender;
+  private final InternalCache cache;
+  private final CountDownLatch allowDestroys = new CountDownLatch(1);
+  private volatile boolean isCancelled = false;
+
+  public BackupManager(InternalDistributedMember sender, InternalCache gemFireCache) {
+    this.sender = sender;
+    this.cache = gemFireCache;
+  }
+
+  public void validateRequestingAdmin() {
+    // We need to watch for pure admin guys that depart. this allMembershipListener set
+    // looks like it should receive those events.
+    Set allIds = getDistributionManager().addAllMembershipListenerAndGetAllIds(this);
+    if (!allIds.contains(sender)) {
+      cleanup();
+      throw new IllegalStateException("The admin member requesting a backup has already departed");
+    }
+  }
+
+  public HashSet<PersistentID> prepareForBackup() {
+    HashSet<PersistentID> persistentIds = new HashSet<>();
+    for (DiskStore store : cache.listDiskStoresIncludingRegionOwned()) {
+      DiskStoreImpl storeImpl = (DiskStoreImpl) store;
+      storeImpl.lockStoreBeforeBackup();
+      if (storeImpl.hasPersistedData()) {
+        persistentIds.add(storeImpl.getPersistentID());
+        storeImpl.getStats().startBackup();
+      }
+    }
+    return persistentIds;
+  }
+
+  public HashSet<PersistentID> doBackup(File targetDir, File baselineDir, boolean abort)
+      throws IOException {
+    try {
+      if (abort) {
+        return new HashSet<>();
+      }
+      HashSet<PersistentID> persistentIds = new HashSet<>();
+      File backupDir = getBackupDir(targetDir);
+
+      // Make sure our baseline is okay for this member
+      baselineDir = checkBaseline(baselineDir);
+
+      // Create an inspector for the baseline backup
+      BackupInspector inspector =
+          (baselineDir == null ? null : BackupInspector.createInspector(baselineDir));
+
+      File storesDir = new File(backupDir, DATA_STORES_DIRECTORY);
+      Collection<DiskStore> diskStores = cache.listDiskStoresIncludingRegionOwned();
+      Map<DiskStoreImpl, DiskStoreBackup> backupByDiskStore = new HashMap<>();
+
+      boolean foundPersistentData = false;
+      for (DiskStore store : diskStores) {
+        DiskStoreImpl diskStore = (DiskStoreImpl) store;
+        if (diskStore.hasPersistedData()) {
+          if (!foundPersistentData) {
+            createBackupDir(backupDir);
+            foundPersistentData = true;
+          }
+          File diskStoreDir = new File(storesDir, getBackupDirName(diskStore));
+          diskStoreDir.mkdir();
+          DiskStoreBackup backup = startDiskStoreBackup(diskStore, diskStoreDir, inspector);
+          backupByDiskStore.put(diskStore, backup);
+        }
+        diskStore.releaseBackupLock();
+      }
+
+      allowDestroys.countDown();
+
+      for (Map.Entry<DiskStoreImpl, DiskStoreBackup> entry : backupByDiskStore.entrySet()) {
+        DiskStoreImpl diskStore = entry.getKey();
+        completeBackup(diskStore, entry.getValue());
+        diskStore.getStats().endBackup();
+        persistentIds.add(diskStore.getPersistentID());
+      }
+
+      if (!backupByDiskStore.isEmpty()) {
+        completeRestoreScript(backupDir);
+      }
+
+      return persistentIds;
+
+    } finally {
+      cleanup();
+    }
+  }
+
+  public void abort() {
+    cleanup();
+  }
+
+  private DM getDistributionManager() {
+    return cache.getInternalDistributedSystem().getDistributionManager();
+  }
+
+  private void cleanup() {
+    isCancelled = true;
+    allowDestroys.countDown();
+    releaseBackupLocks();
+    getDistributionManager().removeAllMembershipListener(this);
+    cache.clearBackupManager();
+  }
+
+  private void releaseBackupLocks() {
+    for (DiskStore store : cache.listDiskStoresIncludingRegionOwned()) {
+      ((DiskStoreImpl) store).releaseBackupLock();
+    }
+  }
+
+  /**
+   * Returns the memberId directory for this member in the baseline. The memberId may have changed
+   * if this member has been restarted since the last backup.
+   * 
+   * @param baselineParentDir parent directory of last backup.
+   * @return null if the baseline for this member could not be located.
+   */
+  private File findBaselineForThisMember(File baselineParentDir) {
+    File baselineDir = null;
+
+    /*
+     * Find the first matching DiskStoreId directory for this member.
+     */
+    for (DiskStore diskStore : cache.listDiskStoresIncludingRegionOwned()) {
+      File[] matchingFiles = baselineParentDir
+          .listFiles((file, name) -> name.endsWith(getBackupDirName((DiskStoreImpl) diskStore)));
+      // We found it? Good. Set this member's baseline to the backed up disk store's member dir (two
+      // levels up).
+      if (null != matchingFiles && matchingFiles.length > 0)
+        baselineDir = matchingFiles[0].getParentFile().getParentFile();
+    }
+    return baselineDir;
+  }
+
+  /**
+   * Performs a sanity check on the baseline directory for incremental backups. If a baseline
+   * directory exists for the member and there is no INCOMPLETE_BACKUP_FILE file then return the
+   * data stores directory for this member.
+   * 
+   * @param baselineParentDir a previous backup directory. This is used with the incremental backup
+   *        option. May be null if the user specified a full backup.
+   * @return null if the backup is to be a full backup otherwise return the data store directory in
+   *         the previous backup for this member (if incremental).
+   */
+  private File checkBaseline(File baselineParentDir) throws IOException {
+    File baselineDir = null;
+
+    if (null != baselineParentDir) {
+      // Start by looking for this memberId
+      baselineDir = getBackupDir(baselineParentDir);
+
+      if (!baselineDir.exists()) {
+        // hmmm, did this member have a restart?
+        // Determine which member dir might be a match for us
+        baselineDir = findBaselineForThisMember(baselineParentDir);
+      }
+
+      if (null != baselineDir) {
+        // check for existence of INCOMPLETE_BACKUP_FILE file
+        File incompleteBackup = new File(baselineDir, INCOMPLETE_BACKUP_FILE);
+        if (incompleteBackup.exists()) {
+          baselineDir = null;
+        }
+      }
+    }
+
+    return baselineDir;
+  }
+
+  private void completeRestoreScript(File backupDir) throws IOException {
+    backupConfigFiles(restoreScript, backupDir);
+    backupUserFiles(restoreScript, backupDir);
+    backupDeployedJars(restoreScript, backupDir);
+    restoreScript.generate(backupDir);
+    File incompleteFile = new File(backupDir, INCOMPLETE_BACKUP_FILE);
+    if (!incompleteFile.delete()) {
+      throw new IOException("Could not delete file " + INCOMPLETE_BACKUP_FILE);
+    }
+  }
+
+  /**
+   * Copy the oplogs to the backup directory. This is the final step of the backup process. The
+   * oplogs we copy are defined in the startDiskStoreBackup method.
+   */
+  private void completeBackup(DiskStoreImpl diskStore, DiskStoreBackup backup) throws IOException {
+    if (backup == null) {
+      return;
+    }
+    try {
+      // Wait for oplogs to be unpreblown before backing them up.
+      diskStore.waitForDelayedWrites();
+
+      // Backup all of the oplogs
+      for (Oplog oplog : backup.getPendingBackup()) {
+        if (isCancelled()) {
+          break;
+        }
+        // Copy theoplog to the destination directory
+        int index = oplog.getDirectoryHolder().getArrayIndex();
+        File backupDir = getBackupDir(backup.getTargetDir(), index);
+        // TODO prpersist - We could probably optimize this to *move* the files
+        // that we know are supposed to be deleted.
+        oplog.copyTo(backupDir);
+
+        // Allow the oplog to be deleted, and process any pending delete
+        backup.backupFinished(oplog);
+      }
+    } finally {
+      backup.cleanup();
+    }
+  }
+
+  /**
+   * Returns the dir name used to back up this DiskStore's directories under. The name is a
+   * concatenation of the disk store name and id.
+   */
+  private String getBackupDirName(DiskStoreImpl diskStore) {
+    String name = diskStore.getName();
+
+    if (name == null) {
+      name = GemFireCacheImpl.getDefaultDiskStoreName();
+    }
+
+    return (name + "_" + diskStore.getDiskStoreID().toString());
+  }
+
+  /**
+   * Start the backup process. This is the second step of the backup process. In this method, we
+   * define the data we're backing up by copying the init file and rolling to the next file. After
+   * this method returns operations can proceed as normal, except that we don't remove oplogs.
+   */
+  private DiskStoreBackup startDiskStoreBackup(DiskStoreImpl diskStore, File targetDir,
+      BackupInspector baselineInspector) throws IOException {
+    diskStore.getBackupLock().setBackupThread();
+    DiskStoreBackup backup = null;
+    boolean done = false;
+    try {
+      for (;;) {
+        Oplog childOplog = diskStore.getPersistentOplogSet().getChild();
+        if (childOplog == null) {
+          backup = new DiskStoreBackup(new Oplog[0], targetDir);
+          backupByDiskStore.put(diskStore, backup);
+          break;
+        }
+
+        // Get an appropriate lock object for each set of oplogs.
+        Object childLock = childOplog.lock;
+
+        // TODO - We really should move this lock into the disk store, but
+        // until then we need to do this magic to make sure we're actually
+        // locking the latest child for both types of oplogs
+
+        // This ensures that all writing to disk is blocked while we are
+        // creating the snapshot
+        synchronized (childLock) {
+          if (diskStore.getPersistentOplogSet().getChild() != childOplog) {
+            continue;
+          }
+
+          if (logger.isDebugEnabled()) {
+            logger.debug("snapshotting oplogs for disk store {}", diskStore.getName());
+          }
+
+          createDiskStoreBackupDirs(diskStore, targetDir);
+
+          restoreScript.addExistenceTest(diskStore.getDiskInitFile().getIFFile());
+
+          // Contains all oplogs that will backed up
+          Oplog[] allOplogs = null;
+
+          // Incremental backup so filter out oplogs that have already been
+          // backed up
+          if (null != baselineInspector) {
+            Map<File, File> baselineCopyMap = new HashMap<>();
+            allOplogs = filterBaselineOplogs(diskStore, baselineInspector, baselineCopyMap);
+            restoreScript.addBaselineFiles(baselineCopyMap);
+          } else {
+            allOplogs = diskStore.getAllOplogsForBackup();
+          }
+
+          // mark all oplogs as being backed up. This will
+          // prevent the oplogs from being deleted
+          backup = new DiskStoreBackup(allOplogs, targetDir);
+          backupByDiskStore.put(diskStore, backup);
+
+          // copy the init file
+          File firstDir = getBackupDir(targetDir, diskStore.getInforFileDirIndex());
+          diskStore.getDiskInitFile().copyTo(firstDir);
+          diskStore.getPersistentOplogSet().forceRoll(null);
+
+          if (logger.isDebugEnabled()) {
+            logger.debug("done snaphotting for disk store {}", diskStore.getName());
+          }
+          break;
+        }
+      }
+      done = true;
+    } finally {
+      if (!done) {
+        if (backup != null) {
+          backupByDiskStore.remove(diskStore);
+          backup.cleanup();
+        }
+      }
+    }
+    return backup;
+  }
+
+  private void createDiskStoreBackupDirs(DiskStoreImpl diskStore, File targetDir)
+      throws IOException {
+    // Create the directories for this disk store
+    DirectoryHolder[] directories = diskStore.getDirectoryHolders();
+    for (int i = 0; i < directories.length; i++) {
+      File dir = getBackupDir(targetDir, i);
+      if (!dir.mkdirs()) {
+        throw new IOException("Could not create directory " + dir);
+      }
+      restoreScript.addFile(directories[i].getDir(), dir);
+    }
+  }
+
+  /**
+   * Filters and returns the current set of oplogs that aren't already in the baseline for
+   * incremental backup
+   *
+   * @param baselineInspector the inspector for the previous backup.
+   * @param baselineCopyMap this will be populated with baseline oplogs Files that will be used in
+   *        the restore script.
+   * @return an array of Oplogs to be copied for an incremental backup.
+   */
+  private Oplog[] filterBaselineOplogs(DiskStoreImpl diskStore, BackupInspector baselineInspector,
+      Map<File, File> baselineCopyMap) throws IOException {
+    File baselineDir =
+        new File(baselineInspector.getBackupDir(), BackupManager.DATA_STORES_DIRECTORY);
+    baselineDir = new File(baselineDir, getBackupDirName(diskStore));
+
+    // Find all of the member's diskstore oplogs in the member's baseline
+    // diskstore directory structure (*.crf,*.krf,*.drf)
+    Collection<File> baselineOplogFiles =
+        FileUtils.listFiles(baselineDir, new String[] {"krf", "drf", "crf"}, true);
+    // Our list of oplogs to copy (those not already in the baseline)
+    List<Oplog> oplogList = new LinkedList<>();
+
+    // Total list of member oplogs
+    Oplog[] allOplogs = diskStore.getAllOplogsForBackup();
+
+    /*
+     * Loop through operation logs and see if they are already part of the baseline backup.
+     */
+    for (Oplog log : allOplogs) {
+      // See if they are backed up in the current baseline
+      Map<File, File> oplogMap = log.mapBaseline(baselineOplogFiles);
+
+      // No? Then see if they were backed up in previous baselines
+      if (oplogMap.isEmpty() && baselineInspector.isIncremental()) {
+        Set<String> matchingOplogs =
+            log.gatherMatchingOplogFiles(baselineInspector.getIncrementalOplogFileNames());
+        if (!matchingOplogs.isEmpty()) {
+          for (String matchingOplog : matchingOplogs) {
+            oplogMap.put(new File(baselineInspector.getCopyFromForOplogFile(matchingOplog)),
+                new File(baselineInspector.getCopyToForOplogFile(matchingOplog)));
+          }
+        }
+      }
+
+      if (oplogMap.isEmpty()) {
+        /*
+         * These are fresh operation log files so lets back them up.
+         */
+        oplogList.add(log);
+      } else {
+        /*
+         * These have been backed up before so lets just add their entries from the previous backup
+         * or restore script into the current one.
+         */
+        baselineCopyMap.putAll(oplogMap);
+      }
+    }
+
+    // Convert the filtered oplog list to an array
+    return oplogList.toArray(new Oplog[oplogList.size()]);
+  }
+
+  private File getBackupDir(File targetDir, int index) {
+    return new File(targetDir, BACKUP_DIR_PREFIX + index);
+  }
+
+  private void backupConfigFiles(RestoreScript restoreScript, File backupDir) throws IOException {
+    File configBackupDir = new File(backupDir, CONFIG_DIRECTORY);
+    configBackupDir.mkdirs();
+    URL url = cache.getCacheXmlURL();
+    if (url != null) {
+      File cacheXMLBackup =
+          new File(configBackupDir, DistributionConfig.DEFAULT_CACHE_XML_FILE.getName());
+      FileUtils.copyFile(new File(cache.getCacheXmlURL().getFile()), cacheXMLBackup);
+    }
+
+    URL propertyURL = DistributedSystem.getPropertiesFileURL();
+    if (propertyURL != null) {
+      File propertyBackup =
+          new File(configBackupDir, DistributionConfig.GEMFIRE_PREFIX + "properties");
+      FileUtils.copyFile(new File(DistributedSystem.getPropertiesFile()), propertyBackup);
+    }
+
+    // TODO: should the gfsecurity.properties file be backed up?
+  }
+
+  private void backupUserFiles(RestoreScript restoreScript, File backupDir) throws IOException {
+    List<File> backupFiles = cache.getBackupFiles();
+    File userBackupDir = new File(backupDir, USER_FILES);
+    if (!userBackupDir.exists()) {
+      userBackupDir.mkdir();
+    }
+    for (File original : backupFiles) {
+      if (original.exists()) {
+        original = original.getAbsoluteFile();
+        File dest = new File(userBackupDir, original.getName());
+        if (original.isDirectory()) {
+          FileUtils.copyDirectory(original, dest);
+        } else {
+          FileUtils.copyFile(original, dest);
+        }
+        restoreScript.addExistenceTest(original);
+        restoreScript.addFile(original, dest);
+      }
+    }
+  }
+
+  /**
+   * Copies user deployed jars to the backup directory.
+   * 
+   * @param restoreScript Used to restore from this backup.
+   * @param backupDir The backup directory for this member.
+   * @throws IOException one or more of the jars did not successfully copy.
+   */
+  private void backupDeployedJars(RestoreScript restoreScript, File backupDir) throws IOException {
+    JarDeployer deployer = null;
+
+    try {
+      /*
+       * Suspend any user deployed jar file updates during this backup.
+       */
+      deployer = ClassPathLoader.getLatest().getJarDeployer();
+      deployer.suspendAll();
+
+      List<DeployedJar> jarList = deployer.findDeployedJars();
+      if (!jarList.isEmpty()) {
+        File userBackupDir = new File(backupDir, USER_FILES);
+        if (!userBackupDir.exists()) {
+          userBackupDir.mkdir();
+        }
+
+        for (DeployedJar loader : jarList) {
+          File source = new File(loader.getFileCanonicalPath());
+          File dest = new File(userBackupDir, source.getName());
+          if (source.isDirectory()) {
+            FileUtils.copyDirectory(source, dest);
+          } else {
+            FileUtils.copyFile(source, dest);
+          }
+          restoreScript.addFile(source, dest);
+        }
+      }
+    } finally {
+      /*
+       * Re-enable user deployed jar file updates.
+       */
+      if (null != deployer) {
+        deployer.resumeAll();
+      }
+    }
+  }
+
+  private File getBackupDir(File targetDir) throws IOException {
+    InternalDistributedMember memberId =
+        cache.getInternalDistributedSystem().getDistributedMember();
+    String vmId = memberId.toString();
+    vmId = cleanSpecialCharacters(vmId);
+    return new File(targetDir, vmId);
+  }
+
+  private void createBackupDir(File backupDir) throws IOException {
+    if (backupDir.exists()) {
+      throw new IOException("Backup directory " + backupDir.getAbsolutePath() + " already exists.");
+    }
+
+    if (!backupDir.mkdirs()) {
+      throw new IOException("Could not create directory: " + backupDir);
+    }
+
+    File incompleteFile = new File(backupDir, INCOMPLETE_BACKUP_FILE);
+    if (!incompleteFile.createNewFile()) {
+      throw new IOException("Could not create file: " + incompleteFile);
+    }
+
+    File readme = new File(backupDir, README_FILE);
+    FileOutputStream fos = new FileOutputStream(readme);
+
+    try {
+      String text = LocalizedStrings.BackupManager_README.toLocalizedString();
+      fos.write(text.getBytes());
+    } finally {
+      fos.close();
+    }
+  }
+
+  private String cleanSpecialCharacters(String string) {
+    return string.replaceAll("[^\\w]+", "_");
+  }
+
+  public void memberDeparted(InternalDistributedMember id, boolean crashed) {
+    cleanup();
+  }
+
+  public void memberJoined(InternalDistributedMember id) {}
+
+  public void quorumLost(Set<InternalDistributedMember> failures,
+      List<InternalDistributedMember> remaining) {}
+
+  public void memberSuspect(InternalDistributedMember id, InternalDistributedMember whoSuspected,
+      String reason) {}
+
+  public void waitForBackup() {
+    try {
+      allowDestroys.await();
+    } catch (InterruptedException e) {
+      throw new InternalGemFireError(e);
+    }
+  }
+
+  public boolean isCancelled() {
+    return isCancelled;
+  }
+
+  public DiskStoreBackup getBackupForDiskStore(DiskStoreImpl diskStore) {
+    return backupByDiskStore.get(diskStore);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java
index 309dea3..53c5ca1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java
@@ -25,17 +25,16 @@ import org.apache.geode.internal.cache.persistence.BackupInspector;
  * This class manages the state of the backup of an individual disk store. It holds the list of
  * oplogs that still need to be backed up, along with the lists of oplog files that should be
  * deleted when the oplog is backed up. See
- * {@link DiskStoreImpl#startBackup(File, BackupInspector, org.apache.geode.internal.cache.persistence.RestoreScript)}
  */
 public class DiskStoreBackup {
 
   private final Set<Oplog> pendingBackup;
-  private final Set<Oplog> deferredCrfDeletes = new HashSet<Oplog>();
-  private final Set<Oplog> deferredDrfDeletes = new HashSet<Oplog>();
+  private final Set<Oplog> deferredCrfDeletes = new HashSet<>();
+  private final Set<Oplog> deferredDrfDeletes = new HashSet<>();
   private final File targetDir;
 
   public DiskStoreBackup(Oplog[] allOplogs, File targetDir) {
-    this.pendingBackup = new HashSet<Oplog>(Arrays.asList(allOplogs));
+    this.pendingBackup = new HashSet<>(Arrays.asList(allOplogs));
     this.targetDir = targetDir;
   }
 
@@ -70,7 +69,7 @@ public class DiskStoreBackup {
   }
 
   public synchronized Set<Oplog> getPendingBackup() {
-    return new HashSet<Oplog>(pendingBackup);
+    return new HashSet<>(pendingBackup);
   }
 
   public synchronized void backupFinished(Oplog oplog) {

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java
index 0288ef1..d6d55d6 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java
@@ -21,7 +21,6 @@ import org.apache.geode.GemFireIOException;
 import org.apache.geode.cache.DiskStoreFactory;
 import org.apache.geode.cache.DiskStore;
 import org.apache.geode.distributed.internal.ResourceEvent;
-import org.apache.geode.internal.cache.persistence.BackupManager;
 import org.apache.geode.internal.cache.xmlcache.CacheCreation;
 import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.internal.cache.xmlcache.DiskStoreAttributesCreation;

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
index 94d1253..a8a8a53 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
@@ -33,8 +33,6 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
@@ -60,7 +58,6 @@ import java.util.regex.Pattern;
 
 import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
 import it.unimi.dsi.fastutil.longs.LongOpenHashSet;
-import org.apache.commons.io.FileUtils;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelCriterion;
@@ -86,8 +83,6 @@ import org.apache.geode.internal.cache.DiskEntry.RecoveredEntry;
 import org.apache.geode.internal.cache.ExportDiskRegion.ExportWriter;
 import org.apache.geode.internal.cache.lru.LRUAlgorithm;
 import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.persistence.BackupInspector;
-import org.apache.geode.internal.cache.persistence.BackupManager;
 import org.apache.geode.internal.cache.persistence.BytesAndBits;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
@@ -97,7 +92,6 @@ import org.apache.geode.internal.cache.persistence.OplogType;
 import org.apache.geode.internal.cache.persistence.PRPersistentConfig;
 import org.apache.geode.internal.cache.persistence.PersistentMemberID;
 import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
-import org.apache.geode.internal.cache.persistence.RestoreScript;
 import org.apache.geode.internal.cache.snapshot.GFSnapshot;
 import org.apache.geode.internal.cache.snapshot.GFSnapshot.SnapshotWriter;
 import org.apache.geode.internal.cache.snapshot.SnapshotPacket.SnapshotRecord;
@@ -126,8 +120,6 @@ import org.apache.geode.pdx.internal.PeerTypeRegistration;
 public class DiskStoreImpl implements DiskStore {
   private static final Logger logger = LogService.getLogger();
 
-  private static final String BACKUP_DIR_PREFIX = "dir";
-
   public static final boolean KRF_DEBUG = Boolean.getBoolean("disk.KRF_DEBUG");
 
   public static final int MAX_OPEN_INACTIVE_OPLOGS =
@@ -302,8 +294,6 @@ public class DiskStoreImpl implements DiskStore {
 
   private DiskInitFile initFile = null;
 
-  private volatile DiskStoreBackup diskStoreBackup = null;
-
   private final ReentrantReadWriteLock compactorLock = new ReentrantReadWriteLock();
 
   private final WriteLock compactorWriteLock = compactorLock.writeLock();
@@ -672,6 +662,10 @@ public class DiskStoreImpl implements DiskStore {
     }
   }
 
+  public PersistentOplogSet getPersistentOplogSet() {
+    return persistentOplogs;
+  }
+
   PersistentOplogSet getPersistentOplogSet(DiskRegionView drv) {
     assert drv.isBackup();
     return persistentOplogs;
@@ -2031,6 +2025,10 @@ public class DiskStoreImpl implements DiskStore {
     return this.directories[this.infoFileDirIndex];
   }
 
+  int getInforFileDirIndex() {
+    return this.infoFileDirIndex;
+  }
+
   /**
    * returns the size of the biggest directory available to the region
    */
@@ -2692,84 +2690,9 @@ public class DiskStoreImpl implements DiskStore {
   }
 
   /**
-   * Returns the dir name used to back up this DiskStore's directories under. The name is a
-   * concatenation of the disk store name and id.
-   */
-  public String getBackupDirName() {
-    String name = getName();
-
-    if (name == null) {
-      name = GemFireCacheImpl.getDefaultDiskStoreName();
-    }
-
-    return (name + "_" + getDiskStoreID().toString());
-  }
-
-  /**
-   * Filters and returns the current set of oplogs that aren't already in the baseline for
-   * incremental backup
-   * 
-   * @param baselineInspector the inspector for the previous backup.
-   * @param baselineCopyMap this will be populated with baseline oplogs Files that will be used in
-   *        the restore script.
-   * @return an array of Oplogs to be copied for an incremental backup.
-   */
-  private Oplog[] filterBaselineOplogs(BackupInspector baselineInspector,
-      Map<File, File> baselineCopyMap) throws IOException {
-    File baselineDir = new File(baselineInspector.getBackupDir(), BackupManager.DATA_STORES);
-    baselineDir = new File(baselineDir, getBackupDirName());
-
-    // Find all of the member's diskstore oplogs in the member's baseline
-    // diskstore directory structure (*.crf,*.krf,*.drf)
-    Collection<File> baselineOplogFiles =
-        FileUtils.listFiles(baselineDir, new String[] {"krf", "drf", "crf"}, true);
-    // Our list of oplogs to copy (those not already in the baseline)
-    List<Oplog> oplogList = new LinkedList<Oplog>();
-
-    // Total list of member oplogs
-    Oplog[] allOplogs = getAllOplogsForBackup();
-
-    /*
-     * Loop through operation logs and see if they are already part of the baseline backup.
-     */
-    for (Oplog log : allOplogs) {
-      // See if they are backed up in the current baseline
-      Map<File, File> oplogMap = log.mapBaseline(baselineOplogFiles);
-
-      // No? Then see if they were backed up in previous baselines
-      if (oplogMap.isEmpty() && baselineInspector.isIncremental()) {
-        Set<String> matchingOplogs =
-            log.gatherMatchingOplogFiles(baselineInspector.getIncrementalOplogFileNames());
-        if (!matchingOplogs.isEmpty()) {
-          for (String matchingOplog : matchingOplogs) {
-            oplogMap.put(new File(baselineInspector.getCopyFromForOplogFile(matchingOplog)),
-                new File(baselineInspector.getCopyToForOplogFile(matchingOplog)));
-          }
-        }
-      }
-
-      if (oplogMap.isEmpty()) {
-        /*
-         * These are fresh operation log files so lets back them up.
-         */
-        oplogList.add(log);
-      } else {
-        /*
-         * These have been backed up before so lets just add their entries from the previous backup
-         * or restore script into the current one.
-         */
-        baselineCopyMap.putAll(oplogMap);
-      }
-    }
-
-    // Convert the filtered oplog list to an array
-    return oplogList.toArray(new Oplog[oplogList.size()]);
-  }
-
-  /**
    * Get all of the oplogs
    */
-  private Oplog[] getAllOplogsForBackup() {
+  Oplog[] getAllOplogsForBackup() {
     return persistentOplogs.getAllOplogs();
   }
 
@@ -4066,124 +3989,6 @@ public class DiskStoreImpl implements DiskStore {
     getBackupLock().unlockForBackup();
   }
 
-  /**
-   * Start the backup process. This is the second step of the backup process. In this method, we
-   * define the data we're backing up by copying the init file and rolling to the next file. After
-   * this method returns operations can proceed as normal, except that we don't remove oplogs.
-   */
-  public void startBackup(File targetDir, BackupInspector baselineInspector,
-      RestoreScript restoreScript) throws IOException {
-    getBackupLock().setBackupThread();
-    boolean done = false;
-    try {
-      for (;;) {
-        Oplog childOplog = persistentOplogs.getChild();
-        if (childOplog == null) {
-          this.diskStoreBackup = new DiskStoreBackup(new Oplog[0], targetDir);
-          break;
-        }
-
-        // Get an appropriate lock object for each set of oplogs.
-        Object childLock = childOplog.lock;
-
-        // TODO - We really should move this lock into the disk store, but
-        // until then we need to do this magic to make sure we're actually
-        // locking the latest child for both types of oplogs
-
-        // This ensures that all writing to disk is blocked while we are
-        // creating the snapshot
-        synchronized (childLock) {
-          if (persistentOplogs.getChild() != childOplog) {
-            continue;
-          }
-
-          if (logger.isDebugEnabled()) {
-            logger.debug("snapshotting oplogs for disk store {}", getName());
-          }
-
-          // Create the directories for this disk store
-          for (int i = 0; i < directories.length; i++) {
-            File dir = getBackupDir(targetDir, i);
-            if (!dir.mkdirs()) {
-              throw new IOException("Could not create directory " + dir);
-            }
-            restoreScript.addFile(directories[i].getDir(), dir);
-          }
-
-          restoreScript.addExistenceTest(this.initFile.getIFFile());
-
-          // Contains all oplogs that will backed up
-          Oplog[] allOplogs = null;
-
-          // Incremental backup so filter out oplogs that have already been
-          // backed up
-          if (null != baselineInspector) {
-            Map<File, File> baselineCopyMap = new HashMap<File, File>();
-            allOplogs = filterBaselineOplogs(baselineInspector, baselineCopyMap);
-            restoreScript.addBaselineFiles(baselineCopyMap);
-          } else {
-            allOplogs = getAllOplogsForBackup();
-          }
-
-          // mark all oplogs as being backed up. This will
-          // prevent the oplogs from being deleted
-          this.diskStoreBackup = new DiskStoreBackup(allOplogs, targetDir);
-
-          // copy the init file
-          File firstDir = getBackupDir(targetDir, infoFileDirIndex);
-          initFile.copyTo(firstDir);
-          persistentOplogs.forceRoll(null);
-
-          if (logger.isDebugEnabled()) {
-            logger.debug("done snaphotting for disk store {}", getName());
-          }
-          break;
-        }
-      }
-      done = true;
-    } finally {
-      if (!done) {
-        clearBackup();
-      }
-    }
-  }
-
-  private File getBackupDir(File targetDir, int index) {
-    return new File(targetDir, BACKUP_DIR_PREFIX + index);
-  }
-
-  /**
-   * Copy the oplogs to the backup directory. This is the final step of the backup process. The
-   * oplogs we copy are defined in the startBackup method.
-   */
-  public void finishBackup(BackupManager backupManager) throws IOException {
-    if (diskStoreBackup == null) {
-      return;
-    }
-    try {
-      // Wait for oplogs to be unpreblown before backing them up.
-      waitForDelayedWrites();
-
-      // Backup all of the oplogs
-      for (Oplog oplog : this.diskStoreBackup.getPendingBackup()) {
-        if (backupManager.isCancelled()) {
-          break;
-        }
-        // Copy theoplog to the destination directory
-        int index = oplog.getDirectoryHolder().getArrayIndex();
-        File backupDir = getBackupDir(this.diskStoreBackup.getTargetDir(), index);
-        // TODO prpersist - We could probably optimize this to *move* the files
-        // that we know are supposed to be deleted.
-        oplog.copyTo(backupDir);
-
-        // Allow the oplog to be deleted, and process any pending delete
-        this.diskStoreBackup.backupFinished(oplog);
-      }
-    } finally {
-      clearBackup();
-    }
-  }
-
   private int getArrayIndexOfDirectory(File searchDir) {
     for (DirectoryHolder holder : directories) {
       if (holder.getDir().equals(searchDir)) {
@@ -4197,16 +4002,9 @@ public class DiskStoreImpl implements DiskStore {
     return this.directories;
   }
 
-  private void clearBackup() {
-    DiskStoreBackup backup = this.diskStoreBackup;
-    if (backup != null) {
-      this.diskStoreBackup = null;
-      backup.cleanup();
-    }
-  }
-
   public DiskStoreBackup getInProgressBackup() {
-    return diskStoreBackup;
+    BackupManager backupManager = cache.getBackupManager();
+    return backupManager == null ? null : backupManager.getBackupForDiskStore(this);
   }
 
   public Collection<DiskRegionView> getKnown() {

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index 67c8add..6d250d9 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -79,6 +79,8 @@ import com.sun.jna.Platform;
 import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
 
+import org.apache.geode.internal.cache.event.EventTrackerExpiryTask;
+import org.apache.geode.internal.security.SecurityServiceFactory;
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.CancelException;
 import org.apache.geode.ForcedDisconnectException;
@@ -184,7 +186,6 @@ import org.apache.geode.internal.cache.locks.TXLockService;
 import org.apache.geode.internal.cache.lru.HeapEvictor;
 import org.apache.geode.internal.cache.lru.OffHeapEvictor;
 import org.apache.geode.internal.cache.partitioned.RedundancyAlreadyMetException;
-import org.apache.geode.internal.cache.persistence.BackupManager;
 import org.apache.geode.internal.cache.persistence.PersistentMemberID;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.internal.cache.snapshot.CacheSnapshotServiceImpl;
@@ -4351,7 +4352,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     if (!this.backupManager.compareAndSet(null, manager)) {
       throw new IOException("Backup already in progress");
     }
-    manager.start();
+    manager.validateRequestingAdmin();
     return manager;
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
index d162010..84aa66e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
@@ -55,7 +55,6 @@ import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.ResourceAdvisor;
 import org.apache.geode.internal.cache.event.EventTrackerExpiryTask;
 import org.apache.geode.internal.cache.extension.Extensible;
-import org.apache.geode.internal.cache.persistence.BackupManager;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java b/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
index 80f19b5..860db98 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
@@ -5702,6 +5702,7 @@ public class Oplog implements CompactableOplog, Flushable {
 
   public void deleteCRF() {
     oplogSet.crfDelete(this.oplogId);
+    BackupManager backupManager = getInternalCache().getBackupManager();
     DiskStoreBackup inProgressBackup = getParent().getInProgressBackup();
     if (inProgressBackup == null || !inProgressBackup.deferCrfDelete(this)) {
       deleteCRFFileOnly();

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
index 893ca6b..3d9ac18 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
@@ -42,7 +42,6 @@ import org.apache.geode.internal.cache.execute.PartitionedRegionFunctionResultSe
 import org.apache.geode.internal.cache.execute.RegionFunctionContextImpl;
 import org.apache.geode.internal.cache.partitioned.*;
 import org.apache.geode.internal.cache.partitioned.RemoveBucketMessage.RemoveBucketResponse;
-import org.apache.geode.internal.cache.persistence.BackupManager;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/BackupManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/BackupManager.java b/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/BackupManager.java
deleted file mode 100644
index f464e0d..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/BackupManager.java
+++ /dev/null
@@ -1,389 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.persistence;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.io.FileUtils;
-
-import org.apache.geode.InternalGemFireError;
-import org.apache.geode.cache.DiskStore;
-import org.apache.geode.cache.persistence.PersistentID;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.DM;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.MembershipListener;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.internal.DeployedJar;
-import org.apache.geode.internal.JarDeployer;
-import org.apache.geode.internal.cache.DiskStoreImpl;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-
-/**
- * This class manages the state an logic to backup a single cache.
- */
-public class BackupManager implements MembershipListener {
-
-  // TODO prpersist internationalize this.
-  public static final String INCOMPLETE_BACKUP = "INCOMPLETE_BACKUP";
-  public static final String README = "README.txt";
-  public static final String DATA_STORES = "diskstores";
-  public static final String USER_FILES = "user";
-  public static final String CONFIG = "config";
-  private InternalDistributedMember sender;
-  private InternalCache cache;
-  private CountDownLatch allowDestroys = new CountDownLatch(1);
-  private volatile boolean isCancelled = false;
-
-  public BackupManager(InternalDistributedMember sender, InternalCache gemFireCache) {
-    this.sender = sender;
-    this.cache = gemFireCache;
-  }
-
-  public void start() {
-    final DM distributionManager = cache.getInternalDistributedSystem().getDistributionManager();
-    // We need to watch for pure admin guys that depart. this allMembershipListener set
-    // looks like it should receive those events.
-    Set allIds = distributionManager.addAllMembershipListenerAndGetAllIds(this);
-    if (!allIds.contains(sender)) {
-      cleanup();
-      throw new IllegalStateException("The admin member requesting a backup has already departed");
-    }
-  }
-
-  private void cleanup() {
-    isCancelled = true;
-    allowDestroys.countDown();
-    Collection<DiskStore> diskStores = cache.listDiskStoresIncludingRegionOwned();
-    for (DiskStore store : diskStores) {
-      ((DiskStoreImpl) store).releaseBackupLock();
-    }
-    final DM distributionManager = cache.getInternalDistributedSystem().getDistributionManager();
-    distributionManager.removeAllMembershipListener(this);
-    cache.clearBackupManager();
-  }
-
-  public HashSet<PersistentID> prepareBackup() {
-    HashSet<PersistentID> persistentIds = new HashSet<PersistentID>();
-    Collection<DiskStore> diskStores = cache.listDiskStoresIncludingRegionOwned();
-    for (DiskStore store : diskStores) {
-      DiskStoreImpl storeImpl = (DiskStoreImpl) store;
-      storeImpl.lockStoreBeforeBackup();
-      if (storeImpl.hasPersistedData()) {
-        persistentIds.add(storeImpl.getPersistentID());
-        storeImpl.getStats().startBackup();
-      }
-    }
-    return persistentIds;
-  }
-
-  /**
-   * Returns the memberId directory for this member in the baseline. The memberId may have changed
-   * if this member has been restarted since the last backup.
-   * 
-   * @param baselineParentDir parent directory of last backup.
-   * @return null if the baseline for this member could not be located.
-   */
-  private File findBaselineForThisMember(File baselineParentDir) {
-    File baselineDir = null;
-
-    /*
-     * Find the first matching DiskStoreId directory for this member.
-     */
-    for (DiskStore diskStore : cache.listDiskStoresIncludingRegionOwned()) {
-      File[] matchingFiles = baselineParentDir.listFiles(new FilenameFilter() {
-        Pattern pattern =
-            Pattern.compile(".*" + ((DiskStoreImpl) diskStore).getBackupDirName() + "$");
-
-        public boolean accept(File dir, String name) {
-          Matcher m = pattern.matcher(name);
-          return m.find();
-        }
-      });
-      // We found it? Good. Set this member's baseline to the backed up disk store's member dir (two
-      // levels up).
-      if (null != matchingFiles && matchingFiles.length > 0)
-        baselineDir = matchingFiles[0].getParentFile().getParentFile();
-    }
-    return baselineDir;
-  }
-
-  /**
-   * Performs a sanity check on the baseline directory for incremental backups. If a baseline
-   * directory exists for the member and there is no INCOMPLETE_BACKUP file then return the data
-   * stores directory for this member.
-   * 
-   * @param baselineParentDir a previous backup directory. This is used with the incremental backup
-   *        option. May be null if the user specified a full backup.
-   * @return null if the backup is to be a full backup otherwise return the data store directory in
-   *         the previous backup for this member (if incremental).
-   */
-  private File checkBaseline(File baselineParentDir) throws IOException {
-    File baselineDir = null;
-
-    if (null != baselineParentDir) {
-      // Start by looking for this memberId
-      baselineDir = getBackupDir(baselineParentDir);
-
-      if (!baselineDir.exists()) {
-        // hmmm, did this member have a restart?
-        // Determine which member dir might be a match for us
-        baselineDir = findBaselineForThisMember(baselineParentDir);
-      }
-
-      if (null != baselineDir) {
-        // check for existence of INCOMPLETE_BACKUP file
-        File incompleteBackup = new File(baselineDir, INCOMPLETE_BACKUP);
-        if (incompleteBackup.exists()) {
-          baselineDir = null;
-        }
-      }
-    }
-
-    return baselineDir;
-  }
-
-  public HashSet<PersistentID> finishBackup(File targetDir, File baselineDir, boolean abort)
-      throws IOException {
-    try {
-      if (abort) {
-        return new HashSet<PersistentID>();
-      }
-
-      File backupDir = getBackupDir(targetDir);
-
-      // Make sure our baseline is okay for this member
-      baselineDir = checkBaseline(baselineDir);
-
-      // Create an inspector for the baseline backup
-      BackupInspector inspector =
-          (baselineDir == null ? null : BackupInspector.createInspector(baselineDir));
-
-      File storesDir = new File(backupDir, DATA_STORES);
-      RestoreScript restoreScript = new RestoreScript();
-      HashSet<PersistentID> persistentIds = new HashSet<PersistentID>();
-      Collection<DiskStore> diskStores =
-          new ArrayList<DiskStore>(cache.listDiskStoresIncludingRegionOwned());
-
-      boolean foundPersistentData = false;
-      for (Iterator<DiskStore> itr = diskStores.iterator(); itr.hasNext();) {
-        DiskStoreImpl store = (DiskStoreImpl) itr.next();
-        if (store.hasPersistedData()) {
-          if (!foundPersistentData) {
-            createBackupDir(backupDir);
-            foundPersistentData = true;
-          }
-          File diskStoreDir = new File(storesDir, store.getBackupDirName());
-          diskStoreDir.mkdir();
-          store.startBackup(diskStoreDir, inspector, restoreScript);
-        } else {
-          itr.remove();
-        }
-        store.releaseBackupLock();
-      }
-
-      allowDestroys.countDown();
-
-      for (DiskStore store : diskStores) {
-        DiskStoreImpl storeImpl = (DiskStoreImpl) store;
-        storeImpl.finishBackup(this);
-        storeImpl.getStats().endBackup();
-        persistentIds.add(storeImpl.getPersistentID());
-      }
-
-      if (foundPersistentData) {
-        backupConfigFiles(restoreScript, backupDir);
-        backupUserFiles(restoreScript, backupDir);
-        backupDeployedJars(restoreScript, backupDir);
-        restoreScript.generate(backupDir);
-        File incompleteFile = new File(backupDir, INCOMPLETE_BACKUP);
-        if (!incompleteFile.delete()) {
-          throw new IOException("Could not delete file " + INCOMPLETE_BACKUP);
-        }
-      }
-
-      return persistentIds;
-
-    } finally {
-      cleanup();
-    }
-  }
-
-  public void abort() {
-    cleanup();
-  }
-
-  private void backupConfigFiles(RestoreScript restoreScript, File backupDir) throws IOException {
-    File configBackupDir = new File(backupDir, CONFIG);
-    configBackupDir.mkdirs();
-    URL url = cache.getCacheXmlURL();
-    if (url != null) {
-      File cacheXMLBackup =
-          new File(configBackupDir, DistributionConfig.DEFAULT_CACHE_XML_FILE.getName());
-      FileUtils.copyFile(new File(cache.getCacheXmlURL().getFile()), cacheXMLBackup);
-    }
-
-    URL propertyURL = DistributedSystem.getPropertiesFileURL();
-    if (propertyURL != null) {
-      File propertyBackup =
-          new File(configBackupDir, DistributionConfig.GEMFIRE_PREFIX + "properties");
-      FileUtils.copyFile(new File(DistributedSystem.getPropertiesFile()), propertyBackup);
-    }
-
-    // TODO: should the gfsecurity.properties file be backed up?
-  }
-
-  private void backupUserFiles(RestoreScript restoreScript, File backupDir) throws IOException {
-    List<File> backupFiles = cache.getBackupFiles();
-    File userBackupDir = new File(backupDir, USER_FILES);
-    if (!userBackupDir.exists()) {
-      userBackupDir.mkdir();
-    }
-    for (File original : backupFiles) {
-      if (original.exists()) {
-        original = original.getAbsoluteFile();
-        File dest = new File(userBackupDir, original.getName());
-        if (original.isDirectory()) {
-          FileUtils.copyDirectory(original, dest);
-        } else {
-          FileUtils.copyFile(original, dest);
-        }
-        restoreScript.addExistenceTest(original);
-        restoreScript.addFile(original, dest);
-      }
-    }
-  }
-
-  /**
-   * Copies user deployed jars to the backup directory.
-   * 
-   * @param restoreScript Used to restore from this backup.
-   * @param backupDir The backup directory for this member.
-   * @throws IOException one or more of the jars did not successfully copy.
-   */
-  private void backupDeployedJars(RestoreScript restoreScript, File backupDir) throws IOException {
-    JarDeployer deployer = null;
-
-    try {
-      /*
-       * Suspend any user deployed jar file updates during this backup.
-       */
-      deployer = ClassPathLoader.getLatest().getJarDeployer();
-      deployer.suspendAll();
-
-      List<DeployedJar> jarList = deployer.findDeployedJars();
-      if (!jarList.isEmpty()) {
-        File userBackupDir = new File(backupDir, USER_FILES);
-        if (!userBackupDir.exists()) {
-          userBackupDir.mkdir();
-        }
-
-        for (DeployedJar loader : jarList) {
-          File source = new File(loader.getFileCanonicalPath());
-          File dest = new File(userBackupDir, source.getName());
-          if (source.isDirectory()) {
-            FileUtils.copyDirectory(source, dest);
-          } else {
-            FileUtils.copyFile(source, dest);
-          }
-          restoreScript.addFile(source, dest);
-        }
-      }
-    } finally {
-      /*
-       * Re-enable user deployed jar file updates.
-       */
-      if (null != deployer) {
-        deployer.resumeAll();
-      }
-    }
-  }
-
-  private File getBackupDir(File targetDir) throws IOException {
-    InternalDistributedMember memberId =
-        cache.getInternalDistributedSystem().getDistributedMember();
-    String vmId = memberId.toString();
-    vmId = cleanSpecialCharacters(vmId);
-    return new File(targetDir, vmId);
-  }
-
-  private void createBackupDir(File backupDir) throws IOException {
-    if (backupDir.exists()) {
-      throw new IOException("Backup directory " + backupDir.getAbsolutePath() + " already exists.");
-    }
-
-    if (!backupDir.mkdirs()) {
-      throw new IOException("Could not create directory: " + backupDir);
-    }
-
-    File incompleteFile = new File(backupDir, INCOMPLETE_BACKUP);
-    if (!incompleteFile.createNewFile()) {
-      throw new IOException("Could not create file: " + incompleteFile);
-    }
-
-    File readme = new File(backupDir, README);
-    FileOutputStream fos = new FileOutputStream(readme);
-
-    try {
-      String text = LocalizedStrings.BackupManager_README.toLocalizedString();
-      fos.write(text.getBytes());
-    } finally {
-      fos.close();
-    }
-  }
-
-  private String cleanSpecialCharacters(String string) {
-    return string.replaceAll("[^\\w]+", "_");
-  }
-
-  public void memberDeparted(InternalDistributedMember id, boolean crashed) {
-    cleanup();
-  }
-
-  public void memberJoined(InternalDistributedMember id) {}
-
-  public void quorumLost(Set<InternalDistributedMember> failures,
-      List<InternalDistributedMember> remaining) {}
-
-  public void memberSuspect(InternalDistributedMember id, InternalDistributedMember whoSuspected,
-      String reason) {}
-
-  public void waitForBackup() {
-    try {
-      allowDestroys.await();
-    } catch (InterruptedException e) {
-      throw new InternalGemFireError(e);
-    }
-  }
-
-  public boolean isCancelled() {
-    return isCancelled;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
index a7f2a11..e5e372d 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
@@ -108,7 +108,7 @@ import org.apache.geode.internal.cache.extension.Extensible;
 import org.apache.geode.internal.cache.extension.ExtensionPoint;
 import org.apache.geode.internal.cache.extension.SimpleExtensionPoint;
 import org.apache.geode.internal.cache.ha.HARegionQueue;
-import org.apache.geode.internal.cache.persistence.BackupManager;
+import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
index dd905eb..5105c3d 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
@@ -77,7 +77,7 @@ import org.apache.geode.internal.cache.PartitionedRegionStats;
 import org.apache.geode.internal.cache.control.ResourceManagerStats;
 import org.apache.geode.internal.cache.execute.FunctionServiceStats;
 import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.persistence.BackupManager;
+import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
@@ -1037,10 +1037,10 @@ public class MemberMBeanBridge {
         Set<PersistentID> existingDataStores;
         Set<PersistentID> successfulDataStores;
         try {
-          existingDataStores = manager.prepareBackup();
+          existingDataStores = manager.prepareForBackup();
           abort = false;
         } finally {
-          successfulDataStores = manager.finishBackup(targetDir, null/* TODO rishi */, abort);
+          successfulDataStores = manager.doBackup(targetDir, null/* TODO rishi */, abort);
         }
         diskBackUpResult = new DiskBackupResult[existingDataStores.size()];
         int j = 0;

http://git-wip-us.apache.org/repos/asf/geode/blob/3bb6a221/geode-core/src/test/java/org/apache/geode/internal/cache/BackupDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/BackupDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/BackupDUnitTest.java
index f2cee71..338c712 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/BackupDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/BackupDUnitTest.java
@@ -46,11 +46,13 @@ import org.apache.geode.test.dunit.DUnitEnv;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.Invoke;
-import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.junit.categories.DistributedTest;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -67,34 +69,38 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
 
 @Category(DistributedTest.class)
 public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
+  Logger logger = LogManager.getLogger(BackupDUnitTest.class);
 
-  private static final long MAX_WAIT = 30 * 1000;
+  private static final long MAX_WAIT_SECONDS = 30;
+  private VM vm0;
+  private VM vm1;
 
   @Override
   public final void preTearDownCacheTestCase() throws Exception {
     StringBuilder failures = new StringBuilder();
     delete(getBackupDir(), failures);
     if (failures.length() > 0) {
-      LogWriterUtils.getLogWriter().error(failures.toString());
+      logger.error(failures.toString());
     }
   }
 
   @Test
   public void testBackupPR() throws Throwable {
     Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
+    vm0 = host.getVM(0);
+    vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
 
-    LogWriterUtils.getLogWriter().info("Creating region in VM0");
+    logger.info("Creating region in VM0");
     createPersistentRegion(vm0);
-    LogWriterUtils.getLogWriter().info("Creating region in VM1");
+    logger.info("Creating region in VM1");
     createPersistentRegion(vm1);
 
     long lm0 = setBackupFiles(vm0);
@@ -107,7 +113,6 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     assertEquals(2, status.getBackedUpDiskStores().size());
     assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
 
-    Pattern pattern = Pattern.compile(".*my.txt.*");
     Collection<File> files = FileUtils.listFiles(getBackupDir(), new String[] {"txt"}, true);
     assertEquals(4, files.size());
     deleteOldUserUserFile(vm0);
@@ -136,13 +141,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
     restoreBackup(2);
 
-    LogWriterUtils.getLogWriter().info("Creating region in VM0");
-    AsyncInvocation async0 = createPersistentRegionAsync(vm0);
-    LogWriterUtils.getLogWriter().info("Creating region in VM1");
-    AsyncInvocation async1 = createPersistentRegionAsync(vm1);
-
-    async0.getResult(MAX_WAIT);
-    async1.getResult(MAX_WAIT);
+    createPersistentRegionsAsync();
 
     checkData(vm0, 0, 5, "A", "region1");
     checkData(vm0, 0, 5, "B", "region2");
@@ -156,12 +155,12 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
   @Test
   public void testBackupFromMemberWithDiskStore() throws Throwable {
     Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
+    vm0 = host.getVM(0);
+    vm1 = host.getVM(1);
 
-    LogWriterUtils.getLogWriter().info("Creating region in VM0");
+    logger.info("Creating region in VM0");
     createPersistentRegion(vm0);
-    LogWriterUtils.getLogWriter().info("Creating region in VM1");
+    logger.info("Creating region in VM1");
     createPersistentRegion(vm1);
 
     createData(vm0, 0, 5, "A", "region1");
@@ -192,25 +191,21 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
     restoreBackup(2);
 
-    LogWriterUtils.getLogWriter().info("Creating region in VM0");
-    AsyncInvocation async0 = createPersistentRegionAsync(vm0);
-    LogWriterUtils.getLogWriter().info("Creating region in VM1");
-    AsyncInvocation async1 = createPersistentRegionAsync(vm1);
-
-    async0.getResult(MAX_WAIT);
-    async1.getResult(MAX_WAIT);
+    createPersistentRegionsAsync();
 
     checkData(vm0, 0, 5, "A", "region1");
     checkData(vm0, 0, 5, "B", "region2");
   }
 
-  // public void testLoop() throws Throwable {
-  // for(int i =0 ;i < 100; i++) {
-  // testBackupWhileBucketIsCreated();
-  // setUp();
-  // tearDown();
-  // }
-  // }
+  private void createPersistentRegionsAsync() throws java.util.concurrent.ExecutionException,
+      InterruptedException, java.util.concurrent.TimeoutException {
+    logger.info("Creating region in VM0");
+    AsyncInvocation async0 = createPersistentRegionAsync(vm0);
+    logger.info("Creating region in VM1");
+    AsyncInvocation async1 = createPersistentRegionAsync(vm1);
+    async0.get(MAX_WAIT_SECONDS, TimeUnit.SECONDS);
+    async1.get(MAX_WAIT_SECONDS, TimeUnit.SECONDS);
+  }
 
   /**
    * Test for bug 42419
@@ -218,40 +213,27 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
   @Test
   public void testBackupWhileBucketIsCreated() throws Throwable {
     Host host = Host.getHost(0);
-    final VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
+    vm0 = host.getVM(0);
+    vm1 = host.getVM(1);
     final VM vm2 = host.getVM(2);
 
-    LogWriterUtils.getLogWriter().info("Creating region in VM0");
+    logger.info("Creating region in VM0");
     createPersistentRegion(vm0);
 
     // create a bucket on vm0
     createData(vm0, 0, 1, "A", "region1");
 
     // create the pr on vm1, which won't have any buckets
-    LogWriterUtils.getLogWriter().info("Creating region in VM1");
+    logger.info("Creating region in VM1");
     createPersistentRegion(vm1);
 
-    final AtomicReference<BackupStatus> statusRef = new AtomicReference<BackupStatus>();
-    Thread thread1 = new Thread() {
-      public void run() {
+    CompletableFuture<BackupStatus> backupStatusFuture =
+        CompletableFuture.supplyAsync(() -> backup(vm2));
+    CompletableFuture<Void> createDataFuture =
+        CompletableFuture.runAsync(() -> createData(vm0, 1, 5, "A", "region1"));
+    CompletableFuture.allOf(backupStatusFuture, createDataFuture);
 
-        BackupStatus status = backup(vm2);
-        statusRef.set(status);
-
-      }
-    };
-    thread1.start();
-    Thread thread2 = new Thread() {
-      public void run() {
-        createData(vm0, 1, 5, "A", "region1");
-      }
-    };
-    thread2.start();
-    thread1.join();
-    thread2.join();
-
-    BackupStatus status = statusRef.get();
+    BackupStatus status = backupStatusFuture.get();
     assertEquals(2, status.getBackedUpDiskStores().size());
     assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
 
@@ -278,13 +260,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
     restoreBackup(2);
 
-    LogWriterUtils.getLogWriter().info("Creating region in VM0");
-    AsyncInvocation async0 = createPersistentRegionAsync(vm0);
-    LogWriterUtils.getLogWriter().info("Creating region in VM1");
-    AsyncInvocation async1 = createPersistentRegionAsync(vm1);
-
-    async0.getResult(MAX_WAIT);
-    async1.getResult(MAX_WAIT);
+    createPersistentRegionsAsync();
 
     checkData(vm0, 0, 1, "A", "region1");
   }
@@ -296,8 +272,6 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
     DistributionMessageObserver observer = new SerializableDistributionMessageObserver() {
       private volatile boolean done;
-      private AtomicInteger count = new AtomicInteger();
-      private volatile int replyId = -0xBAD;
 
       @Override
       public void beforeSendMessage(DistributionManager dm, DistributionMessage msg) {
@@ -316,8 +290,8 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
   @Test
   public void testBackupWhileBucketIsMovedBackupAfterSendDestroy() throws Throwable {
     Host host = Host.getHost(0);
-    final VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
+    vm0 = host.getVM(0);
+    vm1 = host.getVM(1);
     final VM vm2 = host.getVM(2);
 
     DistributionMessageObserver observer = new SerializableDistributionMessageObserver() {
@@ -407,12 +381,11 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
    * 
    * @param observer - a message observer that triggers at the backup at the correct time.
    */
-  public void backupWhileBucketIsMoved(final DistributionMessageObserver observer)
+  private void backupWhileBucketIsMoved(final DistributionMessageObserver observer)
       throws Throwable {
     Host host = Host.getHost(0);
-    final VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-    final VM vm2 = host.getVM(2);
+    vm0 = host.getVM(0);
+    vm1 = host.getVM(1);
 
     vm0.invoke(new SerializableRunnable("Add listener to invoke backup") {
 
@@ -428,14 +401,14 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     });
     try {
 
-      LogWriterUtils.getLogWriter().info("Creating region in VM0");
+      logger.info("Creating region in VM0");
       createPersistentRegion(vm0);
 
       // create twos bucket on vm0
       createData(vm0, 0, 2, "A", "region1");
 
       // create the pr on vm1, which won't have any buckets
-      LogWriterUtils.getLogWriter().info("Creating region in VM1");
+      logger.info("Creating region in VM1");
 
       createPersistentRegion(vm1);
 
@@ -476,13 +449,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
       restoreBackup(2);
 
-      LogWriterUtils.getLogWriter().info("Creating region in VM0");
-      AsyncInvocation async0 = createPersistentRegionAsync(vm0);
-      LogWriterUtils.getLogWriter().info("Creating region in VM1");
-      AsyncInvocation async1 = createPersistentRegionAsync(vm1);
-
-      async0.getResult(MAX_WAIT);
-      async1.getResult(MAX_WAIT);
+      createPersistentRegionsAsync();
 
       checkData(vm0, 0, 2, "A", "region1");
     } finally {
@@ -502,13 +469,13 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
   @Test
   public void testBackupOverflow() throws Throwable {
     Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
+    vm0 = host.getVM(0);
+    vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
 
-    LogWriterUtils.getLogWriter().info("Creating region in VM0");
+    logger.info("Creating region in VM0");
     createPersistentRegion(vm0);
-    LogWriterUtils.getLogWriter().info("Creating region in VM1");
+    logger.info("Creating region in VM1");
     createOverflowRegion(vm1);
 
     createData(vm0, 0, 5, "A", "region1");
@@ -526,16 +493,16 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
   @Test
   public void testBackupPRWithOfflineMembers() throws Throwable {
     Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
+    vm0 = host.getVM(0);
+    vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
     VM vm3 = host.getVM(3);
 
-    LogWriterUtils.getLogWriter().info("Creating region in VM0");
+    logger.info("Creating region in VM0");
     createPersistentRegion(vm0);
-    LogWriterUtils.getLogWriter().info("Creating region in VM1");
+    logger.info("Creating region in VM1");
     createPersistentRegion(vm1);
-    LogWriterUtils.getLogWriter().info("Creating region in VM2");
+    logger.info("Creating region in VM2");
     createPersistentRegion(vm2);
 
     createData(vm0, 0, 5, "A", "region1");
@@ -562,11 +529,11 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     assertTrue(files.length == 0);
   }
 
-  protected void createPersistentRegion(VM vm) throws Throwable {
+  private void createPersistentRegion(VM vm) throws Throwable {
     AsyncInvocation future = createPersistentRegionAsync(vm);
-    future.join(MAX_WAIT);
+    future.get(MAX_WAIT_SECONDS, TimeUnit.SECONDS);
     if (future.isAlive()) {
-      fail("Region not created within" + MAX_WAIT);
+      fail("Region not created within" + MAX_WAIT_SECONDS);
     }
     if (future.exceptionOccurred()) {
       throw new RuntimeException(future.getException());
@@ -576,9 +543,8 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
   private void deleteOldUserUserFile(final VM vm) {
     SerializableRunnable validateUserFileBackup = new SerializableRunnable("set user backups") {
       public void run() {
-        final int pid = vm.getPid();
         try {
-          FileUtils.deleteDirectory(new File("userbackup_" + pid));
+          FileUtils.deleteDirectory(new File("userbackup_" + vm.getPid()));
         } catch (IOException e) {
           fail(e.getMessage());
         }
@@ -587,7 +553,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     vm.invoke(validateUserFileBackup);
   }
 
-  protected long setBackupFiles(final VM vm) {
+  private long setBackupFiles(final VM vm) {
     SerializableCallable setUserBackups = new SerializableCallable("set user backups") {
       public Object call() {
         final int pid = DUnitEnv.get().getPid();
@@ -595,7 +561,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
         File test1 = new File(vmdir, "test1");
         File test2 = new File(test1, "test2");
         File mytext = new File(test2, "my.txt");
-        final ArrayList<File> backuplist = new ArrayList<File>();
+        final ArrayList<File> backuplist = new ArrayList<>();
         test2.mkdirs();
         PrintStream ps = null;
         try {
@@ -619,7 +585,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     return (long) vm.invoke(setUserBackups);
   }
 
-  protected void verifyUserFileRestored(VM vm, final long lm) {
+  private void verifyUserFileRestored(VM vm, final long lm) {
     vm.invoke(new SerializableRunnable() {
       public void run() {
         final int pid = DUnitEnv.get().getPid();
@@ -640,8 +606,6 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
           BufferedReader bin = new BufferedReader(fr);
           String content = bin.readLine();
           assertTrue(content.equals("" + pid));
-        } catch (FileNotFoundException e) {
-          fail(e.getMessage());
         } catch (IOException e) {
           fail(e.getMessage());
         }
@@ -649,7 +613,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     });
   }
 
-  protected AsyncInvocation createPersistentRegionAsync(final VM vm) {
+  private AsyncInvocation createPersistentRegionAsync(final VM vm) {
     SerializableRunnable createRegion = new SerializableRunnable("Create persistent region") {
       public void run() {
         Cache cache = getCache();
@@ -670,7 +634,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
         dsf = cache.createDiskStoreFactory();
         dsf.setDiskDirs(getDiskDirs(getUniqueName() + 2));
         dsf.setMaxOplogSize(1);
-        ds = dsf.create(getUniqueName() + 2);
+        dsf.create(getUniqueName() + 2);
         rf.setDiskStoreName(getUniqueName() + 2);
         rf.create("region2");
       }
@@ -678,7 +642,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     return vm.invokeAsync(createRegion);
   }
 
-  protected void createOverflowRegion(final VM vm) {
+  private void createOverflowRegion(final VM vm) {
     SerializableRunnable createRegion = new SerializableRunnable("Create persistent region") {
       public void run() {
         Cache cache = getCache();
@@ -760,14 +724,14 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
       public Object call() throws Exception {
         Cache cache = getCache();
         PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
-        return new TreeSet<Integer>(region.getDataStore().getAllLocalBucketIds());
+        return new TreeSet<>(region.getDataStore().getAllLocalBucketIds());
       }
     };
 
     return (Set<Integer>) vm0.invoke(getBuckets);
   }
 
-  public File[] getDiskDirs(String dsName) {
+  private File[] getDiskDirs(String dsName) {
     File[] dirs = getDiskDirs();
     File[] diskStoreDirs = new File[1];
     diskStoreDirs[0] = new File(dirs[0], dsName);
@@ -775,7 +739,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     return diskStoreDirs;
   }
 
-  protected DataPolicy getDataPolicy() {
+  private DataPolicy getDataPolicy() {
     return DataPolicy.PERSISTENT_PARTITION;
   }
 


[06/11] geode git commit: GEODE-3395 Variable-ize product version and name in user guide - Developing

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/implementing_write_behind_event_handler.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/implementing_write_behind_event_handler.html.md.erb b/geode-docs/developing/events/implementing_write_behind_event_handler.html.md.erb
index 54cf174..76b1248 100644
--- a/geode-docs/developing/events/implementing_write_behind_event_handler.html.md.erb
+++ b/geode-docs/developing/events/implementing_write_behind_event_handler.html.md.erb
@@ -25,13 +25,13 @@ An `AsyncEventListener` asynchronously processes batches of events after they ha
 
 An `AsyncEventListener` instance is serviced by its own dedicated thread in which a callback method is invoked. Events that update a region are placed in an internal `AsyncEventQueue`, and one or more threads dispatch batches of events at a time to the listener implementation.
 
-You can configure an `AsyncEventQueue` to be either serial or parallel. A serial queue is deployed to one Geode member, and it delivers all of a region's events, in order of occurrence, to a configured `AsyncEventListener` implementation. A parallel queue is deployed to multiple Geode members, and each instance of the queue delivers region events, possibly simultaneously, to a local `AsyncEventListener` implementation.
+You can configure an `AsyncEventQueue` to be either serial or parallel. A serial queue is deployed to one <%=vars.product_name%> member, and it delivers all of a region's events, in order of occurrence, to a configured `AsyncEventListener` implementation. A parallel queue is deployed to multiple <%=vars.product_name%> members, and each instance of the queue delivers region events, possibly simultaneously, to a local `AsyncEventListener` implementation.
 
-While a parallel queue provides the best throughput for writing events, it provides less control for ordering those events. With a parallel queue, you cannot preserve event ordering for a region as a whole because multiple Geode servers queue and deliver the region's events at the same time. However, the ordering of events for a given partition (or for a given queue of a distributed region) can be preserved.
+While a parallel queue provides the best throughput for writing events, it provides less control for ordering those events. With a parallel queue, you cannot preserve event ordering for a region as a whole because multiple <%=vars.product_name%> servers queue and deliver the region's events at the same time. However, the ordering of events for a given partition (or for a given queue of a distributed region) can be preserved.
 
 For both serial and parallel queues, you can control the maximum amount of memory that each queue uses, as well as the batch size and frequency for processing batches in the queue. You can also configure queues to persist to disk (instead of simply overflowing to disk) so that write-behind caching can pick up where it left off when a member shuts down and is later restarted.
 
-Optionally, a queue can use multiple threads to dispatch queued events. When you configure multiple threads for a serial queue, the logical queue that is hosted on a Geode member is divided into multiple physical queues, each with a dedicated dispatcher thread. You can then configure whether the threads dispatch queued events by key, by thread, or in the same order in which events were added to the queue. When you configure multiple threads for a parallel queue, each queue hosted on a Geode member is processed by dispatcher threads; the total number of queues created depends on the number of members that host the region.
+Optionally, a queue can use multiple threads to dispatch queued events. When you configure multiple threads for a serial queue, the logical queue that is hosted on a <%=vars.product_name%> member is divided into multiple physical queues, each with a dedicated dispatcher thread. You can then configure whether the threads dispatch queued events by key, by thread, or in the same order in which events were added to the queue. When you configure multiple threads for a parallel queue, each queue hosted on a <%=vars.product_name%> member is processed by dispatcher threads; the total number of queues created depends on the number of members that host the region.
 
 A `GatewayEventFilter` can be placed on the `AsyncEventQueue` to control whether a particular event is sent to a selected `AsyncEventListener`. For example, events associated with sensitive data could be detected and not queued. For more detail, see the Javadocs for `GatewayEventFilter`.
 
@@ -61,11 +61,11 @@ Review the following guidelines before using an AsyncEventListener:
 
 -   If you use an `AsyncEventListener` to implement a write-behind cache listener, your code should check for the possibility that an existing database connection may have been closed due to an earlier exception. For example, check for `Connection.isClosed()` in a catch block and re-create the connection as needed before performing further operations.
 -   Use a serial `AsyncEventQueue` if you need to preserve the order of region events within a thread when delivering events to your listener implementation. Use parallel queues when the order of events within a thread is not important, and when you require maximum throughput for processing events. In both cases, serial and parallel, the order of operations on a given key is preserved within the scope of the thread.
--   You must install the `AsyncEventListener` implementation on a Geode member that hosts the region whose events you want to process.
--   If you configure a parallel `AsyncEventQueue`, deploy the queue on each Geode member that hosts the region.
+-   You must install the `AsyncEventListener` implementation on a <%=vars.product_name%> member that hosts the region whose events you want to process.
+-   If you configure a parallel `AsyncEventQueue`, deploy the queue on each <%=vars.product_name%> member that hosts the region.
 -   You can install a listener on more than one member to provide high availability and guarantee delivery for events, in the event that a member with the active `AsyncEventListener` shuts down. At any given time only one member has an active listener for dispatching events. The listeners on other members remain on standby for redundancy. For best performance and most efficient use of memory, install only one standby listener (redundancy of at most one).
 -   Install no more than one standby listener (redundancy of at most one) for performance and memory reasons.
--   To preserve pending events through member shutdowns, configure Geode to persist the internal queue of the `AsyncEventListener` to an available disk store. By default, any pending events that reside in the internal queue of an `AsyncEventListener` are lost if the active listener's member shuts down.
+-   To preserve pending events through member shutdowns, configure <%=vars.product_name%> to persist the internal queue of the `AsyncEventListener` to an available disk store. By default, any pending events that reside in the internal queue of an `AsyncEventListener` are lost if the active listener's member shuts down.
 -   To ensure high availability and reliable delivery of events, configure the event queue to be both persistent and redundant.
 
 ## <a id="implementing_write_behind_cache_event_handling__section_FB3EB382E37945D9895E09B47A64D6B9" class="no-quick-link"></a>Implementing an AsyncEventListener
@@ -94,7 +94,7 @@ class MyAsyncEventListener implements AsyncEventListener {
 
 ## <a id="implementing_write_behind_cache_event_handling__section_AB80262CFB6D4867B52A5D6D880A5294" class="no-quick-link"></a>Processing AsyncEvents
 
-Use the [AsyncEventListener.processEvents](/releases/latest/javadoc/org/apache/geode/cache/asyncqueue/AsyncEventListener.html) method to process AsyncEvents. This method is called asynchronously when events are queued to be processed. The size of the list reflects the number of batch events where batch size is defined in the AsyncEventQueueFactory. The `processEvents` method returns a boolean; true if the AsyncEvents are processed correctly, and false if any events fail processing. As long as `processEvents` returns false, Geode continues to re-try processing the events.
+Use the [AsyncEventListener.processEvents](/releases/latest/javadoc/org/apache/geode/cache/asyncqueue/AsyncEventListener.html) method to process AsyncEvents. This method is called asynchronously when events are queued to be processed. The size of the list reflects the number of batch events where batch size is defined in the AsyncEventQueueFactory. The `processEvents` method returns a boolean; true if the AsyncEvents are processed correctly, and false if any events fail processing. As long as `processEvents` returns false, <%=vars.product_name%> continues to re-try processing the events.
 
 You can use the `getDeserializedValue` method to obtain cache values for entries that have been updated or created. Since the `getDeserializedValue` method will return a null value for destroyed entries, you should use the `getKey` method to obtain references to cache objects that have been destroyed. Here's an example of processing AsyncEvents:
 
@@ -188,11 +188,11 @@ To configure a write-behind cache listener, you first configure an asynchronous
     AsyncEventQueue asyncQueue = factory.create("sampleQueue", listener);
     ```
 
-2.  If you are using a parallel `AsyncEventQueue`, the gfsh example above requires no alteration, as gfsh applies to all members. If using cache.xml or the Java API to configure your `AsyncEventQueue`, repeat the above configuration in each Geode member that will host the region. Use the same ID and configuration settings for each queue configuration.
+2.  If you are using a parallel `AsyncEventQueue`, the gfsh example above requires no alteration, as gfsh applies to all members. If using cache.xml or the Java API to configure your `AsyncEventQueue`, repeat the above configuration in each <%=vars.product_name%> member that will host the region. Use the same ID and configuration settings for each queue configuration.
     **Note:**
     You can ensure other members use the sample configuration by using the cluster configuration service available in gfsh. See [Overview of the Cluster Configuration Service](../../configuring/cluster_config/gfsh_persist.html).
 
-3.  On each Geode member that hosts the `AsyncEventQueue`, assign the queue to each region that you want to use with the `AsyncEventListener` implementation.
+3.  On each <%=vars.product_name%> member that hosts the `AsyncEventQueue`, assign the queue to each region that you want to use with the `AsyncEventListener` implementation.
 
     **gfsh Configuration**
 
@@ -234,7 +234,7 @@ To configure a write-behind cache listener, you first configure an asynchronous
     mutator.addAsyncEventQueueId("sampleQueue");        
     ```
 
-    See the [Geode API documentation](/releases/latest/javadoc/org/apache/geode/cache/AttributesMutator.html) for more information.
+    See the [<%=vars.product_name%> API documentation](/releases/latest/javadoc/org/apache/geode/cache/AttributesMutator.html) for more information.
 
 4.  Optionally configure persistence and conflation for the queue.
     **Note:**

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/list_of_event_handlers_and_events.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/list_of_event_handlers_and_events.html.md.erb b/geode-docs/developing/events/list_of_event_handlers_and_events.html.md.erb
index cda18ee..a1e763a 100644
--- a/geode-docs/developing/events/list_of_event_handlers_and_events.html.md.erb
+++ b/geode-docs/developing/events/list_of_event_handlers_and_events.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode provides many types of events and event handlers to help you manage your different data and application needs.
+<%=vars.product_name%> provides many types of events and event handlers to help you manage your different data and application needs.
 
 ## <a id="event_handlers_and_events__section_E7B7502F673B43E794884D0F6BF537CF" class="no-quick-link"></a>Event Handlers
 
@@ -79,7 +79,7 @@ Use either cache handlers or membership handlers in any single application. Do n
 <td><code class="ph codeph">MembershipListener</code>
 <p>(org.apache.geode.management .membership.MembershipListener)</p></td>
 <td><code class="ph codeph">MembershipEvent</code></td>
-<td>Use this interface to receive membership events only about peers. This listener's callback methods are invoked when peer members join or leave the Geode distributed system. Callback methods include <code class="ph codeph">memberCrashed</code>, <code class="ph codeph">memberJoined</code>, and <code class="ph codeph">memberLeft</code> (graceful exit).</td>
+<td>Use this interface to receive membership events only about peers. This listener's callback methods are invoked when peer members join or leave the <%=vars.product_name%> distributed system. Callback methods include <code class="ph codeph">memberCrashed</code>, <code class="ph codeph">memberJoined</code>, and <code class="ph codeph">memberLeft</code> (graceful exit).</td>
 </tr>
 <tr>
 <td><code class="ph codeph">RegionMembershipListener</code></td>
@@ -151,7 +151,7 @@ The events in this table are cache events unless otherwise noted.
 <tr>
 <td><code class="ph codeph">EntryEvent</code></td>
 <td><code class="ph codeph">CacheListener</code>, <code class="ph codeph">CacheWriter</code>, <code class="ph codeph">TransactionListener</code> (inside the <code class="ph codeph">TransactionEvent</code>)</td>
-<td>Extends <code class="ph codeph">CacheEvent</code> for entry events. Contains information about an event affecting a data entry in the cache. The information includes the key, the value before this event, and the value after this event. <code class="ph codeph">EntryEvent.getNewValue</code> returns the current value of the data entry. <code class="ph codeph">EntryEvent.getOldValue</code> returns the value before this event if it is available. For a partitioned region, returns the old value if the local cache holds the primary copy of the entry. <code class="ph codeph">EntryEvent</code> provides the Geode transaction ID if available.
+<td>Extends <code class="ph codeph">CacheEvent</code> for entry events. Contains information about an event affecting a data entry in the cache. The information includes the key, the value before this event, and the value after this event. <code class="ph codeph">EntryEvent.getNewValue</code> returns the current value of the data entry. <code class="ph codeph">EntryEvent.getOldValue</code> returns the value before this event if it is available. For a partitioned region, returns the old value if the local cache holds the primary copy of the entry. <code class="ph codeph">EntryEvent</code> provides the <%=vars.product_name%> transaction ID if available.
 <p>You can retrieve serialized values from <code class="ph codeph">EntryEvent</code> using the <code class="ph codeph">getSerialized</code>* methods. This is useful if you get values from one region’s events just to put them into a separate cache region. There is no counterpart <code class="ph codeph">put</code> function as the put recognizes that the value is serialized and bypasses the serialization step.</p></td>
 </tr>
 <tr>

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/tune_client_server_event_messaging.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/tune_client_server_event_messaging.html.md.erb b/geode-docs/developing/events/tune_client_server_event_messaging.html.md.erb
index db43ed5..06e14b1 100644
--- a/geode-docs/developing/events/tune_client_server_event_messaging.html.md.erb
+++ b/geode-docs/developing/events/tune_client_server_event_messaging.html.md.erb
@@ -28,10 +28,10 @@ A single client thread receives and processes messages from the server, tracking
 
 The client’s message tracking list holds the highest sequence ID of any message received for each originating thread. The list can become quite large in systems where there are many different threads coming and going and doing work on the cache. After a thread dies, its tracking entry is not needed. To avoid maintaining tracking information for threads that have died, the client expires entries that have had no activity for more than the `subscription-message-tracking-timeout`.
 
--   **[Conflate the Server Subscription Queue](../../developing/events/conflate_server_subscription_queue.html)**
+-   **[Conflate the Server Subscription Queue](conflate_server_subscription_queue.html)**
 
--   **[Limit the Server's Subscription Queue Memory Use](../../developing/events/limit_server_subscription_queue_size.html)**
+-   **[Limit the Server's Subscription Queue Memory Use](limit_server_subscription_queue_size.html)**
 
--   **[Tune the Client's Subscription Message Tracking Timeout](../../developing/events/tune_client_message_tracking_timeout.html)**
+-   **[Tune the Client's Subscription Message Tracking Timeout](tune_client_message_tracking_timeout.html)**
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/events/writing_callbacks_that_modify_the_cache.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/events/writing_callbacks_that_modify_the_cache.html.md.erb b/geode-docs/developing/events/writing_callbacks_that_modify_the_cache.html.md.erb
index 7b201bc..56a3b12 100644
--- a/geode-docs/developing/events/writing_callbacks_that_modify_the_cache.html.md.erb
+++ b/geode-docs/developing/events/writing_callbacks_that_modify_the_cache.html.md.erb
@@ -23,20 +23,20 @@ Event handlers are synchronous. If you need to change the cache or perform any o
 
 ## <a id="writing_callbacks_that_modify_the_cache__section_98E49363C91945DEB0A3B2FD9A209969" class="no-quick-link"></a>Operations to Avoid in Event Handlers
 
-Do not perform distributed operations of any kind directly from your event handler. Geode is a highly distributed system and many operations that may seem local invoke distributed operations.
+Do not perform distributed operations of any kind directly from your event handler. <%=vars.product_name%> is a highly distributed system and many operations that may seem local invoke distributed operations.
 
 These are common distributed operations that can get you into trouble:
 
 -   Calling `Region` methods, on the event's region or any other region.
--   Using the Geode `DistributedLockService`.
+-   Using the <%=vars.product_name%> `DistributedLockService`.
 -   Modifying region attributes.
--   Executing a function through the Geode `FunctionService`.
+-   Executing a function through the <%=vars.product_name%> `FunctionService`.
 
-To be on the safe side, do not make any calls to the Geode API directly from your event handler. Make all Geode API calls from within a separate thread or executor.
+To be on the safe side, do not make any calls to the <%=vars.product_name%> API directly from your event handler. Make all <%=vars.product_name%> API calls from within a separate thread or executor.
 
 ## <a id="writing_callbacks_that_modify_the_cache__section_78648D4177E14EA695F0B059E336137C" class="no-quick-link"></a>How to Perform Distributed Operations Based on Events
 
-If you need to use the Geode API from your handlers, make your work asynchronous to the event handler. You can spawn a separate thread or use a solution like the `java.util.concurrent.Executor` interface.
+If you need to use the <%=vars.product_name%> API from your handlers, make your work asynchronous to the event handler. You can spawn a separate thread or use a solution like the `java.util.concurrent.Executor` interface.
 
 This example shows a serial executor where the callback creates a `Runnable` that can be pulled off a queue and run by another object. This preserves the ordering of events.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/eviction/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/eviction/chapter_overview.html.md.erb b/geode-docs/developing/eviction/chapter_overview.html.md.erb
index c5d1417..1cd9814 100644
--- a/geode-docs/developing/eviction/chapter_overview.html.md.erb
+++ b/geode-docs/developing/eviction/chapter_overview.html.md.erb
@@ -23,11 +23,11 @@ Use eviction to control data region size.
 
 <a id="eviction__section_C3409270DD794822B15E819E2276B21A"></a>
 
--   **[How Eviction Works](../../developing/eviction/how_eviction_works.html)**
+-   **[How Eviction Works](how_eviction_works.html)**
 
-    Eviction settings cause Apache Geode to work to keep a region's resource use under a specified level by removing least recently used (LRU) entries to make way for new entries.
+    Eviction settings cause <%=vars.product_name_long%> to work to keep a region's resource use under a specified level by removing least recently used (LRU) entries to make way for new entries.
 
--   **[Configure Data Eviction](../../developing/eviction/configuring_data_eviction.html)**
+-   **[Configure Data Eviction](configuring_data_eviction.html)**
 
     Use eviction controllers to configure the eviction-attributes region attribute settings to keep your region within a specified limit.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/eviction/configuring_data_eviction.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/eviction/configuring_data_eviction.html.md.erb b/geode-docs/developing/eviction/configuring_data_eviction.html.md.erb
index 6c22284..530c22f 100644
--- a/geode-docs/developing/eviction/configuring_data_eviction.html.md.erb
+++ b/geode-docs/developing/eviction/configuring_data_eviction.html.md.erb
@@ -22,21 +22,21 @@ limitations under the License.
 Use eviction controllers to configure the eviction-attributes region attribute settings to keep your region within a specified limit.
 
 <a id="configuring_data_eviction__section_8515EC9635C342C0916EE9E6120E2AC9"></a>
-Eviction controllers monitor region and memory use and, when the limit is reached, remove older entries to make way for new data. For heap percentage, the controller used is the Geode resource manager, configured in conjunction with the JVM's garbage collector for optimum performance.
+Eviction controllers monitor region and memory use and, when the limit is reached, remove older entries to make way for new data. For heap percentage, the controller used is the <%=vars.product_name%> resource manager, configured in conjunction with the JVM's garbage collector for optimum performance.
 
 Configure data eviction as follows. You do not need to perform these steps in the sequence shown.
 
 1.  Decide whether to evict based on:
     -   Entry count (useful if your entry sizes are relatively uniform).
     -   Total bytes used. In partitioned regions, this is set using `local-max-memory`. In non-partitioned, it is set in `eviction-attributes`.
-    -   Percentage of application heap used. This uses the Geode resource manager. When the manager determines that eviction is required, the manager orders the eviction controller to start evicting from all regions where the eviction algorithm is set to `lru-heap-percentage`. Eviction continues until the manager calls a halt. Geode evicts the least recently used entry hosted by the member for the region. See [Managing Heap and Off-heap Memory](../../managing/heap_use/heap_management.html#resource_manager).
+    -   Percentage of application heap used. This uses the <%=vars.product_name%> resource manager. When the manager determines that eviction is required, the manager orders the eviction controller to start evicting from all regions where the eviction algorithm is set to `lru-heap-percentage`. Eviction continues until the manager calls a halt. <%=vars.product_name%> evicts the least recently used entry hosted by the member for the region. See [Managing Heap and Off-heap Memory](../../managing/heap_use/heap_management.html#resource_manager).
 
 2.  Decide what action to take when the limit is reached:
     -   Locally destroy the entry.
     -   Overflow the entry data to disk. See [Persistence and Overflow](../storing_data_on_disk/chapter_overview.html).
 
 3.  Decide the maximum amount of data to allow in the member for the eviction measurement indicated. This is the maximum for all storage for the region in the member. For partitioned regions, this is the total for all buckets stored in the member for the region - including any secondary buckets used for redundancy.
-4.  Decide whether to program a custom sizer for your region. If you are able to provide such a class, it might be faster than the standard sizing done by Geode. Your custom class must follow the guidelines for defining custom classes and, additionally, must implement `org.apache.geode.cache.util.ObjectSizer`. See [Requirements for Using Custom Classes in Data Caching](../../basic_config/data_entries_custom_classes/using_custom_classes.html).
+4.  Decide whether to program a custom sizer for your region. If you are able to provide such a class, it might be faster than the standard sizing done by <%=vars.product_name%>. Your custom class must follow the guidelines for defining custom classes and, additionally, must implement `org.apache.geode.cache.util.ObjectSizer`. See [Requirements for Using Custom Classes in Data Caching](../../basic_config/data_entries_custom_classes/using_custom_classes.html).
 
 **Note:**
 You can also configure Regions using the gfsh command-line interface, however, you cannot configure `eviction-attributes` using gfsh. See [Region Commands](../../tools_modules/gfsh/quick_ref_commands_by_area.html#topic_EF03119A40EE492984F3B6248596E1DD) and [Disk Store Commands](../../tools_modules/gfsh/quick_ref_commands_by_area.html#topic_1ACC91B493EE446E89EC7DBFBBAE00EA).

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/eviction/how_eviction_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/eviction/how_eviction_works.html.md.erb b/geode-docs/developing/eviction/how_eviction_works.html.md.erb
index a714253..0c11f0b 100644
--- a/geode-docs/developing/eviction/how_eviction_works.html.md.erb
+++ b/geode-docs/developing/eviction/how_eviction_works.html.md.erb
@@ -19,16 +19,16 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Eviction settings cause Apache Geode to work to keep a region's resource use under a specified level by removing least recently used (LRU) entries to make way for new entries.
+Eviction settings cause <%=vars.product_name_long%> to work to keep a region's resource use under a specified level by removing least recently used (LRU) entries to make way for new entries.
 
 <a id="how_eviction_works__section_C3409270DD794822B15E819E2276B21A"></a>
 You configure for eviction based on entry count, percentage of available heap, and absolute memory usage. You also configure what to do when you need to evict: destroy entries or overflow them to disk. See [Persistence and Overflow](../storing_data_on_disk/chapter_overview.html).
 
-When Geode determines that adding or updating an entry would take the region over the specified level, it overflows or removes enough older entries to make room. For entry count eviction, this means a one-to-one trade of an older entry for the newer one. For the memory settings, the number of older entries that need to be removed to make space depends entirely on the relative sizes of the older and newer entries.
+When <%=vars.product_name%> determines that adding or updating an entry would take the region over the specified level, it overflows or removes enough older entries to make room. For entry count eviction, this means a one-to-one trade of an older entry for the newer one. For the memory settings, the number of older entries that need to be removed to make space depends entirely on the relative sizes of the older and newer entries.
 
 ## <a id="how_eviction_works__section_69E2AA453EDE4E088D1C3332C071AFE1" class="no-quick-link"></a>Eviction in Partitioned Regions
 
-In partitioned regions, Geode removes the oldest entry it can find *in the bucket where the new entry operation is being performed*. Geode maintains LRU entry information on a bucket-by-bucket basis, as the cost of maintaining information across the partitioned region would be too great a performance hit.
+In partitioned regions, <%=vars.product_name%> removes the oldest entry it can find *in the bucket where the new entry operation is being performed*. <%=vars.product_name%> maintains LRU entry information on a bucket-by-bucket basis, as the cost of maintaining information across the partitioned region would be too great a performance hit.
 
 -   For memory and entry count eviction, LRU eviction is done in the bucket where the new entry operation is being performed until the overall size of the combined buckets in the member has dropped enough to perform the operation without going over the limit.
 -   For heap eviction, each partitioned region bucket is treated as if it were a separate region, with each eviction action only considering the LRU for the bucket, and not the partitioned region as a whole.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/expiration/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/expiration/chapter_overview.html.md.erb b/geode-docs/developing/expiration/chapter_overview.html.md.erb
index 546af32..3764b6f 100644
--- a/geode-docs/developing/expiration/chapter_overview.html.md.erb
+++ b/geode-docs/developing/expiration/chapter_overview.html.md.erb
@@ -21,11 +21,11 @@ limitations under the License.
 
 Use expiration to keep data current by removing stale entries. You can also use it to remove entries you are not using so your region uses less space. Expired entries are reloaded the next time they are requested.
 
--   **[How Expiration Works](../../developing/expiration/how_expiration_works.html)**
+-   **[How Expiration Works](how_expiration_works.html)**
 
     Expiration removes old entries and entries that you are not using. You can destroy or invalidate entries.
 
--   **[Configure Data Expiration](../../developing/expiration/configuring_data_expiration.html)**
+-   **[Configure Data Expiration](configuring_data_expiration.html)**
 
     Configure the type of expiration and the expiration action to use.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/expiration/how_expiration_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/expiration/how_expiration_works.html.md.erb b/geode-docs/developing/expiration/how_expiration_works.html.md.erb
index 4ec5015..e005581 100644
--- a/geode-docs/developing/expiration/how_expiration_works.html.md.erb
+++ b/geode-docs/developing/expiration/how_expiration_works.html.md.erb
@@ -30,14 +30,14 @@ This figure shows two basic expiration settings for a producer/consumer system.
 
 ## <a id="how_expiration_works__section_B6C55A610F4243ED8F1986E8A98858CF" class="no-quick-link"></a>Expiration Types
 
-Apache Geode uses the following expiration types:
+<%=vars.product_name_long%> uses the following expiration types:
 
 -   **Time to live (TTL)**. The amount of time, in seconds, the object may remain in the cache after the last creation or update. For entries, the counter is set to zero for create and put operations. Region counters are reset when the region is created and when an entry has its counter reset. The TTL expiration attributes are `region-time-to-live` and `entry-time-to-live`.
 -   **Idle timeout**. The amount of time, in seconds, the object may remain in the cache after the last access. The idle timeout counter for an object is reset any time its TTL counter is reset. In addition, an entry’s idle timeout counter is reset any time the entry is accessed through a get operation or a netSearch . The idle timeout counter for a region is reset whenever the idle timeout is reset for one of its entries. Idle timeout expiration attributes are: `region-idle-time` and `entry-idle-time`.
 
 ## <a id="how_expiration_works__section_BA995343EF584104B9853CFE4CAD88AD" class="no-quick-link"></a>Expiration Actions
 
-Apache Geode uses the following expiration actions:
+<%=vars.product_name_long%> uses the following expiration actions:
 
 -   destroy
 -   local destroy

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/function_exec/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/function_exec/chapter_overview.html.md.erb b/geode-docs/developing/function_exec/chapter_overview.html.md.erb
index c85e9c8..46d39f8 100644
--- a/geode-docs/developing/function_exec/chapter_overview.html.md.erb
+++ b/geode-docs/developing/function_exec/chapter_overview.html.md.erb
@@ -31,6 +31,6 @@ A function is a body of code that resides on a server and that an application ca
 
 -   **[How Function Execution Works](how_function_execution_works.html)**
 
--   **[Executing a Function in Apache Geode](function_execution.html)**
+-   **[Executing a Function in <%=vars.product_name_long%>](function_execution.html)**
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/function_exec/function_execution.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/function_exec/function_execution.html.md.erb b/geode-docs/developing/function_exec/function_execution.html.md.erb
index 221098b..a7ce138 100644
--- a/geode-docs/developing/function_exec/function_execution.html.md.erb
+++ b/geode-docs/developing/function_exec/function_execution.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Executing a Function in Apache Geode
----
+<% set_title("Executing a Function in", product_name_long) %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -37,7 +35,7 @@ Code the methods you need for the function. These steps do not have to be done i
 
 1.  Code `getId` to return a unique name for your function. You can use this name to access the function through the `FunctionService` API.
 2.  For high availability:
-    1.  Code `isHa` to return true to indicate to Geode that it can re-execute your function after one or more members fails
+    1.  Code `isHa` to return true to indicate to <%=vars.product_name%> that it can re-execute your function after one or more members fails
     2.  Code your function to return a result
     3.  Code `hasResult` to return true
 
@@ -57,7 +55,7 @@ Code the methods you need for the function. These steps do not have to be done i
             **Note:**
             When you use `PartitionRegionHelper.getLocalDataForContext`, `putIfAbsent` may not return expected results if you are working on local data set instead of the region.
 
-    4.  To propagate an error condition or exception back to the caller of the function, throw a FunctionException from the `execute` method. Geode transmits the exception back to the caller as if it had been thrown on the calling side. See the Java API documentation for [FunctionException](/releases/latest/javadoc/org/apache/geode/cache/execute/FunctionException.html) for more information.
+    4.  To propagate an error condition or exception back to the caller of the function, throw a FunctionException from the `execute` method. <%=vars.product_name%> transmits the exception back to the caller as if it had been thrown on the calling side. See the Java API documentation for [FunctionException](/releases/latest/javadoc/org/apache/geode/cache/execute/FunctionException.html) for more information.
 
 Example function code:
 
@@ -114,7 +112,7 @@ When you deploy a JAR file that contains a Function (in other words, contains a
 To register a function by using `gfsh`:
 
 1.  Package your class files into a JAR file.
-2.  Start a `gfsh` prompt. If necessary, start a Locator and connect to the Geode distributed system where you want to run the function.
+2.  Start a `gfsh` prompt. If necessary, start a Locator and connect to the <%=vars.product_name%> distributed system where you want to run the function.
 3.  At the gfsh prompt, type the following command:
 
     ``` pre
@@ -125,7 +123,7 @@ To register a function by using `gfsh`:
 
 If another JAR file is deployed (either with the same JAR filename or another filename) with the same Function, the new implementation of the Function will be registered, overwriting the old one. If a JAR file is undeployed, any Functions that were auto-registered at the time of deployment will be unregistered. Since deploying a JAR file that has the same name multiple times results in the JAR being un-deployed and re-deployed, Functions in the JAR will be unregistered and re-registered each time this occurs. If a Function with the same ID is registered from multiple differently named JAR files, the Function will be unregistered if either of those JAR files is re-deployed or un-deployed.
 
-See [Deploying Application JARs to Apache Geode Members](../../configuring/cluster_config/deploying_application_jars.html#concept_4436C021FB934EC4A330D27BD026602C) for more details on deploying JAR files.
+See [Deploying Application JARs to <%=vars.product_name_long%> Members](../../configuring/cluster_config/deploying_application_jars.html#concept_4436C021FB934EC4A330D27BD026602C) for more details on deploying JAR files.
 
 ## <a id="function_execution__section_1D1056F843044F368FB76F47061FCD50" class="no-quick-link"></a>Register the Function Programmatically
 
@@ -169,7 +167,7 @@ In every member where you want to explicitly execute the function and process th
 **Running the Function Using gfsh**
 
 1.  Start a gfsh prompt.
-2.  If necessary, start a Locator and connect to the Geode distributed system where you want to run the function.
+2.  If necessary, start a Locator and connect to the <%=vars.product_name%> distributed system where you want to run the function.
 3.  At the gfsh prompt, type the following command:
 
     ``` pre
@@ -228,12 +226,12 @@ ResultCollector rc = execution.execute(function);
 List result = (List)rc.getResult();
 ```
 
-Geode’s default `ResultCollector` collects all results into an `ArrayList`. Its `getResult` methods block until all results are received. Then they return the full result set.
+<%=vars.product_name%>’s default `ResultCollector` collects all results into an `ArrayList`. Its `getResult` methods block until all results are received. Then they return the full result set.
 
 To customize results collecting:
 
 1.  Write a class that extends `ResultCollector` and code the methods to store and retrieve the results as you need. Note that the methods are of two types:
-    1.  `addResult` and `endResults` are called by Geode when results arrive from the `Function` instance `SendResults` methods
+    1.  `addResult` and `endResults` are called by <%=vars.product_name%> when results arrive from the `Function` instance `SendResults` methods
     2.  `getResult` is available to your executing application (the one that calls `Execution.execute`) to retrieve the results
 
 2.  Use high availability for `onRegion` functions that have been coded for it:

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/function_exec/how_function_execution_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/function_exec/how_function_execution_works.html.md.erb b/geode-docs/developing/function_exec/how_function_execution_works.html.md.erb
index a72045f..ae80b01 100644
--- a/geode-docs/developing/function_exec/how_function_execution_works.html.md.erb
+++ b/geode-docs/developing/function_exec/how_function_execution_works.html.md.erb
@@ -21,7 +21,7 @@ limitations under the License.
 
 ## <a id="how_function_execution_works__section_881D2FF6761B4D689DDB46C650E2A2E1" class="no-quick-link"></a>Where Functions Are Executed
 
-You can execute data-independent functions or data-dependent functions in Geode in the following places:
+You can execute data-independent functions or data-dependent functions in <%=vars.product_name%> in the following places:
 
 **For Data-independent Functions**
 
@@ -39,13 +39,13 @@ See the `org.apache.geode.cache.execute.FunctionService` Java API documentation
 
 The following things occur when executing a function:
 
-1.  When you call the `execute` method on the `Execution` object, Geode invokes the function on all members where it needs to run. The locations are determined by the `FunctionService` `on*` method calls, region configuration, and any filters.
+1.  When you call the `execute` method on the `Execution` object, <%=vars.product_name%> invokes the function on all members where it needs to run. The locations are determined by the `FunctionService` `on*` method calls, region configuration, and any filters.
 2.  If the function has results, they are returned to the `addResult` method call in a `ResultCollector` object.
 3.  The originating member collects results using `ResultCollector.getResult`.
 
 ## <a id="how_function_execution_works__section_14FF9932C7134C5584A14246BB4D4FF6" class="no-quick-link"></a>Highly Available Functions
 
-Generally, function execution errors are returned to the calling application. You can code for high availability for `onRegion` functions that return a result, so Geode automatically retries a function if it does not execute successfully. You must code and configure the function to be highly available, and the calling application must invoke the function using the results collector `getResult` method.
+Generally, function execution errors are returned to the calling application. You can code for high availability for `onRegion` functions that return a result, so <%=vars.product_name%> automatically retries a function if it does not execute successfully. You must code and configure the function to be highly available, and the calling application must invoke the function using the results collector `getResult` method.
 
 When a failure (such as an execution error or member crash while executing) occurs, the system responds by:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/outside_data_sources/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/outside_data_sources/chapter_overview.html.md.erb b/geode-docs/developing/outside_data_sources/chapter_overview.html.md.erb
index a008ede..3d8c30c 100644
--- a/geode-docs/developing/outside_data_sources/chapter_overview.html.md.erb
+++ b/geode-docs/developing/outside_data_sources/chapter_overview.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Apache Geode has application plug-ins to read data into the cache and write it out.
+<%=vars.product_name_long%> has application plug-ins to read data into the cache and write it out.
 
 <a id="outside_data_sources__section_100B707BB812430E8D9CFDE3BE4698D1"></a>
 The application plug-ins:

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/outside_data_sources/how_data_loaders_work.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/outside_data_sources/how_data_loaders_work.html.md.erb b/geode-docs/developing/outside_data_sources/how_data_loaders_work.html.md.erb
index 4f309a0..b342e41 100644
--- a/geode-docs/developing/outside_data_sources/how_data_loaders_work.html.md.erb
+++ b/geode-docs/developing/outside_data_sources/how_data_loaders_work.html.md.erb
@@ -24,7 +24,7 @@ By default, a region has no data loader defined. Plug an application-defined loa
 <a id="how_data_loaders_work__section_1E600469D223498DB49446434CE9B0B4"></a>
 The loader is called on cache misses during get operations, and it populates the cache with the new entry value in addition to returning the value to the calling thread.
 
-A loader can be configured to load data into the Geode cache from an outside data store. To do the reverse operation, writing data from the Geode cache to an outside data store, use a cache writer event handler. See [Implementing Cache Event Handlers](../events/implementing_cache_event_handlers.html).
+A loader can be configured to load data into the <%=vars.product_name%> cache from an outside data store. To do the reverse operation, writing data from the <%=vars.product_name%> cache to an outside data store, use a cache writer event handler. See [Implementing Cache Event Handlers](../events/implementing_cache_event_handlers.html).
 
 How to install your cache loader depends on the type of region.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/outside_data_sources/sync_outside_data.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/outside_data_sources/sync_outside_data.html.md.erb b/geode-docs/developing/outside_data_sources/sync_outside_data.html.md.erb
index 728b664..767e507 100644
--- a/geode-docs/developing/outside_data_sources/sync_outside_data.html.md.erb
+++ b/geode-docs/developing/outside_data_sources/sync_outside_data.html.md.erb
@@ -21,15 +21,15 @@ limitations under the License.
 
 Keep your distributed cache in sync with an outside data source by programming and installing application plug-ins for your region.
 
--   **[Overview of Outside Data Sources](../../developing/outside_data_sources/chapter_overview.html)**
+-   **[Overview of Outside Data Sources](chapter_overview.html)**
 
-    Apache Geode has application plug-ins to read data into the cache and write it out.
+    <%=vars.product_name_long%> has application plug-ins to read data into the cache and write it out.
 
--   **[How Data Loaders Work](../../developing/outside_data_sources/how_data_loaders_work.html)**
+-   **[How Data Loaders Work](how_data_loaders_work.html)**
 
     By default, a region has no data loader defined. Plug an application-defined loader into any region by setting the region attribute cache-loader on the members that host data for the region.
 
--   **[Implement a Data Loader](../../developing/outside_data_sources/implementing_data_loaders.html)**
+-   **[Implement a Data Loader](implementing_data_loaders.html)**
 
     Program a data loader and configure your region to use it.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/chapter_overview.html.md.erb b/geode-docs/developing/partitioned_regions/chapter_overview.html.md.erb
index e450ee5..0d41532 100644
--- a/geode-docs/developing/partitioned_regions/chapter_overview.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/chapter_overview.html.md.erb
@@ -21,44 +21,44 @@ limitations under the License.
 
 In addition to basic region management, partitioned regions include options for high availability, data location control, and data balancing across the distributed system.
 
--   **[Understanding Partitioning](../../developing/partitioned_regions/how_partitioning_works.html)**
+-   **[Understanding Partitioning](how_partitioning_works.html)**
 
     To use partitioned regions, you should understand how they work and your options for managing them.
 
--   **[Configuring Partitioned Regions](../../developing/partitioned_regions/managing_partitioned_regions.html)**
+-   **[Configuring Partitioned Regions](managing_partitioned_regions.html)**
 
     Plan the configuration and ongoing management of your partitioned region for host and accessor members and configure the regions for startup.
 
--   **[Configuring the Number of Buckets for a Partitioned Region](../../developing/partitioned_regions/configuring_bucket_for_pr.html)**
+-   **[Configuring the Number of Buckets for a Partitioned Region](configuring_bucket_for_pr.html)**
 
     Decide how many buckets to assign to your partitioned region and set the configuration accordingly.
 
--   **[Custom-Partitioning and Colocating Data](../../developing/partitioned_regions/overview_custom_partitioning_and_data_colocation.html)**
+-   **[Custom-Partitioning and Colocating Data](overview_custom_partitioning_and_data_colocation.html)**
 
-    You can customize how Apache Geode groups your partitioned region data with custom partitioning and data colocation.
+    You can customize how <%=vars.product_name_long%> groups your partitioned region data with custom partitioning and data colocation.
 
--   **[Configuring High Availability for Partitioned Regions](../../developing/partitioned_regions/overview_how_pr_ha_works.html)**
+-   **[Configuring High Availability for Partitioned Regions](overview_how_pr_ha_works.html)**
 
-    By default, Apache Geode stores only a single copy of your partitioned region data among the region's data stores. You can configure Geode to maintain redundant copies of your partitioned region data for high availability.
+    By default, <%=vars.product_name_long%> stores only a single copy of your partitioned region data among the region's data stores. You can configure <%=vars.product_name%> to maintain redundant copies of your partitioned region data for high availability.
 
--   **[Configuring Single-Hop Client Access to Server-Partitioned Regions](../../developing/partitioned_regions/overview_how_pr_single_hop_works.html)**
+-   **[Configuring Single-Hop Client Access to Server-Partitioned Regions](overview_how_pr_single_hop_works.html)**
 
     Single-hop data access enables the client pool to track where a partitioned region’s data is hosted in the servers. To access a single entry, the client directly contacts the server that hosts the key, in a single hop.
 
--   **[Rebalancing Partitioned Region Data](../../developing/partitioned_regions/rebalancing_pr_data.html)**
+-   **[Rebalancing Partitioned Region Data](rebalancing_pr_data.html)**
 
     In a distributed system with minimal contention to the concurrent threads reading or updating from the members, you can use rebalancing to dynamically increase or decrease your data and processing capacity.
 
-- **[Automated Rebalancing of Partitioned Region Data](../../developing/partitioned_regions/automated_rebalance.html)**
+- **[Automated Rebalancing of Partitioned Region Data](automated_rebalance.html)**
 
     The automated rebalance feature triggers a rebalance operation
 based on a time schedule.
 
--   **[Checking Redundancy in Partitioned Regions](../../developing/partitioned_regions/checking_region_redundancy.html)**
+-   **[Checking Redundancy in Partitioned Regions](checking_region_redundancy.html)**
 
     Under some circumstances, it can be important to verify that your partitioned region data is redundant and that upon member restart, redundancy has been recovered properly across partitioned region members.
 
--   **[Moving Partitioned Region Data to Another Member](../../developing/partitioned_regions/moving_partitioned_data.html)**
+-   **[Moving Partitioned Region Data to Another Member](moving_partitioned_data.html)**
 
     You can use the `PartitionRegionHelper` `moveBucketByKey` and `moveData` methods to explicitly move partitioned region data from one member to another.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/colocating_partitioned_region_data.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/colocating_partitioned_region_data.html.md.erb b/geode-docs/developing/partitioned_regions/colocating_partitioned_region_data.html.md.erb
index c20e30e..962c21e 100644
--- a/geode-docs/developing/partitioned_regions/colocating_partitioned_region_data.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/colocating_partitioned_region_data.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-By default, Geode allocates the data locations for a partitioned region independent of the data locations for any other partitioned region. You can change this policy for any group of partitioned regions, so that cross-region, related data is all hosted by the same member. This colocation speeds queries and other operations that access data from the regions.
+By default, <%=vars.product_name%> allocates the data locations for a partitioned region independent of the data locations for any other partitioned region. You can change this policy for any group of partitioned regions, so that cross-region, related data is all hosted by the same member. This colocation speeds queries and other operations that access data from the regions.
 
 <a id="colocating_partitioned_region_data__section_131EC040055E48A6B35E981B5C845A65"></a>
 **Note:**
@@ -39,7 +39,7 @@ Data colocation between partitioned regions generally improves the performance o
 **Procedure**
 
 1.  Identify one region as the central region, with which data in the other regions is explicitly colocated. If you use persistence for any of the regions, you must persist the central region.
-    1.  Create the central region before you create the others, either in the cache.xml or your code. Regions in the XML are created before regions in the code, so if you create any of your colocated regions in the XML, you must create the central region in the XML before the others. Geode will verify its existence when the others are created and return `IllegalStateException` if the central region is not there. Do not add any colocation specifications to this central region.
+    1.  Create the central region before you create the others, either in the cache.xml or your code. Regions in the XML are created before regions in the code, so if you create any of your colocated regions in the XML, you must create the central region in the XML before the others. <%=vars.product_name%> will verify its existence when the others are created and return `IllegalStateException` if the central region is not there. Do not add any colocation specifications to this central region.
     2.  For all other regions, in the region partition attributes, provide the central region's name in the `colocated-with` attribute. Use one of these methods:
         -   XML:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/configuring_bucket_for_pr.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/configuring_bucket_for_pr.html.md.erb b/geode-docs/developing/partitioned_regions/configuring_bucket_for_pr.html.md.erb
index ccb7e71..f8dc971 100644
--- a/geode-docs/developing/partitioned_regions/configuring_bucket_for_pr.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/configuring_bucket_for_pr.html.md.erb
@@ -22,7 +22,7 @@ limitations under the License.
 Decide how many buckets to assign to your partitioned region and set the configuration accordingly.
 
 <a id="configuring_total_buckets__section_DF52B2BF467F4DB4B8B3D16A79EFCA39"></a>
-The total number of buckets for the partitioned region determines the granularity of data storage and thus how evenly the data can be distributed. Geode distributes the buckets as evenly as possible across the data stores. The number of buckets is fixed after region creation.
+The total number of buckets for the partitioned region determines the granularity of data storage and thus how evenly the data can be distributed. <%=vars.product_name%> distributes the buckets as evenly as possible across the data stores. The number of buckets is fixed after region creation.
 
 The partition attribute `total-num-buckets` sets the number for the entire partitioned region across all participating members. Set it using one of the following:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/configuring_ha_for_pr.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/configuring_ha_for_pr.html.md.erb b/geode-docs/developing/partitioned_regions/configuring_ha_for_pr.html.md.erb
index c084f4a..d77006c 100644
--- a/geode-docs/developing/partitioned_regions/configuring_ha_for_pr.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/configuring_ha_for_pr.html.md.erb
@@ -25,33 +25,33 @@ Here are the main steps for configuring high availability for a partitioned regi
 
 1.  Set the number of redundant copies the system should maintain of the region data. See [Set the Number of Redundant Copies](set_pr_redundancy.html#set_pr_redundancy). 
 2.  (Optional) If you want to group your data store members into redundancy zones, configure them accordingly. See [Configure Redundancy Zones for Members](set_redundancy_zones.html#set_redundancy_zones). 
-3.  (Optional) If you want Geode to only place redundant copies on different physical machines, configure for that. See [Set Enforce Unique Host](set_enforce_unique_host.html#set_pr_redundancy). 
-4.  Decide how to manage redundancy recovery and change Geode's default behavior as needed. 
+3.  (Optional) If you want <%=vars.product_name%> to only place redundant copies on different physical machines, configure for that. See [Set Enforce Unique Host](set_enforce_unique_host.html#set_pr_redundancy). 
+4.  Decide how to manage redundancy recovery and change <%=vars.product_name%>'s default behavior as needed. 
     - **After a member crashes**. If you want automatic redundancy recovery, change the configuration for that. See [Configure Member Crash Redundancy Recovery for a Partitioned Region](set_crash_redundancy_recovery.html#set_crash_redundancy_recovery). 
     - **After a member joins**. If you do *not* want immediate, automatic redundancy recovery, change the configuration for that. See [Configure Member Join Redundancy Recovery for a Partitioned Region](set_join_redundancy_recovery.html#set_join_redundancy_recovery). 
 
-5.  Decide how many buckets Geode should attempt to recover in parallel when performing redundancy recovery. By default, the system recovers up to 8 buckets in parallel. Use the `gemfire.MAX_PARALLEL_BUCKET_RECOVERIES` system property to increase or decrease the maximum number of buckets to recover in parallel any time redundancy recovery is performed.
+5.  Decide how many buckets <%=vars.product_name%> should attempt to recover in parallel when performing redundancy recovery. By default, the system recovers up to 8 buckets in parallel. Use the `gemfire.MAX_PARALLEL_BUCKET_RECOVERIES` system property to increase or decrease the maximum number of buckets to recover in parallel any time redundancy recovery is performed.
 6.  For all but fixed partitioned regions, review the points at which you kick off rebalancing. Redundancy recovery is done automatically at the start of any rebalancing. This is most important if you run with no automated recovery after member crashes or joins. See [Rebalancing Partitioned Region Data](rebalancing_pr_data.html#rebalancing_pr_data). 
 
 During runtime, you can add capacity by adding new members for the region. For regions that do not use fixed partitioning, you can also kick off a rebalancing operation to spread the region buckets among all members.
 
--   **[Set the Number of Redundant Copies](../../developing/partitioned_regions/set_pr_redundancy.html)**
+-   **[Set the Number of Redundant Copies](set_pr_redundancy.html)**
 
     Configure in-memory high availability for your partitioned region by specifying the number of secondary copies you want to maintain in the region's data stores.
 
--   **[Configure Redundancy Zones for Members](../../developing/partitioned_regions/set_redundancy_zones.html)**
+-   **[Configure Redundancy Zones for Members](set_redundancy_zones.html)**
 
-    Group members into redundancy zones so Geode will separate redundant data copies into different zones.
+    Group members into redundancy zones so <%=vars.product_name%> will separate redundant data copies into different zones.
 
--   **[Set Enforce Unique Host](../../developing/partitioned_regions/set_enforce_unique_host.html)**
+-   **[Set Enforce Unique Host](set_enforce_unique_host.html)**
 
-    Configure Geode to use only unique physical machines for redundant copies of partitioned region data.
+    Configure <%=vars.product_name%> to use only unique physical machines for redundant copies of partitioned region data.
 
--   **[Configure Member Crash Redundancy Recovery for a Partitioned Region](../../developing/partitioned_regions/set_crash_redundancy_recovery.html)**
+-   **[Configure Member Crash Redundancy Recovery for a Partitioned Region](set_crash_redundancy_recovery.html)**
 
     Configure whether and how redundancy is recovered in a partition region after a member crashes.
 
--   **[Configure Member Join Redundancy Recovery for a Partitioned Region](../../developing/partitioned_regions/set_join_redundancy_recovery.html)**
+-   **[Configure Member Join Redundancy Recovery for a Partitioned Region](set_join_redundancy_recovery.html)**
 
     Configure whether and how redundancy is recovered in a partition region after a member joins.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/custom_partitioning_and_data_colocation.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/custom_partitioning_and_data_colocation.html.md.erb b/geode-docs/developing/partitioned_regions/custom_partitioning_and_data_colocation.html.md.erb
index 0876613..62e5cab 100644
--- a/geode-docs/developing/partitioned_regions/custom_partitioning_and_data_colocation.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/custom_partitioning_and_data_colocation.html.md.erb
@@ -23,7 +23,7 @@ Custom partitioning and data colocation can be used separately or in conjunction
 
 ## <a id="custom_partitioning_and_data_colocation__section_ABFEE9CB17AF44F1AE252AC10FB5E999" class="no-quick-link"></a>Custom Partitioning
 
-Use custom partitioning to group like entries into region buckets within a region. By default, Geode assigns new entries to buckets based on the entry key contents. With custom partitioning, you can assign your entries to buckets in whatever way you want.
+Use custom partitioning to group like entries into region buckets within a region. By default, <%=vars.product_name%> assigns new entries to buckets based on the entry key contents. With custom partitioning, you can assign your entries to buckets in whatever way you want.
 
 You can generally get better performance if you use custom partitioning to group similar data within a region. For example, a query run on all accounts created in January runs faster if all January account data is hosted by a single member. Grouping all data for a single customer can improve performance of data operations that work on customer data. Data aware function execution takes advantage of custom partitioning.
 
@@ -40,19 +40,19 @@ All keys must be strings, specified with a syntax that includes
 a '|' character that delimits the string.
 The substring that precedes the '|' delimiter within the key
 partitions the entry.  
--   **Standard custom partitioning**. With standard partitioning, you group entries into buckets, but you do not specify where the buckets reside. Geode always keeps the entries in the buckets you have specified, but may move the buckets around for load balancing.
+-   **Standard custom partitioning**. With standard partitioning, you group entries into buckets, but you do not specify where the buckets reside. <%=vars.product_name%> always keeps the entries in the buckets you have specified, but may move the buckets around for load balancing.
 -   **Fixed custom partitioning**. With fixed partitioning, you provide standard partitioning plus you specify the exact member where each data entry resides. You do this by assigning the data entry to a bucket and to a partition and by naming specific members as primary and secondary hosts of each partition.
 
     This gives you complete control over the locations of your primary and any secondary buckets for the region. This can be useful when you want to store specific data on specific physical machines or when you need to keep data close to certain hardware elements.
 
     Fixed partitioning has these requirements and caveats:
 
-    -   Geode cannot rebalance fixed partition region data because it cannot move the buckets around among the host members. You must carefully consider your expected data loads for the partitions you create.
+    -   <%=vars.product_name%> cannot rebalance fixed partition region data because it cannot move the buckets around among the host members. You must carefully consider your expected data loads for the partitions you create.
     -   With fixed partitioning, the region configuration is different between host members. Each member identifies the named partitions it hosts, and whether it is hosting the primary copy or a secondary copy. You then program fixed partition resolver to return the partition id, so the entry is placed on the right members. Only one member can be primary for a particular partition name and that member cannot be the partition's secondary.
 
 ## <a id="custom_partitioning_and_data_colocation__section_D2C66951FE38426F9C05050D2B9028D8" class="no-quick-link"></a>Data Colocation Between Regions
 
-With data colocation, Geode stores entries that are related across multiple data regions in a single member. Geode does this by storing all of the regions' buckets with the same ID together in the same member. During rebalancing operations, Geode moves these bucket groups together or not at all.
+With data colocation, <%=vars.product_name%> stores entries that are related across multiple data regions in a single member. <%=vars.product_name%> does this by storing all of the regions' buckets with the same ID together in the same member. During rebalancing operations, <%=vars.product_name%> moves these bucket groups together or not at all.
 
 So, for example, if you have one region with customer contact information and another region with customer orders, you can use colocation to keep all contact information and all orders for a single customer in a single member. This way, any operation done for a single customer uses the cache of only a single member.
 
@@ -60,6 +60,6 @@ This figure shows two regions with data colocation where the data is partitioned
 
 <img src="../../images_svg/colocated_partitioned_regions.svg" id="custom_partitioning_and_data_colocation__image_525AC474950F473ABCDE8E372583C5DF" class="image" />
 
-Data colocation requires the same data partitioning mechanism for all of the colocated regions. You can use the default partitioning provided by Geode or any of the custom partitioning strategies.
+Data colocation requires the same data partitioning mechanism for all of the colocated regions. You can use the default partitioning provided by <%=vars.product_name%> or any of the custom partitioning strategies.
 
 You must use the same high availability settings across your colocated regions.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/how_partitioning_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/how_partitioning_works.html.md.erb b/geode-docs/developing/partitioned_regions/how_partitioning_works.html.md.erb
index c846995..42ea7f8 100644
--- a/geode-docs/developing/partitioned_regions/how_partitioning_works.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/how_partitioning_works.html.md.erb
@@ -32,13 +32,13 @@ A distributed system can have multiple partitioned regions, and it can mix parti
 
 ## <a id="how_partitioning_works__section_260C2455FC8C40A094B39BF585D06B7D" class="no-quick-link"></a>Data Partitioning
 
-Geode automatically determines the physical location of data in the members that host a partitioned region's data. Geode breaks partitioned region data into units of storage known as buckets and stores each bucket in a region host member. Buckets are distributed in accordance to the member’s region attribute settings.
+<%=vars.product_name%> automatically determines the physical location of data in the members that host a partitioned region's data. <%=vars.product_name%> breaks partitioned region data into units of storage known as buckets and stores each bucket in a region host member. Buckets are distributed in accordance to the member’s region attribute settings.
 
 When an entry is created, it is assigned to a bucket. Keys are grouped together in a bucket and always remain there. If the configuration allows, the buckets may be moved between members to balance the load.
 
 You must run the data stores needed to accommodate storage for the partitioned region’s buckets. You can start new data stores on the fly. When a new data store creates the region, it takes responsibility for as many buckets as allowed by the partitioned region and member configuration.
 
-You can customize how Geode groups your partitioned region data with custom partitioning and data colocation.
+You can customize how <%=vars.product_name%> groups your partitioned region data with custom partitioning and data colocation.
 
 ## <a id="how_partitioning_works__section_155F9D4AB539473F848FD05E413B21B3" class="no-quick-link"></a>Partitioned Region Operation
 
@@ -52,7 +52,7 @@ Keep the following in mind about partitioned regions:
 
 -   Partitioned regions never run asynchronously. Operations in partitioned regions always wait for acknowledgement from the caches containing the original data entry and any redundant copies.
 -   A partitioned region needs a cache loader in every region data store (`local-max-memory` &gt; 0).
--   Geode distributes the data buckets as evenly as possible across all members storing the partitioned region data, within the limits of any custom partitioning or data colocation that you use. The number of buckets allotted for the partitioned region determines the granularity of data storage and thus how evenly the data can be distributed. The number of buckets is a total for the entire region across the distributed system.
--   In rebalancing data for the region, Geode moves buckets, but does not move data around inside the buckets.
+-   <%=vars.product_name%> distributes the data buckets as evenly as possible across all members storing the partitioned region data, within the limits of any custom partitioning or data colocation that you use. The number of buckets allotted for the partitioned region determines the granularity of data storage and thus how evenly the data can be distributed. The number of buckets is a total for the entire region across the distributed system.
+-   In rebalancing data for the region, <%=vars.product_name%> moves buckets, but does not move data around inside the buckets.
 -   You can query partitioned regions, but there are certain limitations. See [Querying Partitioned Regions](../querying_basics/querying_partitioned_regions.html#querying_partitioned_regions) for more information.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/how_pr_ha_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/how_pr_ha_works.html.md.erb b/geode-docs/developing/partitioned_regions/how_pr_ha_works.html.md.erb
index ba83732..baa5e56 100644
--- a/geode-docs/developing/partitioned_regions/how_pr_ha_works.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/how_pr_ha_works.html.md.erb
@@ -25,7 +25,7 @@ With high availability, each member that hosts data for the partitioned region g
 
 With redundancy, if one member fails, operations continue on the partitioned region with no interruption of service:
 
--   If the member hosting the primary copy is lost, Geode makes a secondary copy the primary. This might cause a temporary loss of redundancy, but not a loss of data.
+-   If the member hosting the primary copy is lost, <%=vars.product_name%> makes a secondary copy the primary. This might cause a temporary loss of redundancy, but not a loss of data.
 -   Whenever there are not enough secondary copies to satisfy redundancy, the system works to recover redundancy by assigning another member as secondary and copying the data to it.
 
 **Note:**
@@ -37,20 +37,20 @@ Without redundancy, the loss of any of the region's data stores causes the loss
 
 ## <a id="how_pr_ha_works__section_7045530D601F4C65A062B5FDD0DD9206" class="no-quick-link"></a>Controlling Where Your Primaries and Secondaries Reside
 
-By default, Geode places your primary and secondary data copies for you, avoiding placement of two copies on the same physical machine. If there are not enough machines to keep different copies separate, Geode places copies on the same physical machine. You can change this behavior, so Geode only places copies on separate machines.
+By default, <%=vars.product_name%> places your primary and secondary data copies for you, avoiding placement of two copies on the same physical machine. If there are not enough machines to keep different copies separate, <%=vars.product_name%> places copies on the same physical machine. You can change this behavior, so <%=vars.product_name%> only places copies on separate machines.
 
-You can also control which members store your primary and secondary data copies. Geode provides two options:
+You can also control which members store your primary and secondary data copies. <%=vars.product_name%> provides two options:
 
--   **Fixed custom partitioning**. This option is set for the region. Fixed partitioning gives you absolute control over where your region data is hosted. With fixed partitioning, you provide Geode with the code that specifies the bucket and data store for each data entry in the region. When you use this option with redundancy, you specify the primary and secondary data stores. Fixed partitioning does not participate in rebalancing because all bucket locations are fixed by you.
--   **Redundancy zones**. This option is set at the member level. Redundancy zones let you separate primary and secondary copies by member groups, or zones. You assign each data host to a zone. Then Geode places redundant copies in different redundancy zones, the same as it places redundant copies on different physical machines. You can use this to split data copies across different machine racks or networks, This option allows you to add members on the fly and use rebalancing to redistribute the data load, with redundant data maintained in separate zones. When you use redundancy zones, Geode will not place two copies of the data in the same zone, so make sure you have enough zones.
+-   **Fixed custom partitioning**. This option is set for the region. Fixed partitioning gives you absolute control over where your region data is hosted. With fixed partitioning, you provide <%=vars.product_name%> with the code that specifies the bucket and data store for each data entry in the region. When you use this option with redundancy, you specify the primary and secondary data stores. Fixed partitioning does not participate in rebalancing because all bucket locations are fixed by you.
+-   **Redundancy zones**. This option is set at the member level. Redundancy zones let you separate primary and secondary copies by member groups, or zones. You assign each data host to a zone. Then <%=vars.product_name%> places redundant copies in different redundancy zones, the same as it places redundant copies on different physical machines. You can use this to split data copies across different machine racks or networks, This option allows you to add members on the fly and use rebalancing to redistribute the data load, with redundant data maintained in separate zones. When you use redundancy zones, <%=vars.product_name%> will not place two copies of the data in the same zone, so make sure you have enough zones.
 
 ## <a id="how_pr_ha_works__section_87A2429B6277497184926E08E64B81C6" class="no-quick-link"></a>Running Processes in Virtual Machines
 
-By default, Geode stores redundant copies on different machines. When you run your processes in virtual machines, the normal view of the machine becomes the VM and not the physical machine. If you run multiple VMs on the same physical machine, you could end up storing partitioned region primary buckets in separate VMs, but on the same physical machine as your secondaries. If the physical machine fails, you can lose data. When you run in VMs, you can configure Geode to identify the physical machine and store redundant copies on different physical machines.
+By default, <%=vars.product_name%> stores redundant copies on different machines. When you run your processes in virtual machines, the normal view of the machine becomes the VM and not the physical machine. If you run multiple VMs on the same physical machine, you could end up storing partitioned region primary buckets in separate VMs, but on the same physical machine as your secondaries. If the physical machine fails, you can lose data. When you run in VMs, you can configure <%=vars.product_name%> to identify the physical machine and store redundant copies on different physical machines.
 
 ## <a id="how_pr_ha_works__section_CAB9440BABD6484D99525766E937CB55" class="no-quick-link"></a>Reads and Writes in Highly-Available Partitioned Regions
 
-Geode treats reads and writes differently in highly-available partitioned regions than in other regions because the data is available in multiple members:
+<%=vars.product_name%> treats reads and writes differently in highly-available partitioned regions than in other regions because the data is available in multiple members:
 
 -   Write operations (like `put` and `create`) go to the primary for the data keys and then are distributed synchronously to the redundant copies. Events are sent to the members configured with `subscription-attributes` `interest-policy` set to `all`.
 -   Read operations go to any member holding a copy of the data, with the local cache favored, so a read intensive system can scale much better and handle higher loads.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/join_query_partitioned_regions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/join_query_partitioned_regions.html.md.erb b/geode-docs/developing/partitioned_regions/join_query_partitioned_regions.html.md.erb
index 358b1a1..c48e328 100644
--- a/geode-docs/developing/partitioned_regions/join_query_partitioned_regions.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/join_query_partitioned_regions.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-In order to perform equi-join operations on partitioned regions or partitioned regions and replicated regions, you need to use the `query.execute` method and supply it with a function execution context. You need to use Geode's FunctionService executor because join operations are not yet directly supported for partitioned regions without providing a function execution context.
+In order to perform equi-join operations on partitioned regions or partitioned regions and replicated regions, you need to use the `query.execute` method and supply it with a function execution context. You need to use <%=vars.product_name%>'s FunctionService executor because join operations are not yet directly supported for partitioned regions without providing a function execution context.
 
 See [Partitioned Region Query Restrictions](../query_additional/partitioned_region_query_restrictions.html#concept_5353476380D44CC1A7F586E5AE1CE7E8) for more information on partitioned region query limitations.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/overview_custom_partitioning_and_data_colocation.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/overview_custom_partitioning_and_data_colocation.html.md.erb b/geode-docs/developing/partitioned_regions/overview_custom_partitioning_and_data_colocation.html.md.erb
index 1221873..b2ebc08 100644
--- a/geode-docs/developing/partitioned_regions/overview_custom_partitioning_and_data_colocation.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/overview_custom_partitioning_and_data_colocation.html.md.erb
@@ -19,18 +19,18 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-You can customize how Apache Geode groups your partitioned region data with custom partitioning and data colocation.
+You can customize how <%=vars.product_name_long%> groups your partitioned region data with custom partitioning and data colocation.
 
--   **[Understanding Custom Partitioning and Data Colocation](../../developing/partitioned_regions/custom_partitioning_and_data_colocation.html)**
+-   **[Understanding Custom Partitioning and Data Colocation](custom_partitioning_and_data_colocation.html)**
 
     Custom partitioning and data colocation can be used separately or in conjunction with one another.
 
--   **[Custom-Partition Your Region Data](../../developing/partitioned_regions/using_custom_partition_resolvers.html)**
+-   **[Custom-Partition Your Region Data](using_custom_partition_resolvers.html)**
 
-    By default, Geode partitions each data entry into a bucket using a hashing policy on the key. Additionally, the physical location of the key-value pair is abstracted away from the application. You can change these policies for a partitioned region. You can provide your own data partitioning resolver and you can additionally specify which members host which data buckets.
+    By default, <%=vars.product_name%> partitions each data entry into a bucket using a hashing policy on the key. Additionally, the physical location of the key-value pair is abstracted away from the application. You can change these policies for a partitioned region. You can provide your own data partitioning resolver and you can additionally specify which members host which data buckets.
 
--   **[Colocate Data from Different Partitioned Regions](../../developing/partitioned_regions/colocating_partitioned_region_data.html)**
+-   **[Colocate Data from Different Partitioned Regions](colocating_partitioned_region_data.html)**
 
-    By default, Geode allocates the data locations for a partitioned region independent of the data locations for any other partitioned region. You can change this policy for any group of partitioned regions, so that cross-region, related data is all hosted by the same member. This colocation speeds queries and other operations that access data from the regions.
+    By default, <%=vars.product_name%> allocates the data locations for a partitioned region independent of the data locations for any other partitioned region. You can change this policy for any group of partitioned regions, so that cross-region, related data is all hosted by the same member. This colocation speeds queries and other operations that access data from the regions.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/overview_how_pr_ha_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/overview_how_pr_ha_works.html.md.erb b/geode-docs/developing/partitioned_regions/overview_how_pr_ha_works.html.md.erb
index 889c56c..e12ddc5 100644
--- a/geode-docs/developing/partitioned_regions/overview_how_pr_ha_works.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/overview_how_pr_ha_works.html.md.erb
@@ -19,13 +19,13 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-By default, Apache Geode stores only a single copy of your partitioned region data among the region's data stores. You can configure Geode to maintain redundant copies of your partitioned region data for high availability.
+By default, <%=vars.product_name_long%> stores only a single copy of your partitioned region data among the region's data stores. You can configure <%=vars.product_name%> to maintain redundant copies of your partitioned region data for high availability.
 
--   **[Understanding High Availability for Partitioned Regions](../../developing/partitioned_regions/how_pr_ha_works.html)**
+-   **[Understanding High Availability for Partitioned Regions](how_pr_ha_works.html)**
 
     With high availability, each member that hosts data for the partitioned region gets some primary copies and some redundant (secondary) copies.
 
--   **[Configure High Availability for a Partitioned Region](../../developing/partitioned_regions/configuring_ha_for_pr.html)**
+-   **[Configure High Availability for a Partitioned Region](configuring_ha_for_pr.html)**
 
     Configure in-memory high availability for your partitioned region. Set other high-availability options, like redundancy zones and redundancy recovery strategies.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/overview_how_pr_single_hop_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/overview_how_pr_single_hop_works.html.md.erb b/geode-docs/developing/partitioned_regions/overview_how_pr_single_hop_works.html.md.erb
index 8be43f6..13d7498 100644
--- a/geode-docs/developing/partitioned_regions/overview_how_pr_single_hop_works.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/overview_how_pr_single_hop_works.html.md.erb
@@ -21,11 +21,11 @@ limitations under the License.
 
 Single-hop data access enables the client pool to track where a partitioned region’s data is hosted in the servers. To access a single entry, the client directly contacts the server that hosts the key, in a single hop.
 
--   **[Understanding Client Single-Hop Access to Server-Partitioned Regions](../../developing/partitioned_regions/how_pr_single_hop_works.html)**
+-   **[Understanding Client Single-Hop Access to Server-Partitioned Regions](how_pr_single_hop_works.html)**
 
     With single-hop access the client connects to every server, so more connections are generally used. This works fine for smaller installations, but is a barrier to scaling.
 
--   **[Configure Client Single-Hop Access to Server-Partitioned Regions](../../developing/partitioned_regions/configure_pr_single_hop.html)**
+-   **[Configure Client Single-Hop Access to Server-Partitioned Regions](configure_pr_single_hop.html)**
 
     Configure your client/server system for direct, single-hop access to partitioned region data in the servers.
 


[08/11] geode git commit: GEODE-3395 Variable-ize product version and name in user guide - Developing

Posted by zh...@apache.org.
GEODE-3395 Variable-ize product version and name in user guide - Developing


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

Branch: refs/heads/feature/GEM-1601
Commit: ed9a8fd47a56fa84b810f0e4c4261b299150d1de
Parents: 3bb6a22
Author: Dave Barnes <db...@pivotal.io>
Authored: Wed Aug 16 16:20:25 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Fri Aug 18 10:42:31 2017 -0700

----------------------------------------------------------------------
 geode-docs/developing/book_intro.html.md.erb    |  10 +-
 .../chapter_overview.html.md.erb                |   6 +-
 .../PDX_Serialization_Features.html.md.erb      |  10 +-
 .../auto_serialization.html.md.erb              |   2 +-
 ...ation_with_class_pattern_strings.html.md.erb |   2 +-
 .../chapter_overview.html.md.erb                |  16 +--
 .../data_serialization_options.html.md.erb      |  24 ++---
 .../extending_the_autoserializer.html.md.erb    |   2 +-
 .../gemfire_data_serialization.html.md.erb      |   8 +-
 .../gemfire_pdx_serialization.html.md.erb       |  32 +++---
 .../jsonformatter_pdxinstances.html.md.erb      |  16 ++-
 .../persist_pdx_metadata_to_disk.html.md.erb    |  10 +-
 .../program_application_for_pdx.html.md.erb     |   2 +-
 .../use_pdx_high_level_steps.html.md.erb        |   4 +-
 .../use_pdx_serializable.html.md.erb            |   8 +-
 .../use_pdx_serializer.html.md.erb              |   4 +-
 .../delta_propagation_example.html.md.erb       |   6 +-
 .../delta_propagation_properties.html.md.erb    |   4 +-
 .../how_delta_propagation_works.html.md.erb     |  10 +-
 .../implementing_delta_propagation.html.md.erb  |   4 +-
 .../chapter_overview.html.md.erb                |  14 +--
 .../choosing_level_of_dist.html.md.erb          |   2 +-
 .../how_region_versioning_works.html.md.erb     |  42 ++++----
 .../how_region_versioning_works_wan.html.md.erb |  14 +--
 .../locking_in_global_regions.html.md.erb       |   2 +-
 .../managing_distributed_regions.html.md.erb    |   2 +-
 .../region_entry_versions.html.md.erb           |  22 ++--
 .../events/chapter_overview.html.md.erb         |  16 +--
 ...re_client_server_event_messaging.html.md.erb |   6 +-
 ...figure_multisite_event_messaging.html.md.erb |  10 +-
 ...uring_gateway_concurrency_levels.html.md.erb |  12 +--
 ..._highly_available_gateway_queues.html.md.erb |   4 +-
 ...iguring_highly_available_servers.html.md.erb |   2 +-
 .../events/event_handler_overview.html.md.erb   |   4 +-
 .../filtering_multisite_events.html.md.erb      |  12 +--
 .../events/how_cache_events_work.html.md.erb    |   4 +-
 ...client_server_distribution_works.html.md.erb |   2 +-
 .../events/how_events_work.html.md.erb          |  20 ++--
 ...how_multisite_distribution_works.html.md.erb |   2 +-
 ...mplementing_cache_event_handlers.html.md.erb |   2 +-
 ..._durable_client_server_messaging.html.md.erb |   4 +-
 ...nting_write_behind_event_handler.html.md.erb |  20 ++--
 ...ist_of_event_handlers_and_events.html.md.erb |   6 +-
 ...ne_client_server_event_messaging.html.md.erb |   6 +-
 ..._callbacks_that_modify_the_cache.html.md.erb |  10 +-
 .../eviction/chapter_overview.html.md.erb       |   6 +-
 .../configuring_data_eviction.html.md.erb       |   6 +-
 .../eviction/how_eviction_works.html.md.erb     |   6 +-
 .../expiration/chapter_overview.html.md.erb     |   4 +-
 .../expiration/how_expiration_works.html.md.erb |   4 +-
 .../function_exec/chapter_overview.html.md.erb  |   2 +-
 .../function_execution.html.md.erb              |  18 ++--
 .../how_function_execution_works.html.md.erb    |   6 +-
 .../chapter_overview.html.md.erb                |   2 +-
 .../how_data_loaders_work.html.md.erb           |   2 +-
 .../sync_outside_data.html.md.erb               |   8 +-
 .../chapter_overview.html.md.erb                |  24 ++---
 ...locating_partitioned_region_data.html.md.erb |   4 +-
 .../configuring_bucket_for_pr.html.md.erb       |   2 +-
 .../configuring_ha_for_pr.html.md.erb           |  20 ++--
 ...partitioning_and_data_colocation.html.md.erb |  10 +-
 .../how_partitioning_works.html.md.erb          |   8 +-
 .../how_pr_ha_works.html.md.erb                 |  14 +--
 .../join_query_partitioned_regions.html.md.erb  |   2 +-
 ...partitioning_and_data_colocation.html.md.erb |  12 +--
 .../overview_how_pr_ha_works.html.md.erb        |   6 +-
 ...overview_how_pr_single_hop_works.html.md.erb |   4 +-
 .../rebalancing_pr_data.html.md.erb             |   8 +-
 .../set_enforce_unique_host.html.md.erb         |   4 +-
 .../set_redundancy_zones.html.md.erb            |   2 +-
 ...using_custom_partition_resolvers.html.md.erb |  10 +-
 .../advanced_querying.html.md.erb               |  18 ++--
 .../query_additional/literals.html.md.erb       |   4 +-
 .../query_additional/operators.html.md.erb      |   4 +-
 .../query_debugging.html.md.erb                 |   2 +-
 .../query_language_features.html.md.erb         |  16 +--
 .../using_query_bind_parameters.html.md.erb     |   2 +-
 .../create_multiple_indexes.html.md.erb         |   2 +-
 .../query_index/creating_an_index.html.md.erb   |   2 +-
 .../creating_hash_indexes.html.md.erb           |   2 +-
 .../query_index/indexing_guidelines.html.md.erb |   2 +-
 .../query_index/maintaining_indexes.html.md.erb |   2 +-
 .../query_index/query_index.html.md.erb         |  34 +++----
 .../query_index/query_index_hints.html.md.erb   |   2 +-
 .../the_select_statement.html.md.erb            |   4 +-
 .../query_select/the_where_clause.html.md.erb   |   4 +-
 .../chapter_overview.html.md.erb                |   4 +-
 .../querying_basics/query_basics.html.md.erb    |  22 ++--
 .../querying_partitioned_regions.html.md.erb    |  16 +--
 .../querying_basics/reserved_words.html.md.erb  |   2 +-
 ...ictions_and_unsupported_features.html.md.erb |   2 +-
 .../querying_basics/running_a_query.html.md.erb |   6 +-
 .../supported_character_sets.html.md.erb        |   2 +-
 .../what_is_a_query_string.html.md.erb          |  10 +-
 .../region_options/chapter_overview.html.md.erb |  12 +--
 .../dynamic_region_creation.html.md.erb         |   2 +-
 .../region_options/region_types.html.md.erb     |   6 +-
 .../storage_distribution_options.html.md.erb    |   4 +-
 .../chapter_overview.html.md.erb                |  10 +-
 .../how_persist_overflow_work.html.md.erb       |   6 +-
 .../transactions/JTA_transactions.html.md.erb   | 100 +++++++++----------
 .../transactions/about_transactions.html.md.erb |  18 ++--
 .../cache_plugins_with_jta.html.md.erb          |   8 +-
 .../cache_transaction_performance.html.md.erb   |   2 +-
 .../transactions/cache_transactions.html.md.erb |  26 +++--
 ...ache_transactions_by_region_type.html.md.erb |  16 +--
 .../transactions/chapter_overview.html.md.erb   |  24 ++---
 .../client_server_transactions.html.md.erb      |  16 +--
 ...guring_db_connections_using_JNDI.html.md.erb |   4 +-
 .../how_cache_transactions_work.html.md.erb     |  20 ++--
 .../jca_adapter_example.html.md.erb             |   2 +-
 ...onitor_troubleshoot_transactions.html.md.erb |   8 +-
 .../run_a_cache_transaction.html.md.erb         |  16 ++-
 ...che_transaction_with_external_db.html.md.erb |  16 ++-
 .../transaction_coding_examples.html.md.erb     |  14 +--
 .../transaction_event_management.html.md.erb    |   4 +-
 .../transaction_jta_gemfire_example.html.md.erb |   6 +-
 .../transaction_semantics.html.md.erb           |  14 ++-
 ...ctional_and_nontransactional_ops.html.md.erb |   2 +-
 .../working_with_transactions.html.md.erb       |  28 +++---
 120 files changed, 564 insertions(+), 598 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/book_intro.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/book_intro.html.md.erb b/geode-docs/developing/book_intro.html.md.erb
index 8086b7a..c78f753 100644
--- a/geode-docs/developing/book_intro.html.md.erb
+++ b/geode-docs/developing/book_intro.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Developing with Apache Geode
----
+<% set_title("Developing with", product_name_long) %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,13 +17,13 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-*Developing with Apache Geode* explains main concepts of application programming with Apache Geode. It describes how to plan and implement regions, data serialization, event handling, delta propagation, transactions, and more.
+*Developing with <%=vars.product_name_long%>* explains main concepts of application programming with <%=vars.product_name_long%>. It describes how to plan and implement regions, data serialization, event handling, delta propagation, transactions, and more.
 
-For information about Geode REST application development, see [Developing REST Applications for Apache Geode](../rest_apps/book_intro.html).
+For information about Geode REST application development, see [Developing REST Applications for <%=vars.product_name_long%>](../rest_apps/book_intro.html).
 
 -   **[Region Data Storage and Distribution](../developing/region_options/chapter_overview.html)**
 
-    The Apache Geode data storage and distribution models put your data in the right place at the right time. You should understand all the options for data storage in Geode before you start configuring your data regions.
+    The <%=vars.product_name_long%> data storage and distribution models put your data in the right place at the right time. You should understand all the options for data storage in Geode before you start configuring your data regions.
 
 -   **[Partitioned Regions](../developing/partitioned_regions/chapter_overview.html)**
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/continuous_querying/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/continuous_querying/chapter_overview.html.md.erb b/geode-docs/developing/continuous_querying/chapter_overview.html.md.erb
index 3f77edb..865050a 100644
--- a/geode-docs/developing/continuous_querying/chapter_overview.html.md.erb
+++ b/geode-docs/developing/continuous_querying/chapter_overview.html.md.erb
@@ -23,15 +23,15 @@ Continuous querying continuously returns events that match the queries you set u
 
 <a id="continuous__section_779B4E4D06E948618E5792335174E70D"></a>
 
--   **[How Continuous Querying Works](../../developing/continuous_querying/how_continuous_querying_works.html)**
+-   **[How Continuous Querying Works](how_continuous_querying_works.html)**
 
     Clients subscribe to server-side events by using SQL-type query filtering. The server sends all events that modify the query results. CQ event delivery uses the client/server subscription framework.
 
--   **[Implementing Continuous Querying](../../developing/continuous_querying/implementing_continuous_querying.html)**
+-   **[Implementing Continuous Querying](implementing_continuous_querying.html)**
 
     Use continuous querying in your clients to receive continuous updates to queries run on the servers.
 
--   **[Managing Continuous Querying](../../developing/continuous_querying/continuous_querying_whats_next.html)**
+-   **[Managing Continuous Querying](continuous_querying_whats_next.html)**
 
     This topic discusses CQ management options, CQ states, and retrieving initial result sets.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/PDX_Serialization_Features.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/PDX_Serialization_Features.html.md.erb b/geode-docs/developing/data_serialization/PDX_Serialization_Features.html.md.erb
index e6c06f4..6f30d02 100644
--- a/geode-docs/developing/data_serialization/PDX_Serialization_Features.html.md.erb
+++ b/geode-docs/developing/data_serialization/PDX_Serialization_Features.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Geode PDX Serialization Features
----
+<% set_title(product_name, "PDX Serialization Features") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,17 +17,17 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode PDX serialization offers several advantages in terms of functionality.
+<%=vars.product_name%> PDX serialization offers several advantages in terms of functionality.
 
 ## <a id="concept_F02E40517C4B42F2A75B133BB507C626__section_A0EEB4DA3E9F4EA4B65FE727D3951EA1" class="no-quick-link"></a>Application Versioning of PDX Domain Objects
 
 Domain objects evolve along with your application code. You might create an address object with two address lines, then realize later that a third line is required for some situations. Or you might realize that a particular field is not used and want to get rid of it. With PDX, you can use old and new versions of domain objects together in a distributed system if the versions differ by the addition or removal of fields. This compatibility lets you gradually introduce modified code and data into the system, without bringing the system down.
 
-Geode maintains a central registry of the PDX domain object metadata. Using the registry, Geode preserves fields in each member's cache regardless of whether the field is defined. When a member receives an object with a registered field that the member is not aware of, the member does not access the field, but preserves it and passes it along with the entire object to other members. When a member receives an object that is missing one or more fields according to the member's version, Geode assigns the Java default values for the field types to the missing fields.
+<%=vars.product_name%> maintains a central registry of the PDX domain object metadata. Using the registry, <%=vars.product_name%> preserves fields in each member's cache regardless of whether the field is defined. When a member receives an object with a registered field that the member is not aware of, the member does not access the field, but preserves it and passes it along with the entire object to other members. When a member receives an object that is missing one or more fields according to the member's version, <%=vars.product_name%> assigns the Java default values for the field types to the missing fields.
 
 ## <a id="concept_F02E40517C4B42F2A75B133BB507C626__section_D68A6A9C2C0C4D32AE7DADA2A4C3104D" class="no-quick-link"></a>Portability of PDX Serializable Objects
 
-When you serialize an object using PDX, Geode stores the object's type information in the central registry. The information is passed among clients and servers, peers, and distributed systems.
+When you serialize an object using PDX, <%=vars.product_name%> stores the object's type information in the central registry. The information is passed among clients and servers, peers, and distributed systems.
 
 This centralization of object type information is advantageous for client/server installations in which clients and servers are written in different languages. Clients pass registry information to servers automatically when they store a PDX serialized object. Clients can run queries and functions against the data in the servers without compatibility between server and the stored objects. One client can store data on the server to be retrieved by another client, with no requirements on the part of the server.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/auto_serialization.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/auto_serialization.html.md.erb b/geode-docs/developing/data_serialization/auto_serialization.html.md.erb
index cb347a9..0ed38fc 100644
--- a/geode-docs/developing/data_serialization/auto_serialization.html.md.erb
+++ b/geode-docs/developing/data_serialization/auto_serialization.html.md.erb
@@ -33,7 +33,7 @@ Your custom PDX autoserializable classes cannot use the `org.apache.geode` packa
 
 **Prerequisites**
 
--   Understand generally how to configure the Geode cache.
+-   Understand generally how to configure the <%=vars.product_name%> cache.
 -   Understand how PDX serialization works and how to configure your application to use `PdxSerializer`.
 
 <a id="auto_serialization__section_43F6E45FF69E470897FD9D002FBE896D"><strong>Procedure</strong></a>

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/autoserialization_with_class_pattern_strings.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/autoserialization_with_class_pattern_strings.html.md.erb b/geode-docs/developing/data_serialization/autoserialization_with_class_pattern_strings.html.md.erb
index b545610..41879e2 100644
--- a/geode-docs/developing/data_serialization/autoserialization_with_class_pattern_strings.html.md.erb
+++ b/geode-docs/developing/data_serialization/autoserialization_with_class_pattern_strings.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Use class pattern strings to name the classes that you want to serialize using Geode's reflection-based autoserializer and to specify object identity fields and to specify fields to exclude from serialization.
+Use class pattern strings to name the classes that you want to serialize using <%=vars.product_name%>'s reflection-based autoserializer and to specify object identity fields and to specify fields to exclude from serialization.
 
 The class pattern strings used to configured the `ReflectionBasedAutoSerializer` are standard regular expressions. For example, this expression would select all classes defined in the `com.company.domain` package and its subpackages:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/chapter_overview.html.md.erb b/geode-docs/developing/data_serialization/chapter_overview.html.md.erb
index 7e13c20..ebc55f0 100644
--- a/geode-docs/developing/data_serialization/chapter_overview.html.md.erb
+++ b/geode-docs/developing/data_serialization/chapter_overview.html.md.erb
@@ -19,21 +19,21 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Data that you manage in Geode must be serialized and deserialized for storage and transmittal between processes. You can choose among several options for data serialization.
+Data that you manage in <%=vars.product_name%> must be serialized and deserialized for storage and transmittal between processes. You can choose among several options for data serialization.
 
--   **[Overview of Data Serialization](../../developing/data_serialization/data_serialization_options.html)**
+-   **[Overview of Data Serialization](data_serialization_options.html)**
 
-    Geode offers serialization options other than Java serialization that give you higher performance and greater flexibility for data storage, transfers, and language types.
+    <%=vars.product_name%> offers serialization options other than Java serialization that give you higher performance and greater flexibility for data storage, transfers, and language types.
 
--   **[Geode PDX Serialization](../../developing/data_serialization/gemfire_pdx_serialization.html)**
+-   **[<%=vars.product_name%> PDX Serialization](gemfire_pdx_serialization.html)**
 
-    Geode's Portable Data eXchange (PDX) is a cross-language data format that can reduce the cost of distributing and serializing your objects. PDX stores data in named fields that you can access individually, to avoid the cost of deserializing the entire data object. PDX also allows you to mix versions of objects where you have added or removed fields.
+    <%=vars.product_name%>'s Portable Data eXchange (PDX) is a cross-language data format that can reduce the cost of distributing and serializing your objects. PDX stores data in named fields that you can access individually, to avoid the cost of deserializing the entire data object. PDX also allows you to mix versions of objects where you have added or removed fields.
 
--   **[Geode Data Serialization (DataSerializable and DataSerializer)](../../developing/data_serialization/gemfire_data_serialization.html)**
+-   **[<%=vars.product_name%> Data Serialization (DataSerializable and DataSerializer)](gemfire_data_serialization.html)**
 
-    Geode's `DataSerializable` interface gives you quick serialization of your objects.
+    <%=vars.product_name%>'s `DataSerializable` interface gives you quick serialization of your objects.
 
--   **[Standard Java Serialization](../../developing/data_serialization/java_serialization.html)**
+-   **[Standard Java Serialization](java_serialization.html)**
 
     You can use standard Java serialization for data you only distribute between Java applications. If you distribute your data between non-Java clients and Java servers, you need to do additional programming to get the data between the various class formats.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/data_serialization_options.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/data_serialization_options.html.md.erb b/geode-docs/developing/data_serialization/data_serialization_options.html.md.erb
index dfe18d0..0115cfc 100644
--- a/geode-docs/developing/data_serialization/data_serialization_options.html.md.erb
+++ b/geode-docs/developing/data_serialization/data_serialization_options.html.md.erb
@@ -19,10 +19,10 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode offers serialization options other than Java serialization that give you higher performance and greater flexibility for data storage, transfers, and language types.
+<%=vars.product_name%> offers serialization options other than Java serialization that give you higher performance and greater flexibility for data storage, transfers, and language types.
 
 <a id="data_serialization_options__section_B1BDB0E7F6814DFD8BACD8D8C5CAA81B"></a>
-All data that Geode moves out of the local cache must be serializable. However, you do not necessarily need to implement `java.io.Serializable` since other serialization options are available in Geode. Region data that must be serializable falls under the following categories:
+All data that <%=vars.product_name%> moves out of the local cache must be serializable. However, you do not necessarily need to implement `java.io.Serializable` since other serialization options are available in <%=vars.product_name%>. Region data that must be serializable falls under the following categories:
 
 -   Partitioned regions
 -   Distributed regions
@@ -35,34 +35,34 @@ All data that Geode moves out of the local cache must be serializable. However,
 **Note:**
 If you are storing objects with the [HTTP Session Management Modules](../../tools_modules/http_session_mgmt/chapter_overview.html), these objects must be serializable since they are serialized before being stored in the region.
 
-To minimize the cost of serialization and deserialization, Geode avoids changing the data format whenever possible. This means your data might be stored in the cache in serialized or deserialized form, depending on how you use it. For example, if a server acts only as a storage location for data distribution between clients, it makes sense to leave the data in serialized form, ready to be transmitted to clients that request it. Partitioned region data is always initially stored in serialized form.
+To minimize the cost of serialization and deserialization, <%=vars.product_name%> avoids changing the data format whenever possible. This means your data might be stored in the cache in serialized or deserialized form, depending on how you use it. For example, if a server acts only as a storage location for data distribution between clients, it makes sense to leave the data in serialized form, ready to be transmitted to clients that request it. Partitioned region data is always initially stored in serialized form.
 
 ## <a id="data_serialization_options__section_691C2CF5A4E24D599070A7AADEDF2BEC" class="no-quick-link"></a>Data Serialization Options
 
 <a id="data_serialization_options__section_44CC2DEEDA0F41D49D416ABA921A6436"></a>
 
-With Geode, you have the option to serialize your domain objects automatically or to implement serialization using one of Geode's interfaces. Enabling automatic serialization means that domain objects are serialized and deserialized without your having to make any code changes to those objects. This automatic serialization is performed by registering your domain objects with a custom `PdxSerializer` called the `ReflectionBasedAutoSerializer`, which uses Java reflection to infer which fields to serialize.
+With <%=vars.product_name%>, you have the option to serialize your domain objects automatically or to implement serialization using one of <%=vars.product_name%>'s interfaces. Enabling automatic serialization means that domain objects are serialized and deserialized without your having to make any code changes to those objects. This automatic serialization is performed by registering your domain objects with a custom `PdxSerializer` called the `ReflectionBasedAutoSerializer`, which uses Java reflection to infer which fields to serialize.
 
-If autoserialization does not meet your needs, you can serialize your objects by implementing one of the Geode interfaces, `PdxSerializable` or `DataSerializable`. You can use these interfaces to replace any standard Java data serialization for better performance. If you cannot or do not want to modify your domain classes, each interface has an alternate serializer class, `PdxSerializer` and `DataSerializer`. To use these, you create your custom serializer class and then associate it with your domain class in the Geode cache configuration.
+If autoserialization does not meet your needs, you can serialize your objects by implementing one of the <%=vars.product_name%> interfaces, `PdxSerializable` or `DataSerializable`. You can use these interfaces to replace any standard Java data serialization for better performance. If you cannot or do not want to modify your domain classes, each interface has an alternate serializer class, `PdxSerializer` and `DataSerializer`. To use these, you create your custom serializer class and then associate it with your domain class in the <%=vars.product_name%> cache configuration.
 
-Geode Data serialization is about 25% faster than PDX serialization, however using PDX serialization will help you to avoid the even larger costs of performing deserialization.
+<%=vars.product_name%> Data serialization is about 25% faster than PDX serialization, however using PDX serialization will help you to avoid the even larger costs of performing deserialization.
 
 <a id="data_serialization_options__section_993B4A298874459BB4A8A0A9811854D9"></a><a id="data_serialization_options__table_ccf00c9f-9b98-47f7-ab30-3d23ecaff0a1"></a>
 
-| Capability                                                                                                                       | Geode Data Serializable | Geode PDX Serializable |
+| Capability                                                                                                                       | <%=vars.product_name%> Data Serializable | <%=vars.product_name%> PDX Serializable |
 |----------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------|-----------------------------------------------------|
 | Implements Java Serializable.                                                                                                    | X                                                    |                                                     |
 | Handles multiple versions of application domain objects, providing the versions differ by the addition or subtraction of fields. |                                                      | X                                                   |
 | Provides single field access of serialized data, without full deserialization - supported also for OQL querying.                 |                                                      | X                                                   |
-| Automatically ported to other languages by Geode                                                    |                                                      | X                                                   |
+| Automatically ported to other languages by <%=vars.product_name%>                                                    |                                                      | X                                                   |
 | Works with .NET clients.                                                                                                         | X                                                    | X                                                   |
 | Works with C++ clients.                                                                                                         | X                                                    | X                                                   |
-| Works with Geode delta propagation.                                                                 | X                                                    | X (See note below.)                                 |
+| Works with <%=vars.product_name%> delta propagation.                                                                 | X                                                    | X (See note below.)                                 |
 
 <span class="tablecap">**Table 1.** Serialization Options: Comparison of Features</span>
 
-**Note:** By default, you can use Geode delta propagation with PDX serialization. However, delta propagation will not work if you have set the Geode property `read-serialized` to "true". In terms of deserialization, to apply a change delta propagation requires a domain class instance and the `fromDelta `method. If you have set `read-serialized` to true, then you will receive a `PdxInstance` instead of a domain class instance and `PdxInstance` does not have the `fromDelta` method required for delta propagation.
+**Note:** By default, you can use <%=vars.product_name%> delta propagation with PDX serialization. However, delta propagation will not work if you have set the <%=vars.product_name%> property `read-serialized` to "true". In terms of deserialization, to apply a change delta propagation requires a domain class instance and the `fromDelta `method. If you have set `read-serialized` to true, then you will receive a `PdxInstance` instead of a domain class instance and `PdxInstance` does not have the `fromDelta` method required for delta propagation.
 
-## <a id="data_serialization_options__section_D90C2C09B95C40B6803CF202CF8008BF" class="no-quick-link"></a>Differences between Geode Serialization (PDX or Data Serializable) and Java Serialization
+## <a id="data_serialization_options__section_D90C2C09B95C40B6803CF202CF8008BF" class="no-quick-link"></a>Differences between <%=vars.product_name%> Serialization (PDX or Data Serializable) and Java Serialization
 
-Geode serialization (either PDX Serialization or Data Serialization) does not support circular object graphs whereas Java serialization does. In Geode serialization, if the same object is referenced more than once in an object graph, the object is serialized for each reference, and deserialization produces multiple copies of the object. By contrast in this situation, Java serialization serializes the object once and when deserializing the object, it produces one instance of the object with multiple references.
+<%=vars.product_name%> serialization (either PDX Serialization or Data Serialization) does not support circular object graphs whereas Java serialization does. In <%=vars.product_name%> serialization, if the same object is referenced more than once in an object graph, the object is serialized for each reference, and deserialization produces multiple copies of the object. By contrast in this situation, Java serialization serializes the object once and when deserializing the object, it produces one instance of the object with multiple references.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/extending_the_autoserializer.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/extending_the_autoserializer.html.md.erb b/geode-docs/developing/data_serialization/extending_the_autoserializer.html.md.erb
index cfa69f5..47ee92b 100644
--- a/geode-docs/developing/data_serialization/extending_the_autoserializer.html.md.erb
+++ b/geode-docs/developing/data_serialization/extending_the_autoserializer.html.md.erb
@@ -25,7 +25,7 @@ You can extend the `ReflectionBasedAutoSerializer` to handle serialization in a
 
 One of the main use cases for extending the `ReflectionBasedAutoSerializer` is that you want it to handle an object that would currently need to be handled by standard Java serialization. There are several issues with having to use standard Java serialization that can be addressed by extending the PDX `ReflectionBasedAutoSerializer`.
 
--   Each time we transition from a Geode serialized object to an object that will be Java I/O serialized, extra data must get serialized. This can cause a great deal of serialization overhead. This is why it is worth extending the `ReflectionBasedAutoSerializer` to handle any classes that normally would have to be Java I/O serialized.
+-   Each time we transition from a <%=vars.product_name%> serialized object to an object that will be Java I/O serialized, extra data must get serialized. This can cause a great deal of serialization overhead. This is why it is worth extending the `ReflectionBasedAutoSerializer` to handle any classes that normally would have to be Java I/O serialized.
 -   Expanding the number of classes that can use the `ReflectionBasedAutoSerializer` is beneficial when you encounter object graphs. After we use Java I/O serialization on an object, any objects under that object in the object graph will also have to be Java I/O serialized. This includes objects that normally would have been serialized using PDX or `DataSerializable`.
 -   If standard Java I/O serialization is done on an object and you have enabled check-portability, then an exception will be thrown. Even if you are not concerned with the object's portability, you can use this flag to find out what classes would use standard Java serialization (by getting an exception on them) and then enhancing your auto serializer to handle them.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/gemfire_data_serialization.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/gemfire_data_serialization.html.md.erb b/geode-docs/developing/data_serialization/gemfire_data_serialization.html.md.erb
index 24acbfd..96689ec 100644
--- a/geode-docs/developing/data_serialization/gemfire_data_serialization.html.md.erb
+++ b/geode-docs/developing/data_serialization/gemfire_data_serialization.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Geode Data Serialization (DataSerializable and DataSerializer)
----
+<% set_title(product_name, "Data Serialization (DataSerializable and DataSerializer)") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,11 +17,11 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode's `DataSerializable` interface gives you quick serialization of your objects.
+<%=vars.product_name%>'s `DataSerializable` interface gives you quick serialization of your objects.
 
 ## <a id="gemfire_data_serialization__section_0C84D6BF5E9748CB865E6BB944A077DE" class="no-quick-link"></a>Data Serialization with the DataSerializable Interface
 
-Geode's `DataSerializable` interface gives you faster and more compact data serialization than the standard Java serialization or Geode PDX serialization. However, while Geode `DataSerializable` interface is generally more performant than Geode's `PdxSerializable`, it requires full deserialization on the server and then reserialization to send the data back to the client.
+<%=vars.product_name%>'s `DataSerializable` interface gives you faster and more compact data serialization than the standard Java serialization or <%=vars.product_name%> PDX serialization. However, while <%=vars.product_name%> `DataSerializable` interface is generally more performant than <%=vars.product_name%>'s `PdxSerializable`, it requires full deserialization on the server and then reserialization to send the data back to the client.
 
 You can further speed serialization by registering the instantiator for your `DataSerializable` class through `Instantiator`, eliminating the need for reflection to find the right serializer. You can provide your own serialization through the API.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/gemfire_pdx_serialization.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/gemfire_pdx_serialization.html.md.erb b/geode-docs/developing/data_serialization/gemfire_pdx_serialization.html.md.erb
index c8bcdb4..9dd25ec 100644
--- a/geode-docs/developing/data_serialization/gemfire_pdx_serialization.html.md.erb
+++ b/geode-docs/developing/data_serialization/gemfire_pdx_serialization.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Geode PDX Serialization
----
+<% set_title(product_name, "PDX Serialization") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,45 +17,45 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode's Portable Data eXchange (PDX) is a cross-language data format that can reduce the cost of distributing and serializing your objects. PDX stores data in named fields that you can access individually, to avoid the cost of deserializing the entire data object. PDX also allows you to mix versions of objects where you have added or removed fields.
+<%=vars.product_name%>'s Portable Data eXchange (PDX) is a cross-language data format that can reduce the cost of distributing and serializing your objects. PDX stores data in named fields that you can access individually, to avoid the cost of deserializing the entire data object. PDX also allows you to mix versions of objects where you have added or removed fields.
 
--   **[Geode PDX Serialization Features](../../developing/data_serialization/PDX_Serialization_Features.html)**
+-   **[<%=vars.product_name%> PDX Serialization Features](PDX_Serialization_Features.html)**
 
-    Geode PDX serialization offers several advantages in terms of functionality.
+    <%=vars.product_name%> PDX serialization offers several advantages in terms of functionality.
 
--   **[High Level Steps for Using PDX Serialization](../../developing/data_serialization/use_pdx_high_level_steps.html)**
+-   **[High Level Steps for Using PDX Serialization](use_pdx_high_level_steps.html)**
 
-    To use PDX serialization, you can configure and use Geode's reflection-based autoserializer, or you can program the serialization of your objects by using the PDX interfaces and classes.
+    To use PDX serialization, you can configure and use <%=vars.product_name%>'s reflection-based autoserializer, or you can program the serialization of your objects by using the PDX interfaces and classes.
 
--   **[Using Automatic Reflection-Based PDX Serialization](../../developing/data_serialization/auto_serialization.html)**
+-   **[Using Automatic Reflection-Based PDX Serialization](auto_serialization.html)**
 
     You can configure your cache to automatically serialize and deserialize domain objects without having to add any extra code to them.
 
--   **[Serializing Your Domain Object with a PdxSerializer](../../developing/data_serialization/use_pdx_serializer.html)**
+-   **[Serializing Your Domain Object with a PdxSerializer](use_pdx_serializer.html)**
 
     For a domain object that you cannot or do not want to modify, use the `PdxSerializer` class to serialize and deserialize the object's fields. You use one `PdxSerializer` implementation for the entire cache, programming it for all of the domain objects that you handle in this way.
 
--   **[Implementing PdxSerializable in Your Domain Object](../../developing/data_serialization/use_pdx_serializable.html)**
+-   **[Implementing PdxSerializable in Your Domain Object](use_pdx_serializable.html)**
 
     For a domain object with source that you can modify, implement the `PdxSerializable` interface in the object and use its methods to serialize and deserialize the object's fields.
 
--   **[Programming Your Application to Use PdxInstances](../../developing/data_serialization/program_application_for_pdx.html)**
+-   **[Programming Your Application to Use PdxInstances](program_application_for_pdx.html)**
 
     A `PdxInstance` is a light-weight wrapper around PDX serialized bytes. It provides applications with run-time access to fields of a PDX serialized object.
 
--   **[Adding JSON Documents to the Geode Cache](../../developing/data_serialization/jsonformatter_pdxinstances.html)**
+-   **[Adding JSON Documents to the <%=vars.product_name%> Cache](jsonformatter_pdxinstances.html)**
 
     The `JSONFormatter` API allows you to put JSON formatted documents into regions and retrieve them later by storing the documents internally as PdxInstances.
 
--   **[Using PdxInstanceFactory to Create PdxInstances](../../developing/data_serialization/using_PdxInstanceFactory.html)**
+-   **[Using PdxInstanceFactory to Create PdxInstances](using_PdxInstanceFactory.html)**
 
     You can use the `PdxInstanceFactory` interface to create a `PdxInstance` from raw data when the domain class is not available on the server.
 
--   **[Persisting PDX Metadata to Disk](../../developing/data_serialization/persist_pdx_metadata_to_disk.html)**
+-   **[Persisting PDX Metadata to Disk](persist_pdx_metadata_to_disk.html)**
 
-    Geode allows you to persist PDX metadata to disk and specify the disk store to use.
+    <%=vars.product_name%> allows you to persist PDX metadata to disk and specify the disk store to use.
 
--   **[Using PDX Objects as Region Entry Keys](../../developing/data_serialization/using_pdx_region_entry_keys.html)**
+-   **[Using PDX Objects as Region Entry Keys](using_pdx_region_entry_keys.html)**
 
     Using PDX objects as region entry keys is highly discouraged.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/jsonformatter_pdxinstances.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/jsonformatter_pdxinstances.html.md.erb b/geode-docs/developing/data_serialization/jsonformatter_pdxinstances.html.md.erb
index 09aaae2..280012b 100644
--- a/geode-docs/developing/data_serialization/jsonformatter_pdxinstances.html.md.erb
+++ b/geode-docs/developing/data_serialization/jsonformatter_pdxinstances.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Adding JSON Documents to the Geode Cache
----
+<% set_title("Adding JSON Documents to the", product_name, "Cache") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,9 +19,9 @@ limitations under the License.
 
 The `JSONFormatter` API allows you to put JSON formatted documents into regions and retrieve them later by storing the documents internally as PdxInstances.
 
-Geode supports the use of JSON formatted documents natively. When you add a JSON document to a Geode cache, you call the JSONFormatter APIs to transform them into the PDX format (as a `PdxInstance`), which enables Geode to understand the JSON document at a field level.
+<%=vars.product_name%> supports the use of JSON formatted documents natively. When you add a JSON document to a <%=vars.product_name%> cache, you call the JSONFormatter APIs to transform them into the PDX format (as a `PdxInstance`), which enables <%=vars.product_name%> to understand the JSON document at a field level.
 
-In terms of querying and indexing, because the documents are stored internally as PDX, applications can index on any field contained inside the JSON document including any nested field (within JSON objects or JSON arrays.) Any queries run on these stored documents will return PdxInstances as results. To update a JSON document stored in Geode , you can execute a function on the PdxInstance.
+In terms of querying and indexing, because the documents are stored internally as PDX, applications can index on any field contained inside the JSON document including any nested field (within JSON objects or JSON arrays.) Any queries run on these stored documents will return PdxInstances as results. To update a JSON document stored in <%=vars.product_name%> , you can execute a function on the PdxInstance.
 
 You can then use the `JSONFormatter` to convert the PdxInstance results back into the JSON document.
 
@@ -31,14 +29,14 @@ You can then use the `JSONFormatter` to convert the PdxInstance results back int
 
 The `JSONFormatter` class has four static methods that are used to convert JSON document into PdxInstances and then to convert those PdxInstances back into JSON document.
 
-You need to call the following methods before putting any JSON document into the Geode region:
+You need to call the following methods before putting any JSON document into the <%=vars.product_name%> region:
 
 -   `fromJSON`. Creates a PdxInstance from a JSON byte array. Returns the PdxInstance.
 -   `fromJSON`. Creates a PdxInstance from a JSON string. Returns the PdxInstance.
 
-After putting the JSON document into a region as a PdxInstance, you can execute standard Geode queries and create indexes on the JSON document in the same manner you would query or index any other Geode PdxInstance.
+After putting the JSON document into a region as a PdxInstance, you can execute standard <%=vars.product_name%> queries and create indexes on the JSON document in the same manner you would query or index any other <%=vars.product_name%> PdxInstance.
 
-After executing a Geode query or calling `region.get`, you can use the following methods to convert a PdxInstance back into the JSON format:
+After executing a <%=vars.product_name%> query or calling `region.get`, you can use the following methods to convert a PdxInstance back into the JSON format:
 
 -   `toJSON`. Reads a PdxInstance and returns a JSON string.
 -   `toJSONByteArray`. Reads a PdxInstance and returns a JSON byte array.
@@ -47,7 +45,7 @@ For more information on using the JSONFormatter, see the Java API documentation
 
 # Sorting Behavior of Serialized JSON Fields
 
-By default, Geode serialization creates a unique pdx typeID for each unique JSON document, even if the
+By default, <%=vars.product_name%> serialization creates a unique pdx typeID for each unique JSON document, even if the
 only difference between the JSON documents is the order in which their fields are specified. 
 
 If you prefer that JSON documents which differ only in the order in which their fields are specified

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/persist_pdx_metadata_to_disk.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/persist_pdx_metadata_to_disk.html.md.erb b/geode-docs/developing/data_serialization/persist_pdx_metadata_to_disk.html.md.erb
index 2d044ce..7b30eae 100644
--- a/geode-docs/developing/data_serialization/persist_pdx_metadata_to_disk.html.md.erb
+++ b/geode-docs/developing/data_serialization/persist_pdx_metadata_to_disk.html.md.erb
@@ -19,22 +19,22 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode allows you to persist PDX metadata to disk and specify the disk store to use.
+<%=vars.product_name%> allows you to persist PDX metadata to disk and specify the disk store to use.
 
 <a id="persist_pdx_metadata_to_disk__section_7F357A8E56B54BFB9A5778C0F89E034E"></a>
 **Prerequisites**
 
--   Understand generally how to configure the Geode cache. See [Basic Configuration and Programming](../../basic_config/book_intro.html).
--   Understand how Geode disk stores work. See [Disk Storage](../../managing/disk_storage/chapter_overview.html).
+-   Understand generally how to configure the <%=vars.product_name%> cache. See [Basic Configuration and Programming](../../basic_config/book_intro.html).
+-   Understand how <%=vars.product_name%> disk stores work. See [Disk Storage](../../managing/disk_storage/chapter_overview.html).
 
 **Procedure**
 
 1.  Set the `<pdx>` attribute `persistent` to true in your cache configuration. This is required for caches that use PDX with persistent regions and with regions that use a gateway sender to distribute events across a WAN.. Otherwise, it is optional.
-2.  (Optional) If you want to use a disk store that is not the Geode default disk store, set the `<pdx>` attribute `disk-store-name` to the name of your non-default disk store.
+2.  (Optional) If you want to use a disk store that is not the <%=vars.product_name%> default disk store, set the `<pdx>` attribute `disk-store-name` to the name of your non-default disk store.
     **Note:**
     If you are using PDX serialized objects as region entry keys and you are using persistent regions, then you must configure your PDX disk store to be a different one than the disk store used by the persistent regions.
 
-3.  (Optional) If you later want to rename the PDX types that are persisted to disk, you can do so on your offline disk-stores by executing the `pdx                             rename` command. See [pdx rename](../../tools_modules/gfsh/command-pages/pdx.html).
+3.  (Optional) If you later want to rename the PDX types that are persisted to disk, you can do so on your offline disk-stores by executing the `pdx rename` command. See [pdx rename](../../tools_modules/gfsh/command-pages/pdx.html).
 
 **Example cache.xml:**
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/program_application_for_pdx.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/program_application_for_pdx.html.md.erb b/geode-docs/developing/data_serialization/program_application_for_pdx.html.md.erb
index ae8be23..1ff499e 100644
--- a/geode-docs/developing/data_serialization/program_application_for_pdx.html.md.erb
+++ b/geode-docs/developing/data_serialization/program_application_for_pdx.html.md.erb
@@ -44,7 +44,7 @@ When fetching data in a cache with PDX serialized reads enabled, the safest appr
 
 **Prerequisites**
 
--   Understand generally how to configure the Geode cache. See [Basic Configuration and Programming](../../basic_config/book_intro.html#basic_config_management).
+-   Understand generally how to configure the <%=vars.product_name%> cache. See [Basic Configuration and Programming](../../basic_config/book_intro.html#basic_config_management).
 
 <a id="program_application_for_pdx__section_B3C7C7629DFD4483B32B27F84D64DFCF"></a>
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/use_pdx_high_level_steps.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/use_pdx_high_level_steps.html.md.erb b/geode-docs/developing/data_serialization/use_pdx_high_level_steps.html.md.erb
index c4894b6..c21f488 100644
--- a/geode-docs/developing/data_serialization/use_pdx_high_level_steps.html.md.erb
+++ b/geode-docs/developing/data_serialization/use_pdx_high_level_steps.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-To use PDX serialization, you can configure and use Geode's reflection-based autoserializer, or you can program the serialization of your objects by using the PDX interfaces and classes.
+To use PDX serialization, you can configure and use <%=vars.product_name%>'s reflection-based autoserializer, or you can program the serialization of your objects by using the PDX interfaces and classes.
 
 <a id="concept_A7C8890826394B4293C036DD739835BD__section_7F357A8E56B54BFB9A5778C0F89E034E"></a>
 Optionally, program your application code to deserialize individual fields out of PDX representations of your serialized objects. You may also need to persist your PDX metadata to disk for recovery on startup.
@@ -39,7 +39,7 @@ Optionally, program your application code to deserialize individual fields out o
 
     By using gfsh, this configuration can propagated across the cluster through the [Cluster Configuration Service](../../configuring/cluster_config/gfsh_persist.html). Alternately, you would need to configure `pdx read-serialized` in each server's `cache.xml` file.
 
-3.  If you are storing any Geode data on disk, then you must configure PDX serialization to use persistence. See [Persisting PDX Metadata to Disk](persist_pdx_metadata_to_disk.html) for more information.
+3.  If you are storing any <%=vars.product_name%> data on disk, then you must configure PDX serialization to use persistence. See [Persisting PDX Metadata to Disk](persist_pdx_metadata_to_disk.html) for more information.
 4.  (Optional) Wherever you run explicit application code to retrieve and manage your cached entries, you may want to manage your data objects without using full deserialization. To do this, see [Programming Your Application to Use PdxInstances](program_application_for_pdx.html).
 
 ## PDX and Multi-Site (WAN) Deployments

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/use_pdx_serializable.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/use_pdx_serializable.html.md.erb b/geode-docs/developing/data_serialization/use_pdx_serializable.html.md.erb
index 2716814..7c367f6 100644
--- a/geode-docs/developing/data_serialization/use_pdx_serializable.html.md.erb
+++ b/geode-docs/developing/data_serialization/use_pdx_serializable.html.md.erb
@@ -46,8 +46,8 @@ For a domain object with source that you can modify, implement the `PdxSerializa
     ```
 
 3.  Program `PdxSerializable.toData.`
-    1.  Write each standard Java data field of your domain class using the `PdxWriter` write methods. Geode automatically provides `PdxWriter` to the `toData` method for `PdxSerializable` objects.
-    2.  Call the `PdxWriter` `markIdentifyField` method for each field you want to have Geode use to identify your object. Put this after the field's write method. Geode uses this information to compare objects for operations like distinct queries. If you do not set as least one identity field, then the `equals` and `hashCode` methods will use all PDX fields to compare objects and consequently, will not perform as well. It is important that the fields used by your `equals` and `hashCode` implementations are the same fields that you mark as identity fields.
+    1.  Write each standard Java data field of your domain class using the `PdxWriter` write methods. <%=vars.product_name%> automatically provides `PdxWriter` to the `toData` method for `PdxSerializable` objects.
+    2.  Call the `PdxWriter` `markIdentifyField` method for each field you want to have <%=vars.product_name%> use to identify your object. Put this after the field's write method. <%=vars.product_name%> uses this information to compare objects for operations like distinct queries. If you do not set as least one identity field, then the `equals` and `hashCode` methods will use all PDX fields to compare objects and consequently, will not perform as well. It is important that the fields used by your `equals` and `hashCode` implementations are the same fields that you mark as identity fields.
     3.  For a particular version of your class, you need to consistently write the same named field each time. The field names or number of fields must not change from one instance to another for the same class version.
     4.  For best performance, do fixed width fields first and then variable length fields.
 
@@ -85,7 +85,7 @@ For a domain object with source that you can modify, implement the `PdxSerializa
 
     Provide the same names that you did in `toData` and call the read operations in the same order as you called the write operations in your `toData` implementation.
 
-    Geode automatically provides `PdxReader` to the `fromData` method for `PdxSerializable` objects.
+    <%=vars.product_name%> automatically provides `PdxReader` to the `fromData` method for `PdxSerializable` objects.
 
     Example `fromData` code:
 
@@ -110,6 +110,6 @@ For a domain object with source that you can modify, implement the `PdxSerializa
 
 **What to do next**
 
--   As needed, configure and program your Geode applications to use `PdxInstance` for selective object deserialization. See [Programming Your Application to Use PdxInstances](program_application_for_pdx.html).
+-   As needed, configure and program your <%=vars.product_name%> applications to use `PdxInstance` for selective object deserialization. See [Programming Your Application to Use PdxInstances](program_application_for_pdx.html).
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/data_serialization/use_pdx_serializer.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/data_serialization/use_pdx_serializer.html.md.erb b/geode-docs/developing/data_serialization/use_pdx_serializer.html.md.erb
index 8feee8f..74b0a1d 100644
--- a/geode-docs/developing/data_serialization/use_pdx_serializer.html.md.erb
+++ b/geode-docs/developing/data_serialization/use_pdx_serializer.html.md.erb
@@ -81,7 +81,7 @@ The `PdxSerializer` `toData` and `fromData` methods differ from those for `PdxSe
 3.  Program `PdxSerializer.toData` to recognize, cast, and handle your domain object:
 
     1.  Write each standard Java data field of your domain class using the `PdxWriter` write methods.
-    2.  Call the `PdxWriter` `markIdentityField` method for each field you want to have Geode use to identify your object. Put this after the field's write method. Geode uses this information to compare objects for operations like distinct queries. If you do not set as least one identity field, then the `equals` and `hashCode` methods will use all PDX fields to compare objects and consequently, will not perform as well. It is important that the fields used by your `equals` and `hashCode` implementations are the same fields that you mark as identity fields.
+    2.  Call the `PdxWriter` `markIdentityField` method for each field you want to have <%=vars.product_name%> use to identify your object. Put this after the field's write method. <%=vars.product_name%> uses this information to compare objects for operations like distinct queries. If you do not set as least one identity field, then the `equals` and `hashCode` methods will use all PDX fields to compare objects and consequently, will not perform as well. It is important that the fields used by your `equals` and `hashCode` implementations are the same fields that you mark as identity fields.
     3.  For a particular version of your class, you need to consistently write the same named field each time. The field names or number of fields must not change from one instance to another for the same class version.
     4.  For best performance, do fixed width fields first and then variable length fields.
     5.  If desired, you can check the portability of the object before serializing it by adding the `checkPortability` parameter when using the`                                 PdxWriter` `writeObject`, `writeObjectArray`, and `writeField` methods.
@@ -115,7 +115,7 @@ The `PdxSerializer` `toData` and `fromData` methods differ from those for `PdxSe
 
         Provide the same names that you did in `toData` and call the read operations in the same order as you called the write operations in your `toData` implementation.
 
-        Geode provides the domain class type and `PdxReader` to the `fromData` method.
+        <%=vars.product_name%> provides the domain class type and `PdxReader` to the `fromData` method.
 
         Example `fromData` code:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/delta_propagation/delta_propagation_example.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/delta_propagation/delta_propagation_example.html.md.erb b/geode-docs/developing/delta_propagation/delta_propagation_example.html.md.erb
index 7a81962..f4b0b4a 100644
--- a/geode-docs/developing/delta_propagation/delta_propagation_example.html.md.erb
+++ b/geode-docs/developing/delta_propagation/delta_propagation_example.html.md.erb
@@ -28,9 +28,9 @@ In this example, the feeder client is connected to the first server, and the rec
 
 The example demonstrates the following operations:
 
-1.  In the Feeder client, the application updates the entry object and puts the entry. In response to the `put`, Geode calls `hasDelta`, which returns true, so Geode calls `toDelta` and forwards the extracted delta to the server. If `hasDelta` returned false, Geode would distribute the full entry value.
-2.  In Server1, Geode applies the delta to the cache, distributes the received delta to the server's peers, and forwards it to any other clients with interest in the entry (there are no other clients to Server1 in this example)
-3.  In Server2, Geode applies the delta to the cache and forwards it to its interested clients, which in this case is just the Receiver client.
+1.  In the Feeder client, the application updates the entry object and puts the entry. In response to the `put`, <%=vars.product_name%> calls `hasDelta`, which returns true, so <%=vars.product_name%> calls `toDelta` and forwards the extracted delta to the server. If `hasDelta` returned false, <%=vars.product_name%> would distribute the full entry value.
+2.  In Server1, <%=vars.product_name%> applies the delta to the cache, distributes the received delta to the server's peers, and forwards it to any other clients with interest in the entry (there are no other clients to Server1 in this example)
+3.  In Server2, <%=vars.product_name%> applies the delta to the cache and forwards it to its interested clients, which in this case is just the Receiver client.
 
 <a id="delta_propagation_example__section_185444FC51FB467587A62DFEC07C9C7D"></a>
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/delta_propagation/delta_propagation_properties.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/delta_propagation/delta_propagation_properties.html.md.erb b/geode-docs/developing/delta_propagation/delta_propagation_properties.html.md.erb
index 0b5b40a..b5fddd1 100644
--- a/geode-docs/developing/delta_propagation/delta_propagation_properties.html.md.erb
+++ b/geode-docs/developing/delta_propagation/delta_propagation_properties.html.md.erb
@@ -54,12 +54,12 @@ Exceptions to this behavior:
 
 Cloning can be expensive, but it ensures that the new object is fully initialized with the delta before any application code sees it.
 
-When cloning is enabled, by default Geode does a deep copy of the object, using serialization. You may be able to improve performance by implementing `java.lang.Cloneable` and then implementing the `clone` method, making a deep copy of anything to which a delta may be applied. The goal is to reduce significantly the overhead of copying the object while still retaining the isolation needed for your deltas.
+When cloning is enabled, by default <%=vars.product_name%> does a deep copy of the object, using serialization. You may be able to improve performance by implementing `java.lang.Cloneable` and then implementing the `clone` method, making a deep copy of anything to which a delta may be applied. The goal is to reduce significantly the overhead of copying the object while still retaining the isolation needed for your deltas.
 
 Without cloning:
 
 -   It is possible for application code to read the entry value as it is being modified, possibly seeing the value in an intermediate, inconsistent state, with just part of the delta applied. You may choose to resolve this issue by having your application code synchronize on reads and writes.
--   Geode loses any reference to the old value because the old value is transformed in place into the new value. Because of this, your `CacheListener` sees the same new value returned for `EntryEvent.getOldValue` and `EntryEvent.getNewValue` .
+-   <%=vars.product_name%> loses any reference to the old value because the old value is transformed in place into the new value. Because of this, your `CacheListener` sees the same new value returned for `EntryEvent.getOldValue` and `EntryEvent.getNewValue` .
 -   Exceptions thrown from `fromDelta` may leave your cache in an inconsistent state. Without cloning, any interruption of the delta application could leave you with some of the fields in your cached object changed and others unchanged. If you do not use cloning, keep this in mind when you program your error handling in your `fromDelta` implementation.
 
 With cloning:

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb b/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb
index 7aec9ab..3609734 100644
--- a/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb
+++ b/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb
@@ -26,7 +26,7 @@ In most distributed data management systems, the data stored in the system tends
 
 <a id="how_delta_propagation_works__section_ABE3589920D6477BBB2223A583AF169A"></a>
 
-Geode propagates object deltas using methods that you program. The methods are in the `Delta` interface, which you implement in your cached objects' classes. If any of your classes are plain old Java objects, you need to wrap them for this implementation.
+<%=vars.product_name%> propagates object deltas using methods that you program. The methods are in the `Delta` interface, which you implement in your cached objects' classes. If any of your classes are plain old Java objects, you need to wrap them for this implementation.
 
 This figure shows delta propagation for a change to an entry with key, k, and value object, v.
 
@@ -48,21 +48,21 @@ Sometimes `fromDelta` cannot be invoked because there is no object to apply the
 1.  If the system can determine beforehand that the receiver does not have a local copy, it sends the initial message with the full value. This is possible when regions are configured with no local data storage, such as with the region shortcut settings `PARTITION_PROXY` and `REPLICATE_PROXY`. These configurations are used to accomplish things like provide data update information to listeners and to pass updates forward to clients.
 2.  In less obvious cases, such as when an entry has been locally deleted, first the delta is sent, then the receiver requests a full value and that is sent. Whenever the full value is received, any further distributions to the receiver's peers or clients uses the full value.
 
-Geode also does not propagate deltas for:
+<%=vars.product_name%> also does not propagate deltas for:
 
 -   Transactional commit
 -   The `putAll` operation
--   JVMs running Geode versions that do not support delta propagation (6.0 and earlier)
+-   JVMs running <%=vars.product_name%> versions that do not support delta propagation (6.0 and earlier)
 
 ## <a id="how_delta_propagation_works__section_F4A102A74530429F87BEA53C90D5CCFB" class="no-quick-link"></a>Supported Topologies and Limitations
 
 The following topologies support delta propagation (with some limitations):
 
--   **Peer-to-peer**. Geode system members distribute and receive entry changes using delta propagation, with these requirements and caveats:
+-   **Peer-to-peer**. <%=vars.product_name%> system members distribute and receive entry changes using delta propagation, with these requirements and caveats:
     -   Regions must be partitioned or have their scope set to `distributed-ack` or `global`. The region shortcut settings for distributed regions use `distributed-ack` `scope`. Delta propagation does not work for regions with `distributed-no-ack` `scope` because the receiver could not recover if an exception occurred while applying the delta.
     -   For partitioned regions, if a receiving peer does not hold the primary or a secondary copy of the entry, but still requires a value, the system automatically sends the full value.
     -   To receive deltas, a region must be non-empty. The system automatically sends the full value to empty regions. Empty regions can send deltas.
--   **Client/server**. Geode clients can always send deltas to the servers, and servers can usually sent deltas to clients. These configurations require the servers to send full values to the clients, instead of deltas:
+-   **Client/server**. <%=vars.product_name%> clients can always send deltas to the servers, and servers can usually sent deltas to clients. These configurations require the servers to send full values to the clients, instead of deltas:
     -   When the client's `gemfire.properties` setting `conflate-events` is set to true, the servers send full values for all regions.
     -   When the server region attribute `enable-subscription-conflation` is set to true and the client `gemfire.properties` setting `conflate-events` is set to `server`, the servers send full values for the region.
     -   When the client region is configured with the `PROXY` client region shortcut setting (empty client region), servers send full values.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/delta_propagation/implementing_delta_propagation.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/delta_propagation/implementing_delta_propagation.html.md.erb b/geode-docs/developing/delta_propagation/implementing_delta_propagation.html.md.erb
index 4b6ae99..5aacf91 100644
--- a/geode-docs/developing/delta_propagation/implementing_delta_propagation.html.md.erb
+++ b/geode-docs/developing/delta_propagation/implementing_delta_propagation.html.md.erb
@@ -26,12 +26,12 @@ Use the following procedure to implement delta propagation in your distributed s
 
 1.  Study your object types and expected application behavior to determine which regions can benefit from using delta propagation. Delta propagation does not improve performance for all data and data modification scenarios. See [When to Avoid Delta Propagation](when_to_use_delta_prop.html#when_to_use_delta_prop).
 2.  For each region where you are using delta propagation, choose whether to enable cloning using the delta propagation property `cloning-enabled`. Cloning is disabled by default. See [Delta Propagation Properties](delta_propagation_properties.html#delta_propagation_properties).
-3.  If you do not enable cloning, review all associated listener code for dependencies on `EntryEvent.getOldValue`. Without cloning, Geode modifies the entry in place and so loses its reference to the old value. For delta events, the `EntryEvent` methods `getOldValue` and `getNewValue` both return the new value.
+3.  If you do not enable cloning, review all associated listener code for dependencies on `EntryEvent.getOldValue`. Without cloning, <%=vars.product_name%> modifies the entry in place and so loses its reference to the old value. For delta events, the `EntryEvent` methods `getOldValue` and `getNewValue` both return the new value.
 4.  For every class where you want delta propagation, implement `org.apache.geode.Delta` and update your methods to support delta propagation. Exactly how you do this depends on your application and object needs, but these steps describe the basic approach:
     1.  If the class is a plain old Java object (POJO), wrap it for this implementation and update your code to work with the wrapper class.
     2.  Define as transient any extra object fields that you use to manage delta state. This can help performance when the full object is distributed. Whenever standard Java serialization is used, the transient keyword indicates to Java to not serialize the field.
     3.  Study the object contents to decide how to handle delta changes. Delta propagation has the same issues of distributed concurrency control as the distribution of full objects, but on a more detailed level. Some parts of your objects may be able to change independent of one another while others may always need to change together. Send deltas large enough to keep your data logically consistent. If, for example, field A and field B depend on each other, then your delta distributions should either update both fields or neither. As with regular updates, the fewer producers you have on a data region, the lower your likelihood of concurrency issues.
-    4.  In the application code that puts entries, put the fully populated object into the local cache. Even though you are planning to send only deltas, errors on the receiving end could cause Geode to request the full object, so you must provide it to the originating put method. Do this even in empty producers, with regions configured for no local data storage. This usually means doing a get on the entry unless you are sure it does not already exist anywhere in the distributed region.
+    4.  In the application code that puts entries, put the fully populated object into the local cache. Even though you are planning to send only deltas, errors on the receiving end could cause <%=vars.product_name%> to request the full object, so you must provide it to the originating put method. Do this even in empty producers, with regions configured for no local data storage. This usually means doing a get on the entry unless you are sure it does not already exist anywhere in the distributed region.
     5.  Change each field's update method to record information about the update. The information must be sufficient for `toDelta` to encode the delta and any additional required delta information when it is invoked.
     6.  Write `hasDelta` to report on whether a delta is available.
     7.  Write `toDelta` to create a byte stream with the changes to the object and any other information `fromDelta` will need to apply the changes. Before returning from `toDelta`, reset your delta state to indicate that there are no delta changes waiting to be sent.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/distributed_regions/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/distributed_regions/chapter_overview.html.md.erb b/geode-docs/developing/distributed_regions/chapter_overview.html.md.erb
index d24de37..48fe83d 100644
--- a/geode-docs/developing/distributed_regions/chapter_overview.html.md.erb
+++ b/geode-docs/developing/distributed_regions/chapter_overview.html.md.erb
@@ -19,25 +19,25 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-In addition to basic region management, distributed and replicated regions include options for things like push and pull distribution models, global locking, and region entry versions to ensure consistency across Geode members.
+In addition to basic region management, distributed and replicated regions include options for things like push and pull distribution models, global locking, and region entry versions to ensure consistency across <%=vars.product_name%> members.
 
--   **[How Distribution Works](../../developing/distributed_regions/how_distribution_works.html)**
+-   **[How Distribution Works](how_distribution_works.html)**
 
     To use distributed and replicated regions, you should understand how they work and your options for managing them.
 
--   **[Options for Region Distribution](../../developing/distributed_regions/choosing_level_of_dist.html)**
+-   **[Options for Region Distribution](choosing_level_of_dist.html)**
 
-    You can use distribution with and without acknowledgment, or global locking for your region distribution. Regions that are configured for distribution with acknowledgment can also be configured to resolve concurrent updates consistently across all Geode members that host the region.
+    You can use distribution with and without acknowledgment, or global locking for your region distribution. Regions that are configured for distribution with acknowledgment can also be configured to resolve concurrent updates consistently across all <%=vars.product_name%> members that host the region.
 
--   **[How Replication and Preloading Work](../../developing/distributed_regions/how_replication_works.html)**
+-   **[How Replication and Preloading Work](how_replication_works.html)**
 
     To work with replicated and preloaded regions, you should understand how their data is initialized and maintained in the cache.
 
--   **[Configure Distributed, Replicated, and Preloaded Regions](../../developing/distributed_regions/managing_distributed_regions.html)**
+-   **[Configure Distributed, Replicated, and Preloaded Regions](managing_distributed_regions.html)**
 
     Plan the configuration and ongoing management of your distributed, replicated, and preloaded regions, and configure the regions.
 
--   **[Locking in Global Regions](../../developing/distributed_regions/locking_in_global_regions.html)**
+-   **[Locking in Global Regions](locking_in_global_regions.html)**
 
     In global regions, the system locks entries and the region during updates. You can also explicitly lock the region and its entries as needed by your application. Locking includes system settings that help you optimize performance and locking behavior between your members.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/distributed_regions/choosing_level_of_dist.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/distributed_regions/choosing_level_of_dist.html.md.erb b/geode-docs/developing/distributed_regions/choosing_level_of_dist.html.md.erb
index 3d48ab4..72cfcfe 100644
--- a/geode-docs/developing/distributed_regions/choosing_level_of_dist.html.md.erb
+++ b/geode-docs/developing/distributed_regions/choosing_level_of_dist.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-You can use distribution with and without acknowledgment, or global locking for your region distribution. Regions that are configured for distribution with acknowledgment can also be configured to resolve concurrent updates consistently across all Geode members that host the region.
+You can use distribution with and without acknowledgment, or global locking for your region distribution. Regions that are configured for distribution with acknowledgment can also be configured to resolve concurrent updates consistently across all <%=vars.product_name%> members that host the region.
 
 <a id="choosing_level_of_dist__section_F2528B151DD54CEFA05C4BA655BCF016"></a>
 Each distributed region must have the same scope and concurrency checking setting throughout the distributed system.


[10/11] geode git commit: GEODE-3395 Variable-ize product version and name in user guide - Developing (top-level book file)

Posted by zh...@apache.org.
GEODE-3395 Variable-ize product version and name in user guide - Developing (top-level book file)


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

Branch: refs/heads/feature/GEM-1601
Commit: 1c04aabb76b1e990899069fc864c8b96f8f63300
Parents: 36daa9a
Author: Dave Barnes <db...@pivotal.io>
Authored: Fri Aug 18 16:00:46 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Fri Aug 18 16:00:46 2017 -0700

----------------------------------------------------------------------
 geode-docs/developing/book_intro.html.md.erb | 40 +++++++++++------------
 1 file changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/1c04aabb/geode-docs/developing/book_intro.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/book_intro.html.md.erb b/geode-docs/developing/book_intro.html.md.erb
index c78f753..72c2c69 100644
--- a/geode-docs/developing/book_intro.html.md.erb
+++ b/geode-docs/developing/book_intro.html.md.erb
@@ -19,53 +19,53 @@ limitations under the License.
 
 *Developing with <%=vars.product_name_long%>* explains main concepts of application programming with <%=vars.product_name_long%>. It describes how to plan and implement regions, data serialization, event handling, delta propagation, transactions, and more.
 
-For information about Geode REST application development, see [Developing REST Applications for <%=vars.product_name_long%>](../rest_apps/book_intro.html).
+For information about <%=vars.product_name%> REST application development, see [Developing REST Applications for <%=vars.product_name_long%>](../rest_apps/book_intro.html).
 
--   **[Region Data Storage and Distribution](../developing/region_options/chapter_overview.html)**
+-   **[Region Data Storage and Distribution](region_options/chapter_overview.html)**
 
-    The <%=vars.product_name_long%> data storage and distribution models put your data in the right place at the right time. You should understand all the options for data storage in Geode before you start configuring your data regions.
+    The <%=vars.product_name_long%> data storage and distribution models put your data in the right place at the right time. You should understand all the options for data storage in <%=vars.product_name%> before you start configuring your data regions.
 
--   **[Partitioned Regions](../developing/partitioned_regions/chapter_overview.html)**
+-   **[Partitioned Regions](partitioned_regions/chapter_overview.html)**
 
     In addition to basic region management, partitioned regions include options for high availability, data location control, and data balancing across the distributed system.
 
--   **[Distributed and Replicated Regions](../developing/distributed_regions/chapter_overview.html)**
+-   **[Distributed and Replicated Regions](distributed_regions/chapter_overview.html)**
 
-    In addition to basic region management, distributed and replicated regions include options for things like push and pull distribution models, global locking, and region entry versions to ensure consistency across Geode members.
+    In addition to basic region management, distributed and replicated regions include options for things like push and pull distribution models, global locking, and region entry versions to ensure consistency across <%=vars.product_name%> members.
 
--   **[Consistency for Region Updates](../developing/distributed_regions/region_entry_versions.html)**
+-   **[Consistency for Region Updates](distributed_regions/region_entry_versions.html)**
 
-    Geode ensures that all copies of a region eventually reach a consistent state on all members and clients that host the region, including Geode members that distribute region events.
+    <%=vars.product_name%> ensures that all copies of a region eventually reach a consistent state on all members and clients that host the region, including <%=vars.product_name%> members that distribute region events.
 
--   **[General Region Data Management](../developing/management_all_region_types/chapter_overview.html)**
+-   **[General Region Data Management](management_all_region_types/chapter_overview.html)**
 
     For all regions, you have options to control memory use, back up your data to disk, and keep stale data out of your cache.
 
--   **[Data Serialization](../developing/data_serialization/chapter_overview.html)**
+-   **[Data Serialization](data_serialization/chapter_overview.html)**
 
-    Data that you manage in Geode must be serialized and deserialized for storage and transmittal between processes. You can choose among several options for data serialization.
+    Data that you manage in <%=vars.product_name%> must be serialized and deserialized for storage and transmittal between processes. You can choose among several options for data serialization.
 
--   **[Events and Event Handling](../developing/events/chapter_overview.html)**
+-   **[Events and Event Handling](events/chapter_overview.html)**
 
-    Geode provides versatile and reliable event distribution and handling for your cached data and system member events.
+    <%=vars.product_name%> provides versatile and reliable event distribution and handling for your cached data and system member events.
 
--   **[Delta Propagation](../developing/delta_propagation/chapter_overview.html)**
+-   **[Delta Propagation](delta_propagation/chapter_overview.html)**
 
     Delta propagation allows you to reduce the amount of data you send over the network by including only changes to objects rather than the entire object.
 
--   **[Querying](../developing/querying_basics/chapter_overview.html)**
+-   **[Querying](querying_basics/chapter_overview.html)**
 
-    Geode provides a SQL-like querying language called OQL that allows you to access data stored in Geode regions.
+    <%=vars.product_name%> provides a SQL-like querying language called OQL that allows you to access data stored in <%=vars.product_name%> regions.
 
--   **[Continuous Querying](../developing/continuous_querying/chapter_overview.html)**
+-   **[Continuous Querying](continuous_querying/chapter_overview.html)**
 
     Continuous querying continuously returns events that match the queries you set up.
 
--   **[Transactions](../developing/transactions/chapter_overview.html)**
+-   **[Transactions](transactions/chapter_overview.html)**
 
-    Geode provides a transactions API, with `begin`, `commit`, and `rollback` methods. These methods are much the same as the familiar relational database transactions methods.
+    <%=vars.product_name%> provides a transactions API, with `begin`, `commit`, and `rollback` methods. These methods are much the same as the familiar relational database transactions methods.
 
--   **[Function Execution](../developing/function_exec/chapter_overview.html)**
+-   **[Function Execution](function_exec/chapter_overview.html)**
 
     A function is a body of code that resides on a server and that an application can invoke from a client or from another server without the need to send the function code itself. The caller can direct a data-dependent function to operate on a particular dataset, or can direct a data-independent function to operate on a particular server, member, or member group.
 


[04/11] geode git commit: GEODE-3395 Variable-ize product version and name in user guide - Developing

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/JTA_transactions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/JTA_transactions.html.md.erb b/geode-docs/developing/transactions/JTA_transactions.html.md.erb
index ffb6082..0dcc4fe 100644
--- a/geode-docs/developing/transactions/JTA_transactions.html.md.erb
+++ b/geode-docs/developing/transactions/JTA_transactions.html.md.erb
@@ -1,6 +1,4 @@
----
-title: JTA Global Transactions with Geode
----
+<% set_title("JTA Global Transactions with", product_name) %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,42 +18,42 @@ limitations under the License.
 -->
 
 
-Use JTA global transactions to coordinate Geode cache transactions and JDBC transactions.
+Use JTA global transactions to coordinate <%=vars.product_name%> cache transactions and JDBC transactions.
 
-JTA is a standard Java interface you can use to coordinate Geode cache transactions and JDBC transactions globally under one umbrella. JTA provides direct coordination between the Geode cache and another transactional resource, such as a database. The parties involved in a JTA transaction include:
+JTA is a standard Java interface you can use to coordinate <%=vars.product_name%> cache transactions and JDBC transactions globally under one umbrella. JTA provides direct coordination between the <%=vars.product_name%> cache and another transactional resource, such as a database. The parties involved in a JTA transaction include:
 
 -   The Java application, responsible for starting the global transaction
 -   The JTA transaction manager, responsible for opening, committing, and rolling back transactions
--   The transaction resource managers, including the Geode cache transaction manager and the JDBC resource manager, responsible for managing operations in the cache and database respectively
+-   The transaction resource managers, including the <%=vars.product_name%> cache transaction manager and the JDBC resource manager, responsible for managing operations in the cache and database respectively
 
-Using JTA, your application controls all transactions in the same standard way, whether the transactions act on the Geode cache, a JDBC resource, or both together. When a JTA global transaction is done, the Geode transaction and the database transaction are both complete.
+Using JTA, your application controls all transactions in the same standard way, whether the transactions act on the <%=vars.product_name%> cache, a JDBC resource, or both together. When a JTA global transaction is done, the <%=vars.product_name%> transaction and the database transaction are both complete.
 
-When using JTA global transactions with Geode, you have three options:
+When using JTA global transactions with <%=vars.product_name%>, you have three options:
 
 1.  Coordinate with an external JTA transaction manager in a container (such as WebLogic or JBoss)
-2.  Set Geode as the “last resource” while using a container (such as WebLogic or JBoss) as the JTA transaction manager
-3.  Have Geode act as the JTA transaction manager
+2.  Set <%=vars.product_name%> as the “last resource” while using a container (such as WebLogic or JBoss) as the JTA transaction manager
+3.  Have <%=vars.product_name%> act as the JTA transaction manager
 
-An application creates a global transaction by using `javax.transaction.UserTransaction` bound to the JNDI context `java:/UserTransaction` to start and terminate transactions. During the transaction, cache operations are done through Geode as usual as described in [Geode Cache Transactions](cache_transactions.html#topic_e15_mr3_5k).
+An application creates a global transaction by using `javax.transaction.UserTransaction` bound to the JNDI context `java:/UserTransaction` to start and terminate transactions. During the transaction, cache operations are done through <%=vars.product_name%> as usual as described in [<%=vars.product_name%> Cache Transactions](cache_transactions.html#topic_e15_mr3_5k).
 
 **Note:**
 See the Sun documentation for more information on topics such as JTA, `javax.transaction`, committing and rolling back global transactions, and the related exceptions.
 
 -   **[Coordinating with External JTA Transactions Managers](#concept_cp1_zx1_wk)**
 
-    Geode can work with the JTA transaction managers of several containers like JBoss, WebLogic, GlassFish, and so on.
+    <%=vars.product_name%> can work with the JTA transaction managers of several containers like JBoss, WebLogic, GlassFish, and so on.
 
--   **[Using Geode as the "Last Resource" in a Container-Managed JTA Transaction](#concept_csy_vfb_wk)**
+-   **[Using <%=vars.product_name%> as the "Last Resource" in a Container-Managed JTA Transaction](#concept_csy_vfb_wk)**
 
-    The "last resource" feature in certain 3rd party containers such as WebLogic allow the use one non-XAResource (such as Geode) in a transaction with multiple XAResources while ensuring consistency.
+    The "last resource" feature in certain 3rd party containers such as WebLogic allow the use one non-XAResource (such as <%=vars.product_name%>) in a transaction with multiple XAResources while ensuring consistency.
 
--   **[Using Geode as the JTA Transaction Manager](#concept_8567sdkbigige)**
+-   **[Using <%=vars.product_name%> as the JTA Transaction Manager](#concept_8567sdkbigige)**
 
-    You can also use Geode as the JTA transaction manager.
+    You can also use <%=vars.product_name%> as the JTA transaction manager.
 
--   **[Behavior of Geode Cache Writers and Loaders Under JTA](cache_plugins_with_jta.html)**
+-   **[Behavior of <%=vars.product_name%> Cache Writers and Loaders Under JTA](cache_plugins_with_jta.html)**
 
-    When Geode participates in a global transactions, you can still have Geode cache writers and cache loaders operating in the usual way.
+    When <%=vars.product_name%> participates in a global transactions, you can still have <%=vars.product_name%> cache writers and cache loaders operating in the usual way.
 
 -   **[Turning Off JTA Transactions](turning_off_jta.html)**
 
@@ -65,31 +63,31 @@ See the Sun documentation for more information on topics such as JTA, `javax.tra
 
 # Coordinating with External JTA Transactions Managers
 
-Geode can work with the JTA transaction managers of several containers like JBoss, WebLogic, GlassFish, and so on.
+<%=vars.product_name%> can work with the JTA transaction managers of several containers like JBoss, WebLogic, GlassFish, and so on.
 
-At startup Geode looks for a TransactionManager (`javax.transaction.TransactionManager`) that has been bound to its JNDI context. When Geode finds such an external transaction manager, all Geode region operations (such as get and put) will participate in global transactions hosted by this external JTA transaction manager.
+At startup <%=vars.product_name%> looks for a TransactionManager (`javax.transaction.TransactionManager`) that has been bound to its JNDI context. When <%=vars.product_name%> finds such an external transaction manager, all <%=vars.product_name%> region operations (such as get and put) will participate in global transactions hosted by this external JTA transaction manager.
 
-This figure shows the high-level operation of a JTA global transaction whose resources include a Geode cache and a database.
+This figure shows the high-level operation of a JTA global transaction whose resources include a <%=vars.product_name%> cache and a database.
 
 <img src="../../images/transactions_jta_app_server.png" id="concept_cp1_zx1_wk__image_C2935E48415349659FC39BF5C7E75579" class="image" />
 
 An externally coordinated JTA global transaction is run in the following manner:
 
-1.  Each region operation looks up for presence of a global transaction. If one is detected, then a Geode transaction is started automatically, and we register a `javax.transaction.Synchronization` callback with the external JTA transaction manager.
-2.  At transaction commit, Geode gets a `beforeCommit()` callback from the external JTA transaction manager. Geode does all locking and conflict detection at this time. If this fails, an exception is thrown back to JTA transaction manager, which then aborts the transaction.
+1.  Each region operation looks up for presence of a global transaction. If one is detected, then a <%=vars.product_name%> transaction is started automatically, and we register a `javax.transaction.Synchronization` callback with the external JTA transaction manager.
+2.  At transaction commit, <%=vars.product_name%> gets a `beforeCommit()` callback from the external JTA transaction manager. <%=vars.product_name%> does all locking and conflict detection at this time. If this fails, an exception is thrown back to JTA transaction manager, which then aborts the transaction.
 3.  After a successful `beforeCommit()`callback, JTA transaction manager asks other data sources to commit their transaction.
-4.  Geode then gets a `afterCommit()` callback in which changes are applied to the cache and distributed to other members.
+4.  <%=vars.product_name%> then gets a `afterCommit()` callback in which changes are applied to the cache and distributed to other members.
 
 You can disable JTA in any region that should not participate in JTA transactions. See [Turning Off JTA Transactions](turning_off_jta.html#concept_nw2_5gs_xk).
 
 ## <a id="task_j3g_3mn_1l" class="no-quick-link"></a>How to Run a JTA Transaction Coordinated by an External Transaction Manager
 
-Use the following procedure to run a Geode global JTA transaction coordinated by an external JTA transaction manager.
+Use the following procedure to run a <%=vars.product_name%> global JTA transaction coordinated by an external JTA transaction manager.
 
 1.  **Configure the external data sources in the external container.** Do not configure the data sources in cache.xml . They are not guaranteed to get bound to the JNDI tree.
 2.  
 
-    Configure Geode for any necessary transactional behavior in the `cache.xml` file. For example, enable `copy-on-read` and specify a transaction listener, as needed. See [Setting Global Copy on Read](working_with_transactions.html#concept_vx2_gs4_5k) and [Configuring Transaction Plug-In Event Handlers](working_with_transactions.html#concept_ocw_vf1_wk) for details. 
+    Configure <%=vars.product_name%> for any necessary transactional behavior in the `cache.xml` file. For example, enable `copy-on-read` and specify a transaction listener, as needed. See [Setting Global Copy on Read](working_with_transactions.html#concept_vx2_gs4_5k) and [Configuring Transaction Plug-In Event Handlers](working_with_transactions.html#concept_ocw_vf1_wk) for details. 
 3.  
 
     Make sure that JTA transactions are enabled for the regions that will participate in the transaction. See [Turning Off JTA Transactions](turning_off_jta.html#concept_nw2_5gs_xk) for details. 
@@ -98,7 +96,7 @@ Use the following procedure to run a Geode global JTA transaction coordinated by
      Start the transaction through the external container. 
 5.  
 
-    Initialize the Geode cache. Geode will automatically join the transaction. 
+    Initialize the <%=vars.product_name%> cache. <%=vars.product_name%> will automatically join the transaction. 
 6.  
 
      Execute operations in the cache and the database as usual. 
@@ -108,22 +106,22 @@ Use the following procedure to run a Geode global JTA transaction coordinated by
 
 <a id="concept_csy_vfb_wk"></a>
 
-# Using Geode as the "Last Resource" in a Container-Managed JTA Transaction
+# Using <%=vars.product_name%> as the "Last Resource" in a Container-Managed JTA Transaction
 
-The "last resource" feature in certain 3rd party containers such as WebLogic allow the use one non-XAResource (such as Geode) in a transaction with multiple XAResources while ensuring consistency.
+The "last resource" feature in certain 3rd party containers such as WebLogic allow the use one non-XAResource (such as <%=vars.product_name%>) in a transaction with multiple XAResources while ensuring consistency.
 
-In the previous two JTA transaction use cases, if the Geode member fails after the other data sources commit but before Geode receives the `afterCommit` callback, Geode and the other data sources may become inconsistent. To prevent this from occurring, you can use the container's "last resource optimization" feature, with Geode set as the "last resource". Using Geode as the last resource ensures that in the event of failure, Geode remains consistent with the other XAResources involved in the transaction.
+In the previous two JTA transaction use cases, if the <%=vars.product_name%> member fails after the other data sources commit but before <%=vars.product_name%> receives the `afterCommit` callback, <%=vars.product_name%> and the other data sources may become inconsistent. To prevent this from occurring, you can use the container's "last resource optimization" feature, with <%=vars.product_name%> set as the "last resource". Using <%=vars.product_name%> as the last resource ensures that in the event of failure, <%=vars.product_name%> remains consistent with the other XAResources involved in the transaction.
 
-To accomplish this, the application server container must use a JCA Resource Adapter to accomodate Geode as the transaction's last resource. The transaction manager of the container first issues a "prepare" message to the participating XAResources. If the XAResources all accept the transaction, then the manager issues a "commit" instruction to the non-XAResource (in this case, Geode). The non-XAResource (in this case, Geode) participates as a local transaction resource. If the non-XAResource fails, then the transaction manager can rollback the XAResources.
+To accomplish this, the application server container must use a JCA Resource Adapter to accomodate <%=vars.product_name%> as the transaction's last resource. The transaction manager of the container first issues a "prepare" message to the participating XAResources. If the XAResources all accept the transaction, then the manager issues a "commit" instruction to the non-XAResource (in this case, <%=vars.product_name%>). The non-XAResource (in this case, <%=vars.product_name%>) participates as a local transaction resource. If the non-XAResource fails, then the transaction manager can rollback the XAResources.
 
 <img src="../../images/transactions_jca_adapter.png" id="concept_csy_vfb_wk__image_opb_sgb_wk" class="image" />
 
 <a id="task_sln_x3b_wk"></a>
 
-# How to Run JTA Transactions with Geode as a "Last Resource"
+# How to Run JTA Transactions with <%=vars.product_name%> as a "Last Resource"
 
 1.  Locate the version-specific `geode-jca` RAR file within 
-the `lib` directory of your Geode installation. 
+the `lib` directory of your <%=vars.product_name%> installation. 
 2.  Add your container-specific XML file to the `geode-jca` RAR file. 
 <ol>
 <li>Create a container-specific resource adapter XML file named &lt;container&gt;-ra.xml. For example, an XML file for a WebLogic resource adapter XML file might look something like this:
@@ -158,7 +156,7 @@ the CLASSPATH of the JTA transaction coordinator container.
 4.  Deploy the version-specific `geode-jca` RAR file on 
 the JTA transaction coordinator container.
 When deploying the file, you specify the JNDI name and so on. 
-5.  Configure Geode for any necessary transactional behavior. Enable `copy-on-read` and specify a transaction listener, if you need one. See [Setting Global Copy on Read](working_with_transactions.html#concept_vx2_gs4_5k) and [Configuring Transaction Plug-In Event Handlers](working_with_transactions.html#concept_ocw_vf1_wk) for details.
+5.  Configure <%=vars.product_name%> for any necessary transactional behavior. Enable `copy-on-read` and specify a transaction listener, if you need one. See [Setting Global Copy on Read](working_with_transactions.html#concept_vx2_gs4_5k) and [Configuring Transaction Plug-In Event Handlers](working_with_transactions.html#concept_ocw_vf1_wk) for details.
 6.  Get an initial context through `org.apache.geode.cache.GemFireCache.getJNDIContext`. For example:
 
     ``` pre
@@ -167,13 +165,13 @@ When deploying the file, you specify the JNDI name and so on.
 
     This returns `javax.naming.Context` and gives you the JNDI associated with the cache. The context contains the `TransactionManager`, `UserTransaction`, and any configured JDBC resource manager.
 
-7.  Start and commit the global transaction using the `UserTransaction` object rather than with Geode's `CacheTransactionManager`. 
+7.  Start and commit the global transaction using the `UserTransaction` object rather than with <%=vars.product_name%>'s `CacheTransactionManager`. 
 
     ``` pre
     UserTransaction txManager = (UserTransaction)ctx.lookup("java:/UserTransaction");
     ```
 
-8.  Obtain a Geode connection.
+8.  Obtain a <%=vars.product_name%> connection.
 
     ``` pre
     GFConnectionFactory cf = (GFConnectionFactory) ctx.lookup("gfe/jca");
@@ -187,40 +185,40 @@ When deploying the file, you specify the JNDI name and so on.
 
 See [JCA Resource Adapter Example](jca_adapter_example.html#concept_swv_z2p_wk) for an example of how to set up a transaction using the JCA Resource Adapter.
 
-## <a id="concept_8567sdkbigige" class="no-quick-link"></a>Using Geode as the JTA Transaction Manager
+## <a id="concept_8567sdkbigige" class="no-quick-link"></a>Using <%=vars.product_name%> as the JTA Transaction Manager
 
-You can also use Geode as the JTA transaction manager.
-As of Geode 1.2, Geode's JTA transaction manager is deprecated.
+You can also use <%=vars.product_name%> as the JTA transaction manager.
+As of <%=vars.product_name%> 1.2, <%=vars.product_name%>'s JTA transaction manager is deprecated.
 
-Geode ships with its own implementation of a JTA transaction manager. However, note that this implementation is not XA-compliant; therefore, it does not persist any state, which could lead to an inconsistent state after recovering a crashed member.
+<%=vars.product_name%> ships with its own implementation of a JTA transaction manager. However, note that this implementation is not XA-compliant; therefore, it does not persist any state, which could lead to an inconsistent state after recovering a crashed member.
 
 <img src="../../images/transactions_jta.png" id="concept_8567sdkbigige__image_C8D94070E55F4BCC8B5FF3D5BEBA99ED" class="image" />
 
-The Geode JTA transaction manager is initialized when the Geode cache is initialized. Until then, JTA is not available for use. The application starts a JTA transaction by using the `UserTransaction.begin` method. The `UserTransaction` object is the application’s handle to instruct the JTA transaction manager on what to do.
+The <%=vars.product_name%> JTA transaction manager is initialized when the <%=vars.product_name%> cache is initialized. Until then, JTA is not available for use. The application starts a JTA transaction by using the `UserTransaction.begin` method. The `UserTransaction` object is the application’s handle to instruct the JTA transaction manager on what to do.
 
-The Geode JTA implementation also supports the J2EE Connector Architecture (JCA) `ManagedConnectionFactory`.
+The <%=vars.product_name%> JTA implementation also supports the J2EE Connector Architecture (JCA) `ManagedConnectionFactory`.
 
-The Geode implementation of JTA has the following limitations:
+The <%=vars.product_name%> implementation of JTA has the following limitations:
 
 -   Only one JDBC database instance per transaction is allowed, although you can have multiple connections to that database.
 -   Multiple threads cannot participate in a transaction.
 -   Transaction recovery after a crash is not supported.
 
-In addition, JTA transactions are subject to the limitations of Geode cache transactions such as not being supported on regions with global scope. When a global transaction needs to access the Geode cache, JTA silently starts a Geode cache transaction.
+In addition, JTA transactions are subject to the limitations of <%=vars.product_name%> cache transactions such as not being supported on regions with global scope. When a global transaction needs to access the <%=vars.product_name%> cache, JTA silently starts a <%=vars.product_name%> cache transaction.
 
 <a id="task_qjv_khb_wk"></a>
 
-# How to Run a JTA Global Transaction Using Geode as the JTA Transaction Manager
+# How to Run a JTA Global Transaction Using <%=vars.product_name%> as the JTA Transaction Manager
 
-This topic describes how to run a JTA global transaction in Geode .
+This topic describes how to run a JTA global transaction in <%=vars.product_name%> .
 
 To run a global transaction, perform the following steps:
 
 1. Configure the external data sources in the `cache.xml` file. See [Configuring Database Connections Using JNDI](configuring_db_connections_using_JNDI.html#topic_A5E3A67C808D48C08E1F0DC167C5C494) for examples. 
 2. Include the JAR file for any data sources in your CLASSPATH. 
-3.  Configure Geode for any necessary transactional behavior. Enable `copy-on-read` for your cache and specify a transaction listener, if you need one. See [Setting Global Copy on Read](working_with_transactions.html#concept_vx2_gs4_5k) and [Configuring Transaction Plug-In Event Handlers](working_with_transactions.html#concept_ocw_vf1_wk) for details. 
+3.  Configure <%=vars.product_name%> for any necessary transactional behavior. Enable `copy-on-read` for your cache and specify a transaction listener, if you need one. See [Setting Global Copy on Read](working_with_transactions.html#concept_vx2_gs4_5k) and [Configuring Transaction Plug-In Event Handlers](working_with_transactions.html#concept_ocw_vf1_wk) for details. 
 4.  Make sure that JTA transactions are not disabled in the `cache.xml` file or the application code. 
-5.  Initialize the Geode cache. 
+5.  Initialize the <%=vars.product_name%> cache. 
 6.  Get an initial context through `org.apache.geode.cache.GemFireCache.getJNDIContext`. For example: 
 
     ``` pre
@@ -236,11 +234,11 @@ To run a global transaction, perform the following steps:
     ```
 
     With `UserTransaction`, you can begin, commit, and rollback transactions.
-    If a global transaction exists when you use the cache, it automatically joins the transaction. Operations on a region automatically detect and become associated with the existing global transaction through JTA synchronization. If the global transaction has been marked for rollback, however, the Geode cache is not allowed to enlist with that transaction. Any cache operation that causes an attempt to enlist throws a `FailedSynchronizationException`.
+    If a global transaction exists when you use the cache, it automatically joins the transaction. Operations on a region automatically detect and become associated with the existing global transaction through JTA synchronization. If the global transaction has been marked for rollback, however, the <%=vars.product_name%> cache is not allowed to enlist with that transaction. Any cache operation that causes an attempt to enlist throws a `FailedSynchronizationException`.
 
-    The Geode cache transaction’s commit or rollback is triggered when the global transaction commits or rolls back. When the global transaction is committed using the `UserTransaction` interface, the transactions of any registered JTA resources are committed, including the Geode cache transaction. If the cache or database transaction fails to commit, the `UserTransaction` call throws a `TransactionRolledBackException`. If a commit or rollback is attempted directly on a Geode transaction that is registered with JTA, that action throws an `IllegalStateException`.
+    The <%=vars.product_name%> cache transaction’s commit or rollback is triggered when the global transaction commits or rolls back. When the global transaction is committed using the `UserTransaction` interface, the transactions of any registered JTA resources are committed, including the <%=vars.product_name%> cache transaction. If the cache or database transaction fails to commit, the `UserTransaction` call throws a `TransactionRolledBackException`. If a commit or rollback is attempted directly on a <%=vars.product_name%> transaction that is registered with JTA, that action throws an `IllegalStateException`.
 
-See [Geode JTA Transaction Example](transaction_jta_gemfire_example.html#concept_ffg_sj5_1l).
+See [<%=vars.product_name%> JTA Transaction Example](transaction_jta_gemfire_example.html#concept_ffg_sj5_1l).
 
 -   **[Configuring Database Connections Using JNDI](configuring_db_connections_using_JNDI.html)**
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/about_transactions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/about_transactions.html.md.erb b/geode-docs/developing/transactions/about_transactions.html.md.erb
index c0e3261..bc9e371 100644
--- a/geode-docs/developing/transactions/about_transactions.html.md.erb
+++ b/geode-docs/developing/transactions/about_transactions.html.md.erb
@@ -22,26 +22,26 @@ limitations under the License.
 <a id="topic_jbt_2y4_wk"></a>
 
 
-This section covers the features of Geode transactions.
+This section covers the features of <%=vars.product_name%> transactions.
 
-Geode transactions provide the following features:
+<%=vars.product_name%> transactions provide the following features:
 
 -   Basic transaction properties: atomicity, consistency, isolation, and durability
--   Rollback and commit operations along with standard Geode cache operations
+-   Rollback and commit operations along with standard <%=vars.product_name%> cache operations
 -   Ability to suspend and resume transactions
 -   High concurrency and high performance
 -   Transaction statistics gathering and archiving
--   Compatibility with Java Transaction API (JTA) transactions, using either Geode JTA or a third-party implementation
--   Ability to use Geode as a “last resource” in JTA transactions with multiple data sources to guarantee transactional consistency
+-   Compatibility with Java Transaction API (JTA) transactions, using either <%=vars.product_name%> JTA or a third-party implementation
+-   Ability to use <%=vars.product_name%> as a “last resource” in JTA transactions with multiple data sources to guarantee transactional consistency
 
 ## Types of Transactions
 
-Geode supports two kinds of transactions: **Geode cache transactions** and **JTA global transactions**.
+<%=vars.product_name%> supports two kinds of transactions: **<%=vars.product_name%> cache transactions** and **JTA global transactions**.
 
-Geode cache transactions are used to group the execution of cache operations and to gain the control offered by transactional commit and rollback. Applications create cache transactions by using an instance of the Geode `CacheTransactionManager`. During a transaction, cache operations are performed and distributed through Geode as usual. See [Geode Cache Transactions](cache_transactions.html#topic_e15_mr3_5k) for details on Geode cache transactions and how these transactions work.
+<%=vars.product_name%> cache transactions are used to group the execution of cache operations and to gain the control offered by transactional commit and rollback. Applications create cache transactions by using an instance of the <%=vars.product_name%> `CacheTransactionManager`. During a transaction, cache operations are performed and distributed through <%=vars.product_name%> as usual. See [<%=vars.product_name%> Cache Transactions](cache_transactions.html#topic_e15_mr3_5k) for details on <%=vars.product_name%> cache transactions and how these transactions work.
 
-JTA global transactions allow you to use the standard JTA interface to coordinate Geode transactions with JDBC transactions. When performing JTA global transactions, you have the option of using Geode’s own implementation of JTA or a third party’s implementation (typically application servers such as WebLogic or JBoss) of JTA. In addition, some third party JTA implementations allow you to set Geode as a “last resource” to ensure transactional consistency across data sources in the event that Geode or another data source becomes unavailable. For global transactions, applications use `java:/UserTransaction` to start and terminate transactions while Geode cache operations are performed in the same manner as regular Geode cache transactions. See [JTA Global Transactions with Geode](JTA_transactions.html) for details on JTA Global transactions.
+JTA global transactions allow you to use the standard JTA interface to coordinate <%=vars.product_name%> transactions with JDBC transactions. When performing JTA global transactions, you have the option of using <%=vars.product_name%>’s own implementation of JTA or a third party’s implementation (typically application servers such as WebLogic or JBoss) of JTA. In addition, some third party JTA implementations allow you to set <%=vars.product_name%> as a “last resource” to ensure transactional consistency across data sources in the event that <%=vars.product_name%> or another data source becomes unavailable. For global transactions, applications use `java:/UserTransaction` to start and terminate transactions while <%=vars.product_name%> cache operations are performed in the same manner as regular <%=vars.product_name%> cache transactions. See [JTA Global Transactions with <%=vars.product_name%>](JTA_transactions.html) for details on JTA Global transactions.
 
-You can also coordinate a Geode cache transaction with an external database by specifying database operations within cache and transaction application plug-ins (CacheWriters/CacheListeners and TransactionWriters/TransactionListeners.) This is an alternative to using JTA transactions. See [How to Run a Geode Cache Transaction that Coordinates with an External Database](run_a_cache_transaction_with_external_db.html#task_sdn_2qk_2l).
+You can also coordinate a <%=vars.product_name%> cache transaction with an external database by specifying database operations within cache and transaction application plug-ins (CacheWriters/CacheListeners and TransactionWriters/TransactionListeners.) This is an alternative to using JTA transactions. See [How to Run a <%=vars.product_name%> Cache Transaction that Coordinates with an External Database](run_a_cache_transaction_with_external_db.html#task_sdn_2qk_2l).
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/cache_plugins_with_jta.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/cache_plugins_with_jta.html.md.erb b/geode-docs/developing/transactions/cache_plugins_with_jta.html.md.erb
index 7735bf0..7df64bc 100644
--- a/geode-docs/developing/transactions/cache_plugins_with_jta.html.md.erb
+++ b/geode-docs/developing/transactions/cache_plugins_with_jta.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Behavior of Geode Cache Writers and Loaders Under JTA
----
+<% set_title("Behavior of", product_name, "Cache Writers and Loaders Under JTA") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,10 +17,10 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-When Geode participates in a global transactions, you can still have Geode cache writers and cache loaders operating in the usual way.
+When <%=vars.product_name%> participates in a global transactions, you can still have <%=vars.product_name%> cache writers and cache loaders operating in the usual way.
 
 For example, in addition to the transactional connection to the database, the region could also have a cache writer and cache loader configured to exchange data with that same database. As long as the data source is transactional, which means that it can detect the transaction manager, the cache writer and cache loader participate in the transaction. If the JTA rolls back its transaction, the changes made by the cache loader and the cache writer are rolled back. For more on transactional data sources, see the discussion of XAPooledDataSource and ManagedDataSource in[Configuring Database Connections Using JNDI](configuring_db_connections_using_JNDI.html#topic_A5E3A67C808D48C08E1F0DC167C5C494).
 
-If you are using a Geode cache or transaction listener with global transactions, be aware that the EntryEvent returned by a transaction has the Geode transaction ID, not the JTA transaction ID.
+If you are using a <%=vars.product_name%> cache or transaction listener with global transactions, be aware that the EntryEvent returned by a transaction has the <%=vars.product_name%> transaction ID, not the JTA transaction ID.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/cache_transaction_performance.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/cache_transaction_performance.html.md.erb b/geode-docs/developing/transactions/cache_transaction_performance.html.md.erb
index 5f25453..97134bc 100644
--- a/geode-docs/developing/transactions/cache_transaction_performance.html.md.erb
+++ b/geode-docs/developing/transactions/cache_transaction_performance.html.md.erb
@@ -23,7 +23,7 @@ Cache transaction performance can vary depending on the type of regions you are
 
 The most common region configurations for use with transactions are distributed replicated and partitioned:
 
--   Replicated regions are better suited for running transactions on small to mid-size data sets. To ensure all or nothing behavior, at commit time, distributed transactions use the global reservation system of the Geode distributed lock service. This works well as long as the data set is reasonably small.
+-   Replicated regions are better suited for running transactions on small to mid-size data sets. To ensure all or nothing behavior, at commit time, distributed transactions use the global reservation system of the <%=vars.product_name%> distributed lock service. This works well as long as the data set is reasonably small.
 -   Partitioned regions are the right choice for highly-performant, scalable operations. Transactions on partitioned regions use only local locking, and only send messages to the redundant data stores at commit time. Because of this, these transactions perform much better than distributed transactions. There are no global locks, so partitioned transactions are extremely scalable as well.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/cache_transactions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/cache_transactions.html.md.erb b/geode-docs/developing/transactions/cache_transactions.html.md.erb
index 7e00e42..8b5d0f6 100644
--- a/geode-docs/developing/transactions/cache_transactions.html.md.erb
+++ b/geode-docs/developing/transactions/cache_transactions.html.md.erb
@@ -1,6 +1,4 @@
----
-title: Geode Cache Transactions
----
+<% set_title(product_name, "Cache Transactions") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -22,30 +20,30 @@ limitations under the License.
 <a id="topic_e15_mr3_5k"></a>
 
 
-Use Geode cache transactions to group the execution of cache operations and to gain the control offered by transactional commit and rollback. Geode cache transactions control operations within the Geode cache while the Geode distributed system handles data distribution in the usual way.
+Use <%=vars.product_name%> cache transactions to group the execution of cache operations and to gain the control offered by transactional commit and rollback. <%=vars.product_name%> cache transactions control operations within the <%=vars.product_name%> cache while the <%=vars.product_name%> distributed system handles data distribution in the usual way.
 
--   **[Cache Transaction Performance](../../developing/transactions/cache_transaction_performance.html)**
+-   **[Cache Transaction Performance](cache_transaction_performance.html)**
 
     Cache transaction performance can vary depending on the type of regions you are using.
 
--   **[Data Location for Cache Transactions](../../developing/transactions/data_location_cache_transactions.html)**
+-   **[Data Location for Cache Transactions](data_location_cache_transactions.html)**
 
     The location where you can run your transaction depends on where you are storing your data.
 
--   **[How to Run a Geode Cache Transaction](../../developing/transactions/run_a_cache_transaction.html)**
+-   **[How to Run a <%=vars.product_name%> Cache Transaction](run_a_cache_transaction.html)**
 
-    This topic describes how to run a Geode cache transaction.
+    This topic describes how to run a <%=vars.product_name%> cache transaction.
 
--   **[How to Run a Geode Cache Transaction that Coordinates with an External Database](../../developing/transactions/run_a_cache_transaction_with_external_db.html)**
+-   **[How to Run a <%=vars.product_name%> Cache Transaction that Coordinates with an External Database](run_a_cache_transaction_with_external_db.html)**
 
-    Coordinate a Geode cache transaction with an external database by using CacheWriter/CacheListener and TransactionWriter/TransactionListener plug-ins, **to provide an alternative to using JTA transactions**.
+    Coordinate a <%=vars.product_name%> cache transaction with an external database by using CacheWriter/CacheListener and TransactionWriter/TransactionListener plug-ins, **to provide an alternative to using JTA transactions**.
 
--   **[Working with Geode Cache Transactions](../../developing/transactions/working_with_transactions.html)**
+-   **[Working with <%=vars.product_name%> Cache Transactions](working_with_transactions.html)**
 
-    This section contains guidelines and additional information on working with Geode and its cache transactions.
+    This section contains guidelines and additional information on working with <%=vars.product_name%> and its cache transactions.
 
--   **[How Geode Cache Transactions Work](../../developing/transactions/how_cache_transactions_work.html#topic_fls_1j1_wk)**
+-   **[How <%=vars.product_name%> Cache Transactions Work](how_cache_transactions_work.html#topic_fls_1j1_wk)**
 
-    This section provides an explanation of how transactions work on Geode caches.
+    This section provides an explanation of how transactions work on <%=vars.product_name%> caches.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/cache_transactions_by_region_type.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/cache_transactions_by_region_type.html.md.erb b/geode-docs/developing/transactions/cache_transactions_by_region_type.html.md.erb
index 550d755..7811bcb 100644
--- a/geode-docs/developing/transactions/cache_transactions_by_region_type.html.md.erb
+++ b/geode-docs/developing/transactions/cache_transactions_by_region_type.html.md.erb
@@ -21,7 +21,7 @@ limitations under the License.
 <a id="topic_nlq_sk1_wk"></a>
 
 
-A transaction is managed on a per-cache basis, so multiple regions in the cache can participate in a single transaction. The data scope of a Geode cache transaction is the cache that hosts the transactional data. For partitioned regions, this may be a remote host to the one running the transaction application. Any transaction that includes one or more partitioned regions is run on the member storing the primary copy of the partitioned region data. Otherwise, the transaction host is the same one running the application.
+A transaction is managed on a per-cache basis, so multiple regions in the cache can participate in a single transaction. The data scope of a <%=vars.product_name%> cache transaction is the cache that hosts the transactional data. For partitioned regions, this may be a remote host to the one running the transaction application. Any transaction that includes one or more partitioned regions is run on the member storing the primary copy of the partitioned region data. Otherwise, the transaction host is the same one running the application.
 
 -   The client executing the transaction code is called the transaction initiator.
 
@@ -83,7 +83,7 @@ The region’s scope affects how data is distributed during the commit phase. Tr
 Transactions on non-replicated regions (regions that use the old API with DataPolicy EMPTY, NORMAL and PRELOADED) are always transaction initiators, and the transaction data host is always a member with a replicated region. This is similar to the way transactions using the PARTITION\_PROXY shortcut are forwarded to members with primary bucket.
 
 **Note:**
-When you have transactions operating on EMPTY, NORMAL or PARTITION regions, make sure that the Geode property `conserve-sockets` is set to false to avoid distributed deadlocks. An empty region is a region created with the API `RegionShortcut.REPLICATE_PROXY` or a region with that uses the old API of `DataPolicy` set to `EMPTY`.
+When you have transactions operating on EMPTY, NORMAL or PARTITION regions, make sure that the <%=vars.product_name%> property `conserve-sockets` is set to false to avoid distributed deadlocks. An empty region is a region created with the API `RegionShortcut.REPLICATE_PROXY` or a region with that uses the old API of `DataPolicy` set to `EMPTY`.
 
 ## Conflicting Transactions in Distributed-Ack Regions
 
@@ -97,7 +97,7 @@ In this series of figures, even after the commit operation is launched, the tran
 
 <img src="../../images_svg/transactions_replicate_2.svg" id="concept_nl5_pk1_wk__image_sbh_21k_54" class="image" />
 
-**Step 3:** Changes are in transit. T1 commits and its changes are merged into the local cache. The commit does not complete until Geode distributes the changes to the remote regions and acknowledgment is received.
+**Step 3:** Changes are in transit. T1 commits and its changes are merged into the local cache. The commit does not complete until <%=vars.product_name%> distributes the changes to the remote regions and acknowledgment is received.
 
 <img src="../../images_svg/transactions_replicate_3.svg" id="concept_nl5_pk1_wk__image_qgl_k1k_54" class="image" />
 
@@ -113,7 +113,7 @@ These figures show how using the no-ack scope can produce unexpected results. Th
 
 <img src="../../images_svg/transactions_replicate_1.svg" id="concept_nl5_pk1_wk__image_jn2_cbk_54" class="image" />
 
-**Step 2:** Changes are in transit. Transactions T1 and T2 commit and merge their changes into the local cache. Geode then distributes changes to the remote regions.
+**Step 2:** Changes are in transit. Transactions T1 and T2 commit and merge their changes into the local cache. <%=vars.product_name%> then distributes changes to the remote regions.
 
 <img src="../../images_svg/transactions_replicate_no_ack_1.svg" id="concept_nl5_pk1_wk__image_fk1_hbk_54" class="image" />
 
@@ -129,18 +129,18 @@ When encountering conflicts with local scope, the first transaction to start the
 ## Transactions and Persistent Regions
 <a id="concept_omy_341_wk"></a>
 
-By default, Geode does not allow transactions on persistent regions. You can enable the use of transactions on persistent regions by setting the property `gemfire.ALLOW_PERSISTENT_TRANSACTIONS` to true. This may also be accomplished at server startup using gfsh:
+By default, <%=vars.product_name%> does not allow transactions on persistent regions. You can enable the use of transactions on persistent regions by setting the property `gemfire.ALLOW_PERSISTENT_TRANSACTIONS` to true. This may also be accomplished at server startup using gfsh:
 
 ``` pre
 gfsh start server --name=server1 --dir=server1_dir \
 --J=-Dgemfire.ALLOW_PERSISTENT_TRANSACTIONS=true 
 ```
 
-Since Geode does not provide atomic disk persistence guarantees, the default behavior is to disallow disk-persistent regions from participating in transactions. However, when choosing to enable transactions on persistent regions, consider the following:
+Since <%=vars.product_name%> does not provide atomic disk persistence guarantees, the default behavior is to disallow disk-persistent regions from participating in transactions. However, when choosing to enable transactions on persistent regions, consider the following:
 
--   Geode does ensure atomicity for in-memory updates.
+-   <%=vars.product_name%> does ensure atomicity for in-memory updates.
 -   When any failed member is unable to complete the logic triggered by a transaction (including subsequent disk writes), that failed member is removed from the distributed system and, if restarted, must rebuild its state from surviving nodes that successfully complete the updates.
--   The chances of multiple nodes failing to complete the disk writes that result from a transaction commit due to nodes crashing for unrelated reasons are small. The real risk is that the file system buffers holding the persistent updates do not get written to disk in the case of operating system or hardware failure. If only the Geode process crashes, atomicity still exists. The overall risk of losing disk updates can also be mitigated by enabling synchronized disk file mode for the disk stores, but this incurs a high performance penalty.
+-   The chances of multiple nodes failing to complete the disk writes that result from a transaction commit due to nodes crashing for unrelated reasons are small. The real risk is that the file system buffers holding the persistent updates do not get written to disk in the case of operating system or hardware failure. If only the <%=vars.product_name%> process crashes, atomicity still exists. The overall risk of losing disk updates can also be mitigated by enabling synchronized disk file mode for the disk stores, but this incurs a high performance penalty.
 
 To mitigate the risk of data not get fully written to disk on all copies of the participating persistent disk stores:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/chapter_overview.html.md.erb b/geode-docs/developing/transactions/chapter_overview.html.md.erb
index defcf4b..0f2dc37 100644
--- a/geode-docs/developing/transactions/chapter_overview.html.md.erb
+++ b/geode-docs/developing/transactions/chapter_overview.html.md.erb
@@ -19,27 +19,27 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode provides a transactions API, with `begin`, `commit`, and `rollback` methods. These methods are much the same as the familiar relational database transactions methods.
+<%=vars.product_name%> provides a transactions API, with `begin`, `commit`, and `rollback` methods. These methods are much the same as the familiar relational database transactions methods.
 
--   **[About Transactions](../../developing/transactions/about_transactions.html)**
+-   **[About Transactions](about_transactions.html)**
 
-    This section covers the features of Geode transactions.
-It also details the two kinds of transaction that Geode supports:
-**Geode cache transactions** and **JTA global transactions**.
+    This section covers the features of <%=vars.product_name%> transactions.
+It also details the two kinds of transaction that <%=vars.product_name%> supports:
+**<%=vars.product_name%> cache transactions** and **JTA global transactions**.
 
--   **[Geode Cache Transactions](../../developing/transactions/cache_transactions.html)**
+-   **[<%=vars.product_name%> Cache Transactions](cache_transactions.html)**
 
-    Use Geode cache transactions to group the execution of cache operations and to gain the control offered by transactional commit and rollback. Geode cache transactions control operations within the Geode cache while the Geode distributed system handles data distribution in the usual way.
+    Use <%=vars.product_name%> cache transactions to group the execution of cache operations and to gain the control offered by transactional commit and rollback. <%=vars.product_name%> cache transactions control operations within the <%=vars.product_name%> cache while the <%=vars.product_name%> distributed system handles data distribution in the usual way.
 
--   **[JTA Global Transactions with Geode](../../developing/transactions/JTA_transactions.html)**
+-   **[JTA Global Transactions with <%=vars.product_name%>](JTA_transactions.html)**
 
-    Use JTA global transactions to coordinate Geode cache transactions and JDBC transactions.
+    Use JTA global transactions to coordinate <%=vars.product_name%> cache transactions and JDBC transactions.
 
--   **[Monitoring and Troubleshooting Transactions](../../developing/transactions/monitor_troubleshoot_transactions.html)**
+-   **[Monitoring and Troubleshooting Transactions](monitor_troubleshoot_transactions.html)**
 
-    This topic covers errors that may occur when running transactions in Geode.
+    This topic covers errors that may occur when running transactions in <%=vars.product_name%>.
 
--   **[Transaction Coding Examples](../../developing/transactions/transaction_coding_examples.html)**
+-   **[Transaction Coding Examples](transaction_coding_examples.html)**
 
     This section provides several code examples for writing and executing transactions.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/client_server_transactions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/client_server_transactions.html.md.erb b/geode-docs/developing/transactions/client_server_transactions.html.md.erb
index 727683a..4781bd7f 100644
--- a/geode-docs/developing/transactions/client_server_transactions.html.md.erb
+++ b/geode-docs/developing/transactions/client_server_transactions.html.md.erb
@@ -20,19 +20,19 @@ limitations under the License.
 -->
 
 
-The syntax for writing client transactions is the same on the Java client as with any other Geode member, but the underlying behavior in a client-run transaction is different from general transaction behavior.
+The syntax for writing client transactions is the same on the Java client as with any other <%=vars.product_name%> member, but the underlying behavior in a client-run transaction is different from general transaction behavior.
 
-For general information about running a transaction, refer to [How to Run a Geode Cache Transaction](run_a_cache_transaction.html#task_f15_mr3_5k).
+For general information about running a transaction, refer to [How to Run a <%=vars.product_name%> Cache Transaction](run_a_cache_transaction.html#task_f15_mr3_5k).
 
--   **[How Geode Runs Client Transactions](../../developing/transactions/client_server_transactions.html#how_gemfire_runs_clients)**
+-   **[How <%=vars.product_name%> Runs Client Transactions](client_server_transactions.html#how_gemfire_runs_clients)**
 
--   **[Client Cache Access During a Transaction](../../developing/transactions/client_server_transactions.html#client_cache_access)**
+-   **[Client Cache Access During a Transaction](client_server_transactions.html#client_cache_access)**
 
--   **[Client Transactions and Client Application Plug-Ins](../../developing/transactions/client_server_transactions.html#client_app_plugins)**
+-   **[Client Transactions and Client Application Plug-Ins](client_server_transactions.html#client_app_plugins)**
 
--   **[Client Transaction Failures](../../developing/transactions/client_server_transactions.html#client_transaction_failures)**
+-   **[Client Transaction Failures](client_server_transactions.html#client_transaction_failures)**
 
-## <a id="how_gemfire_runs_clients" class="no-quick-link"></a>How Geode Runs Client Transactions
+## <a id="how_gemfire_runs_clients" class="no-quick-link"></a>How <%=vars.product_name%> Runs Client Transactions
 
 When a client performs a transaction, the transaction is delegated to a server that acts as the transaction initiator in the server system. As with regular, non-client transactions, this server delegate may or may not be the transaction host.
 
@@ -42,7 +42,7 @@ In this figure, the application code on the client makes changes to data entries
 
 ## <a id="client_cache_access" class="no-quick-link"></a>Client Cache Access During a Transaction
 
-To maintain cache consistency, Geode blocks access to the local client cache during a transaction. The local client cache may reflect information inconsistent with the transaction in progress. When the transaction completes, the local cache is accessible again.
+To maintain cache consistency, <%=vars.product_name%> blocks access to the local client cache during a transaction. The local client cache may reflect information inconsistent with the transaction in progress. When the transaction completes, the local cache is accessible again.
 
 ## <a id="client_app_plugins" class="no-quick-link"></a>Client Transactions and Client Application Plug-Ins
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/configuring_db_connections_using_JNDI.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/configuring_db_connections_using_JNDI.html.md.erb b/geode-docs/developing/transactions/configuring_db_connections_using_JNDI.html.md.erb
index 4a08b6a..f58d04e 100644
--- a/geode-docs/developing/transactions/configuring_db_connections_using_JNDI.html.md.erb
+++ b/geode-docs/developing/transactions/configuring_db_connections_using_JNDI.html.md.erb
@@ -31,9 +31,9 @@ The following are a list of `DataSource` connection types used in JTA transactio
 -   **PooledDataSource**. Pooled SQL connections.
 -   **SimpleDataSource**. Single SQL connection. No pooling of SQL connections is done. Connections are generated on the fly and cannot be reused.
 
-The `jndi-name` attribute of the `jndi-binding` element is the key binding parameter. If the value of `jndi-name` is a DataSource, it is bound as `java:/`*myDatabase*, where *myDatabase* is the name you assign to your data source. If the data source cannot be bound to JNDI at runtime, Geode logs a warning. For information on the `DataSource` interface, see: [http://docs.oracle.com/javase/8/docs/api/javax/sql/DataSource.html](http://docs.oracle.com/javase/8/docs/api/javax/sql/DataSource.html)
+The `jndi-name` attribute of the `jndi-binding` element is the key binding parameter. If the value of `jndi-name` is a DataSource, it is bound as `java:/`*myDatabase*, where *myDatabase* is the name you assign to your data source. If the data source cannot be bound to JNDI at runtime, <%=vars.product_name%> logs a warning. For information on the `DataSource` interface, see: [http://docs.oracle.com/javase/8/docs/api/javax/sql/DataSource.html](http://docs.oracle.com/javase/8/docs/api/javax/sql/DataSource.html)
 
-Geode supports JDBC 2.0 and 3.0.
+<%=vars.product_name%> supports JDBC 2.0 and 3.0.
 
 **Note:**
 Include any data source JAR files in your CLASSPATH.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/how_cache_transactions_work.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/how_cache_transactions_work.html.md.erb b/geode-docs/developing/transactions/how_cache_transactions_work.html.md.erb
index 4cb0473..c7bca5b 100644
--- a/geode-docs/developing/transactions/how_cache_transactions_work.html.md.erb
+++ b/geode-docs/developing/transactions/how_cache_transactions_work.html.md.erb
@@ -1,6 +1,4 @@
----
-title: How Geode Cache Transactions Work
----
+<% set_title("How", product_name, "Cache Transactions Work") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,21 +19,21 @@ limitations under the License.
 <a id="topic_fls_1j1_wk"></a>
 
 
-This section provides an explanation of how transactions work on Geode caches.
+This section provides an explanation of how transactions work on <%=vars.product_name%> caches.
 
-All the regions in a Geode member cache can participate in a transaction. A Java application can operate on the cache using multiple transactions. A transaction is associated with only one thread, and a thread can operate on only one transaction at a time. Child threads do not inherit existing transactions.
+All the regions in a <%=vars.product_name%> member cache can participate in a transaction. A Java application can operate on the cache using multiple transactions. A transaction is associated with only one thread, and a thread can operate on only one transaction at a time. Child threads do not inherit existing transactions.
 
--   **[Transaction View](../../developing/transactions/how_cache_transactions_work.html#concept_hls_1j1_wk)**
+-   **[Transaction View](#concept_hls_1j1_wk)**
 
--   **[Committing Transactions](../../developing/transactions/how_cache_transactions_work.html#concept_sbj_lj1_wk)**
+-   **[Committing Transactions](#concept_sbj_lj1_wk)**
 
--   **[Transactions by Region Type](../../developing/transactions/cache_transactions_by_region_type.html#topic_nlq_sk1_wk)**
+-   **[Transactions by Region Type](cache_transactions_by_region_type.html#topic_nlq_sk1_wk)**
 
--   **[Client Transactions](../../developing/transactions/client_server_transactions.html)**
+-   **[Client Transactions](client_server_transactions.html)**
 
--   **[Comparing Transactional and Non-Transactional Operations](../../developing/transactions/transactional_and_nontransactional_ops.html#transactional_and_nontransactional_ops)**
+-   **[Comparing Transactional and Non-Transactional Operations](transactional_and_nontransactional_ops.html#transactional_and_nontransactional_ops)**
 
--   **[Geode Cache Transaction Semantics](../../developing/transactions/transaction_semantics.html)**
+-   **[<%=vars.product_name%> Cache Transaction Semantics](transaction_semantics.html)**
 
 ## Transaction View
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/jca_adapter_example.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/jca_adapter_example.html.md.erb b/geode-docs/developing/transactions/jca_adapter_example.html.md.erb
index 409b93e..1c7b420 100644
--- a/geode-docs/developing/transactions/jca_adapter_example.html.md.erb
+++ b/geode-docs/developing/transactions/jca_adapter_example.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-This example shows how to use the JCA Resource Adapter in Geode .
+This example shows how to use the JCA Resource Adapter in <%=vars.product_name%> .
 
 ``` pre
 Hashtable env = new Hashtable();

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/monitor_troubleshoot_transactions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/monitor_troubleshoot_transactions.html.md.erb b/geode-docs/developing/transactions/monitor_troubleshoot_transactions.html.md.erb
index 7956cac..b2ba4df 100644
--- a/geode-docs/developing/transactions/monitor_troubleshoot_transactions.html.md.erb
+++ b/geode-docs/developing/transactions/monitor_troubleshoot_transactions.html.md.erb
@@ -19,14 +19,14 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-This topic covers errors that may occur when running transactions in Geode.
+This topic covers errors that may occur when running transactions in <%=vars.product_name%>.
 
 <a id="monitor_troubleshoot_transactions__section_881D2FF6761B4D689DDB46C650E2A2E1"></a>
-Unlike database transactions, Geode does not write a transaction log to disk. To get the full details about committed operations, use a transaction listener to monitor the transaction events and their contained cache events for each of your transactions.
+Unlike database transactions, <%=vars.product_name%> does not write a transaction log to disk. To get the full details about committed operations, use a transaction listener to monitor the transaction events and their contained cache events for each of your transactions.
 
 ## <a id="monitor_troubleshoot_transactions__section_2B66338C851A4FF386B60CC5CF4DCF77" class="no-quick-link"></a>Statistics on Cache Transactions
 
-During the operation of Geode cache transactions, if statistics are enabled, transaction-related statistics are calculated and accessible from the CachePerfStats statistic resource. Because the transaction’s data scope is the cache, these statistics are collected on a per-cache basis.
+During the operation of <%=vars.product_name%> cache transactions, if statistics are enabled, transaction-related statistics are calculated and accessible from the CachePerfStats statistic resource. Because the transaction’s data scope is the cache, these statistics are collected on a per-cache basis.
 
 ## <a id="monitor_troubleshoot_transactions__section_EA9277E6CFD7423F95BA4D04955FDE2A" class="no-quick-link"></a>Commit
 
@@ -38,7 +38,7 @@ A transaction can create data beyond the capacity limit set in the region’s ev
 
 ## <a id="monitor_troubleshoot_transactions__section_C7588E4F143B4D7FAFAEDCF5AE4FF2C8" class="no-quick-link"></a>Interaction with the Resource Manager
 
-The Geode resource manager, which controls overall heap use, either allows all transactional operations or blocks the entire transaction. If a cache reaches the critical threshold in the middle of a commit, the commit is allowed to finish before the manager starts blocking operations.
+The <%=vars.product_name%> resource manager, which controls overall heap use, either allows all transactional operations or blocks the entire transaction. If a cache reaches the critical threshold in the middle of a commit, the commit is allowed to finish before the manager starts blocking operations.
 
 ## <a id="monitor_troubleshoot_transactions__section_8942ABA6F23C4ED58877C894B13F4F21" class="no-quick-link"></a>Transaction Exceptions
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/run_a_cache_transaction.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/run_a_cache_transaction.html.md.erb b/geode-docs/developing/transactions/run_a_cache_transaction.html.md.erb
index 7ec2be6..90b1183 100644
--- a/geode-docs/developing/transactions/run_a_cache_transaction.html.md.erb
+++ b/geode-docs/developing/transactions/run_a_cache_transaction.html.md.erb
@@ -1,6 +1,4 @@
----
-title: How to Run a Geode Cache Transaction
----
+<% set_title("How to Run a", product_name, "Cache Transaction") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,9 +19,9 @@ limitations under the License.
 <a id="task_f15_mr3_5k"></a>
 
 
-This topic describes how to run a Geode cache transaction.
+This topic describes how to run a <%=vars.product_name%> cache transaction.
 
-Applications manage transactions on a per-cache basis. A Geode cache transaction starts with a `CacheTransactionManager.begin` method and continues with a series of operations, which are typically region operations such as region create, update, clear and destroy. The begin, commit, and rollback are directly controlled by the application. A commit, failed commit, or voluntary rollback by the transaction manager ends the transaction.
+Applications manage transactions on a per-cache basis. A <%=vars.product_name%> cache transaction starts with a `CacheTransactionManager.begin` method and continues with a series of operations, which are typically region operations such as region create, update, clear and destroy. The begin, commit, and rollback are directly controlled by the application. A commit, failed commit, or voluntary rollback by the transaction manager ends the transaction.
 
 You can run transactions on any type of cache region except regions with **global** scope. An operation attempted on a region with global scope throws an `UnsupportedOperationException` exception.
 
@@ -38,10 +36,10 @@ This discussion centers on transactions on replicated and partitioned regions. I
     |---------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
     | **replicated regions**                                                                      | Use `distributed-ack` scope. The region shortcuts specifying `REPLICATE` use `distributed-ack` scope. This is particularly important if you have more than one data producer. With one data producer, you can safely use `distributed-no-ack`.                                                                                                                                                                        |
     | **partitioned regions**                                                                     | Custom partition and colocate data among regions so all the data for any single transaction is hosted by a single member. If the transaction is run from a member other than the one hosting the data, the transaction will run by proxy in the member hosting the data. The partitioned region must be defined for the application that runs the transaction, but the data can be hosted in a remote member. |
-    | **persistent regions**                                                                      | Configure Geode to allow transactions on persistent regions. By default, the configuration does not allow transactions on persistent regions. Enable the use of transactions on persistent regions by setting the property `gemfire.ALLOW_PERSISTENT_TRANSACTIONS` to true.                                                                                              |
+    | **persistent regions**                                                                      | Configure <%=vars.product_name%> to allow transactions on persistent regions. By default, the configuration does not allow transactions on persistent regions. Enable the use of transactions on persistent regions by setting the property `gemfire.ALLOW_PERSISTENT_TRANSACTIONS` to true.                                                                                              |
     | **a mix of partitioned and replicated regions**                                             | Make sure any replicated region involved in the transaction is hosted on every member that hosts the partitioned region data. All data for a single transaction must reside within a single host.                                                                                                                                                                                                             |
-    | **delta propagation**                                                                       | Set the region attribute `cloning-enabled` to true. This lets Geode do conflict checks at commit time. Without this, the transaction will throw an `UnsupportedOperationInTransactionException ` exception.                                                                                                                                                                      |
-    | **global JTA transactions with only Geode cache transactions** | Set the region attribute `ignore-jta` to true for all regions that you do *not* want to participate in JTA global transactions. It is false by default. For instructions on how to run a JTA global transaction, see [JTA Global Transactions with Geode](JTA_transactions.html).   |
+    | **delta propagation**                                                                       | Set the region attribute `cloning-enabled` to true. This lets <%=vars.product_name%> do conflict checks at commit time. Without this, the transaction will throw an `UnsupportedOperationInTransactionException ` exception.                                                                                                                                                                      |
+    | **global JTA transactions with only <%=vars.product_name%> cache transactions** | Set the region attribute `ignore-jta` to true for all regions that you do *not* want to participate in JTA global transactions. It is false by default. For instructions on how to run a JTA global transaction, see [JTA Global Transactions with <%=vars.product_name%>](JTA_transactions.html).   |
 
 3. **Update your cache event handler and transaction event handler implementations to handle your transactions.** 
     Cache event handlers may be used with transactions. Cache listeners are called after the commit, instead of after each cache operation, and the cache listeners receive conflated transaction events. Cache writers and loaders are called as usual, at the time of the operation.
@@ -85,6 +83,6 @@ This discussion centers on transactions on replicated and partitioned regions. I
 5. **Review all of your code for compatibility with transactions.** 
     When you commit a transaction, while the commit is in process, the changes are visible in the distributed cache. This provides better performance than locking everything involved with the transaction updates, but it means that another process accessing data used in the transaction might get some data in the pre-transaction state and some in the post-transaction state.
 
-    For example, suppose keys 1 and 2 are modified within a transaction, such that both values change from A to B. In another thread, it is possible to read key 1 with value B and key 2 with value A, after the commit begins, but before the commit completes. This is possible due to the nature of Geode reads. This choice sacrifices atomic visibility in favor of performance; reads do not block writes, and writes do not block reads.
+    For example, suppose keys 1 and 2 are modified within a transaction, such that both values change from A to B. In another thread, it is possible to read key 1 with value B and key 2 with value A, after the commit begins, but before the commit completes. This is possible due to the nature of <%=vars.product_name%> reads. This choice sacrifices atomic visibility in favor of performance; reads do not block writes, and writes do not block reads.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/run_a_cache_transaction_with_external_db.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/run_a_cache_transaction_with_external_db.html.md.erb b/geode-docs/developing/transactions/run_a_cache_transaction_with_external_db.html.md.erb
index 16a1397..40cf1a1 100644
--- a/geode-docs/developing/transactions/run_a_cache_transaction_with_external_db.html.md.erb
+++ b/geode-docs/developing/transactions/run_a_cache_transaction_with_external_db.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  How to Run a Geode Cache Transaction that Coordinates with an External Database
----
+<% set_title("How to Run a", product_name, "Cache Transaction that Coordinates with an External Database") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,9 +17,9 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Coordinate a Geode cache transaction with an external database by using CacheWriter/CacheListener and TransactionWriter/TransactionListener plug-ins, **to provide an alternative to using JTA transactions**.
+Coordinate a <%=vars.product_name%> cache transaction with an external database by using CacheWriter/CacheListener and TransactionWriter/TransactionListener plug-ins, **to provide an alternative to using JTA transactions**.
 
-There are a few things you should be careful about while working with Geode cache transactions and external databases:
+There are a few things you should be careful about while working with <%=vars.product_name%> cache transactions and external databases:
 
 -   When you set up the JDBC connection, make sure that auto-commit is disabled. For example, in Java:
 
@@ -40,15 +38,15 @@ There are a few things you should be careful about while working with Geode cach
     max_prepared_transactions = 1 # 1 or more enables, zero (default) disables this feature.
     ```
 
-Use the following procedure to write a Geode cache transaction that coordinates with an external database:
+Use the following procedure to write a <%=vars.product_name%> cache transaction that coordinates with an external database:
 
-1.  Configure Geode regions as necessary as described in [How to Run a Geode Cache Transaction](run_a_cache_transaction.html#task_f15_mr3_5k).
+1.  Configure <%=vars.product_name%> regions as necessary as described in [How to Run a <%=vars.product_name%> Cache Transaction](run_a_cache_transaction.html#task_f15_mr3_5k).
 2.  Begin the transaction.
 3.  If you have not previously committed a previous transaction in this connection, start a database transaction by issuing a BEGIN statement.
-4.  Perform Geode cache operations; each cache operation invokes the CacheWriter. Implement the CacheWriter to perform the corresponding external database operations.
+4.  Perform <%=vars.product_name%> cache operations; each cache operation invokes the CacheWriter. Implement the CacheWriter to perform the corresponding external database operations.
 5.  Commit the transaction.
     At this point, the TransactionWriter is invoked. The TransactionWriter returns a TransactionEvent, which contains all the operations in the transaction. Call PREPARE TRANSACTION within your TransactionWriter code.
 
-6.  After a transaction is successfully committed in Geode, the TransactionListener is invoked. The TransactionListener calls COMMIT PREPARED to commit the database transaction.
+6.  After a transaction is successfully committed in <%=vars.product_name%>, the TransactionListener is invoked. The TransactionListener calls COMMIT PREPARED to commit the database transaction.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/transaction_coding_examples.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/transaction_coding_examples.html.md.erb b/geode-docs/developing/transactions/transaction_coding_examples.html.md.erb
index 26aac45..bb13deb 100644
--- a/geode-docs/developing/transactions/transaction_coding_examples.html.md.erb
+++ b/geode-docs/developing/transactions/transaction_coding_examples.html.md.erb
@@ -21,24 +21,24 @@ limitations under the License.
 
 This section provides several code examples for writing and executing transactions.
 
--   **[Basic Transaction Example](../../developing/transactions/transactions_overview.html)**
+-   **[Basic Transaction Example](transactions_overview.html)**
 
     This example operates on two replicated regions. It begins a transaction, updates one entry in each region, and commits the result.
 
--   **[Basic Suspend and Resume Transaction Example](../../developing/transactions/transaction_suspend_resume_example.html)**
+-   **[Basic Suspend and Resume Transaction Example](transaction_suspend_resume_example.html)**
 
     This example suspends and resumes a transaction.
 
--   **[Transaction Embedded within a Function Example](../../developing/transactions/transactional_function_example.html)**
+-   **[Transaction Embedded within a Function Example](transactional_function_example.html)**
 
     This example demonstrates a function that does transactional updates to Customer and Order regions.
 
--   **[Geode JTA Transaction Example](../../developing/transactions/transaction_jta_gemfire_example.html)**
+-   **[<%=vars.product_name%> JTA Transaction Example](transaction_jta_gemfire_example.html)**
 
-    An example code fragment shows how to run a JTA global transaction using Geode as the JTA transaction manager.
+    An example code fragment shows how to run a JTA global transaction using <%=vars.product_name%> as the JTA transaction manager.
 
--   **[JCA Resource Adapter Example](../../developing/transactions/jca_adapter_example.html)**
+-   **[JCA Resource Adapter Example](jca_adapter_example.html)**
 
-    This example shows how to use the JCA Resource Adapter in Geode .
+    This example shows how to use the JCA Resource Adapter in <%=vars.product_name%> .
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/transaction_event_management.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/transaction_event_management.html.md.erb b/geode-docs/developing/transactions/transaction_event_management.html.md.erb
index 9ec6b82..e9d84a9 100644
--- a/geode-docs/developing/transactions/transaction_event_management.html.md.erb
+++ b/geode-docs/developing/transactions/transaction_event_management.html.md.erb
@@ -46,11 +46,11 @@ results in these events stored in the CacheEvent list:
 
 # At commit and after commit
 
-When the transaction is committed, Geode passes the `TransactionEvent` to the transaction writer local to the transactional view, if one is available. After commit, Geode :
+When the transaction is committed, <%=vars.product_name%> passes the `TransactionEvent` to the transaction writer local to the transactional view, if one is available. After commit, <%=vars.product_name%> :
     -   Passes the `TransactionEvent` to each installed transaction listener.
     -   Walks the `CacheEvent` list, calling all locally installed listeners for each operation listed.
     -   Distributes the `TransactionEvent` to all interested caches.
         **Note:**
-        For Geode and global JTA transactions, the `EntryEvent` object contains the Geode transaction ID. JTA transaction events do not contain the JTA transaction ID.
+        For <%=vars.product_name%> and global JTA transactions, the `EntryEvent` object contains the <%=vars.product_name%> transaction ID. JTA transaction events do not contain the JTA transaction ID.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/transaction_jta_gemfire_example.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/transaction_jta_gemfire_example.html.md.erb b/geode-docs/developing/transactions/transaction_jta_gemfire_example.html.md.erb
index 131d164..8f0b1ad 100644
--- a/geode-docs/developing/transactions/transaction_jta_gemfire_example.html.md.erb
+++ b/geode-docs/developing/transactions/transaction_jta_gemfire_example.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Geode JTA Transaction Example
----
+<% set_title(product_name, "JTA Transaction Example") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,7 +17,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-An example code fragment shows how to run a JTA global transaction using Geode as the JTA transaction manager.
+An example code fragment shows how to run a JTA global transaction using <%=vars.product_name%> as the JTA transaction manager.
 
 The external data sources used in this transaction are configured in the `cache.xml` file. See [Configuring Database Connections Using JNDI](configuring_db_connections_using_JNDI.html#topic_A5E3A67C808D48C08E1F0DC167C5C494) for a configuration example.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/transaction_semantics.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/transaction_semantics.html.md.erb b/geode-docs/developing/transactions/transaction_semantics.html.md.erb
index 9a2e21e..3df3f20 100644
--- a/geode-docs/developing/transactions/transaction_semantics.html.md.erb
+++ b/geode-docs/developing/transactions/transaction_semantics.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Geode Cache Transaction Semantics
----
+<% set_title(product_name, "Cache Transaction Semantics") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,13 +17,13 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode transaction semantics differ in some ways from the Atomicity-Consistency-Isolation-Durability (ACID) semantics of traditional relational databases. For performance reasons, Geode transactions do not adhere to ACID constraints by default, but can be configured for ACID support as described in this section.
+<%=vars.product_name%> transaction semantics differ in some ways from the Atomicity-Consistency-Isolation-Durability (ACID) semantics of traditional relational databases. For performance reasons, <%=vars.product_name%> transactions do not adhere to ACID constraints by default, but can be configured for ACID support as described in this section.
 
 ## <a id="transaction_semantics__section_8362ACD06C784B5BBB0B7E986F760169" class="no-quick-link"></a>Atomicity
 
 Atomicity is “all or nothing” behavior: a transaction completes successfully only when all of the operations it contains complete successfully. If problems occur during a transaction, perhaps due to other transactions with overlapping changes, the transaction cannot successfully complete until the problems are resolved.
 
-Geode transactions provide atomicity and realize speed by using a reservation system, instead of using the traditional relational database technique of a two-phase locking of rows. The reservation prevents other, intersecting transactions from completing, allowing the commit to check for conflicts and to reserve resources in an all-or-nothing fashion prior to making changes to the data. After all changes have been made, locally and remotely, the reservation is released. With the reservation system, an intersecting transaction is simply discarded. The serialization of obtaining locks is avoided. See [Committing Transactions](how_cache_transactions_work.html#concept_sbj_lj1_wk) for details on the two-phase commit protocol that implements the reservation system.
+<%=vars.product_name%> transactions provide atomicity and realize speed by using a reservation system, instead of using the traditional relational database technique of a two-phase locking of rows. The reservation prevents other, intersecting transactions from completing, allowing the commit to check for conflicts and to reserve resources in an all-or-nothing fashion prior to making changes to the data. After all changes have been made, locally and remotely, the reservation is released. With the reservation system, an intersecting transaction is simply discarded. The serialization of obtaining locks is avoided. See [Committing Transactions](how_cache_transactions_work.html#concept_sbj_lj1_wk) for details on the two-phase commit protocol that implements the reservation system.
 
 ## <a id="transaction_semantics__section_7C287DA4A5134780B3199CE074E3F890" class="no-quick-link"></a>Consistency
 
@@ -33,9 +31,9 @@ Consistency requires that data written within a transaction must observe the key
 
 ## <a id="transaction_semantics__section_126A24EC499D4CF39AE766A0B526A9A5" class="no-quick-link"></a>Isolation
 
-Isolation assures that operations will see either the pre-transaction state of the system or its post-transaction state, but not the transitional state that occurs while a transaction is in progress. Write operations in a transaction are always confirmed to ensure that stale values are not written. As a distributed cache-based system optimized for performance, Geode in its default configuration does not enforce read isolation. Geode transactions support repeatable read isolation, so once the committed value is read for a given key, it always returns that same value. If a transaction write, such as put or invalidate, deletes a value for a key that has already been read, subsequent reads return the transactional reference.
+Isolation assures that operations will see either the pre-transaction state of the system or its post-transaction state, but not the transitional state that occurs while a transaction is in progress. Write operations in a transaction are always confirmed to ensure that stale values are not written. As a distributed cache-based system optimized for performance, <%=vars.product_name%> in its default configuration does not enforce read isolation. <%=vars.product_name%> transactions support repeatable read isolation, so once the committed value is read for a given key, it always returns that same value. If a transaction write, such as put or invalidate, deletes a value for a key that has already been read, subsequent reads return the transactional reference.
 
-In the default configuration, Geode isolates transactions at the process thread level, so while a transaction is in progress, its changes are visible only inside the thread that is running the transaction. Threads inside the same process and in other processes cannot see changes until after the commit operation begins. At this point, the changes are visible in the cache, but other threads that access the changing data might see only partial results of the transaction leading to a dirty read.
+In the default configuration, <%=vars.product_name%> isolates transactions at the process thread level, so while a transaction is in progress, its changes are visible only inside the thread that is running the transaction. Threads inside the same process and in other processes cannot see changes until after the commit operation begins. At this point, the changes are visible in the cache, but other threads that access the changing data might see only partial results of the transaction leading to a dirty read.
 
 If an application requires the slower conventional isolation model (such that dirty reads of transitional states are not allowed), read operations must be encapsulated within transactions and the `gemfire.detectReadConflicts` parameter must be set to ‘true’:
 
@@ -45,7 +43,7 @@ This parameter causes read operations to succeed only when they read a consisten
 
 ## <a id="transaction_semantics__section_F092E368724945BCBF8E5DCB36B97EB4" class="no-quick-link"></a>Durability
 
-Relational databases provide durability by using disk storage for recovery and transaction logging. As a distributed cache-based system optimized for performance, Geode does not support on-disk or in-memory durability for transactions.
+Relational databases provide durability by using disk storage for recovery and transaction logging. As a distributed cache-based system optimized for performance, <%=vars.product_name%> does not support on-disk or in-memory durability for transactions.
 
 Applications can emulate the conventional disk-based durability model by setting the `gemfire.ALLOW_PERSISTENT_TRANSACTIONS` parameter to ‘true’.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/transactional_and_nontransactional_ops.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/transactional_and_nontransactional_ops.html.md.erb b/geode-docs/developing/transactions/transactional_and_nontransactional_ops.html.md.erb
index dc9f198..7cda91f 100644
--- a/geode-docs/developing/transactions/transactional_and_nontransactional_ops.html.md.erb
+++ b/geode-docs/developing/transactions/transactional_and_nontransactional_ops.html.md.erb
@@ -20,7 +20,7 @@ limitations under the License.
 -->
 
 
-Between the begin operation and the commit or rollback operation are a series of ordinary Geode operations. When they are launched from within a transaction, the Geode operations can be classified into two types:
+Between the begin operation and the commit or rollback operation are a series of ordinary <%=vars.product_name%> operations. When they are launched from within a transaction, the <%=vars.product_name%> operations can be classified into two types:
 
 -   Transactional operations affect the transactional view
 -   Non-transactional operations do not affect the transactional view


[11/11] geode git commit: GEODE-3055: Should use a conservative fix to only catch the PartitionOfflineEx to remove the leader region bucket.

Posted by zh...@apache.org.
GEODE-3055: Should use a conservative fix to only catch the PartitionOfflineEx
to remove the leader region bucket.

Previous fix to catch all RuntimeException is too aggressive.


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

Branch: refs/heads/feature/GEM-1601
Commit: 360c3d536fb12106645d9942546324fc446546fe
Parents: 1c04aab
Author: zhouxh <gz...@pivotal.io>
Authored: Fri Aug 18 14:51:31 2017 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Fri Aug 18 16:10:06 2017 -0700

----------------------------------------------------------------------
 .../apache/geode/internal/cache/PartitionedRegionDataStore.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/360c3d53/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
index 3d9ac18..6b0c0a8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
@@ -21,6 +21,7 @@ import org.apache.geode.cache.Region.Entry;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionException;
 import org.apache.geode.cache.execute.ResultSender;
+import org.apache.geode.cache.persistence.PartitionOfflineException;
 import org.apache.geode.cache.query.QueryInvalidException;
 import org.apache.geode.cache.query.internal.QCompiler;
 import org.apache.geode.cache.query.internal.index.IndexCreationData;
@@ -493,7 +494,7 @@ public class PartitionedRegionDataStore implements HasCachePerfStats {
 
       return result;
 
-    } catch (RuntimeException validationException) {
+    } catch (PartitionOfflineException validationException) {
       // GEODE-3055
       PartitionedRegion leader = ColocationHelper.getLeaderRegion(this.partitionedRegion);
       boolean isLeader = leader.equals(this.partitionedRegion);


[03/11] geode git commit: GEODE-3395 Variable-ize product version and name in user guide - Developing

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/transactions/working_with_transactions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/working_with_transactions.html.md.erb b/geode-docs/developing/transactions/working_with_transactions.html.md.erb
index 4a26d4c..d75f6ad 100644
--- a/geode-docs/developing/transactions/working_with_transactions.html.md.erb
+++ b/geode-docs/developing/transactions/working_with_transactions.html.md.erb
@@ -1,6 +1,4 @@
----
-title: Working with Geode Cache Transactions
----
+<% set_title("Working with", product_name, "Cache Transactions") %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,7 +19,7 @@ limitations under the License.
 <a id="topic_tx2_gs4_5k"></a>
 
 
-This section contains guidelines and additional information on working with Geode and its cache transactions.
+This section contains guidelines and additional information on working with <%=vars.product_name%> and its cache transactions.
 
 -   **[Setting Global Copy on Read](#concept_vx2_gs4_5k)**
 
@@ -134,13 +132,13 @@ Local expiration actions do not cause write conflicts, but distributed expiratio
 
 A transaction that modifies a region in which consistency checking is enabled generates all necessary version information for region updates when the transaction commits.
 
-If a transaction modifies a normal, preloaded or empty region, the transaction is first delegated to a Geode member that holds a replicate for the region. This behavior is similar to the transactional behavior for partitioned regions, where the partitioned region transaction is forwarded to a member that hosts the primary for the partitioned region update.
+If a transaction modifies a normal, preloaded or empty region, the transaction is first delegated to a <%=vars.product_name%> member that holds a replicate for the region. This behavior is similar to the transactional behavior for partitioned regions, where the partitioned region transaction is forwarded to a member that hosts the primary for the partitioned region update.
 
-The limitation for transactions with a normal, preloaded or empty region is that, when consistency checking is enabled, a transaction cannot perform a `localDestroy` or `localInvalidate` operation against the region. Geode throws an `UnsupportedOperationInTransactionException` exception in such cases. An application should use a `Destroy` or `Invalidate` operation in place of a `localDestroy` or `localInvalidate` when consistency checks are enabled.
+The limitation for transactions with a normal, preloaded or empty region is that, when consistency checking is enabled, a transaction cannot perform a `localDestroy` or `localInvalidate` operation against the region. <%=vars.product_name%> throws an `UnsupportedOperationInTransactionException` exception in such cases. An application should use a `Destroy` or `Invalidate` operation in place of a `localDestroy` or `localInvalidate` when consistency checks are enabled.
 
 ## Suspending and Resuming Transactions
 
-The Geode `CacheTransactionManager` API provides the ability to suspend and resume transactions with the `suspend` and `resume` methods. The ability to suspend and resume is useful when a thread must perform some operations that should not be part of the transaction before the transaction can complete. A complex use case of suspend and resume implements a transaction that spans clients in which only one client at a time will not be suspended.
+The <%=vars.product_name%> `CacheTransactionManager` API provides the ability to suspend and resume transactions with the `suspend` and `resume` methods. The ability to suspend and resume is useful when a thread must perform some operations that should not be part of the transaction before the transaction can complete. A complex use case of suspend and resume implements a transaction that spans clients in which only one client at a time will not be suspended.
 
 Once a transaction is suspended, it loses the transactional view of the cache. None of the operations done within the transaction are visible to the thread. Any operations that are performed by the thread while the transaction is suspended are not part of the transaction.
 
@@ -150,17 +148,17 @@ Before resuming a transaction, you may want to check if the transaction exists o
 
 If the member with the primary copy of the data crashes, the transactional view associated with that data is lost. The secondary member for the data will not be able to resume any transactions suspended on the crashed member. You will need to take remedial steps to retry the transaction on a new primary copy of the data.
 
-If a suspended transaction is not touched for a period of time, Geode cleans it up automatically. By default, the timeout for a suspended transaction is 30 minutes and can be configured using the system property `gemfire.suspendedtxTimeout`. For example, `gemfire.suspendedtxTimeout=60` specifies a timeout of 60 minutes.
+If a suspended transaction is not touched for a period of time, <%=vars.product_name%> cleans it up automatically. By default, the timeout for a suspended transaction is 30 minutes and can be configured using the system property `gemfire.suspendedtxTimeout`. For example, `gemfire.suspendedtxTimeout=60` specifies a timeout of 60 minutes.
 
 See [Basic Suspend and Resume Transaction Example](transaction_suspend_resume_example.html) for a sample code fragment that suspends and resumes a transaction.
 
 ## Using Cache Writer and Cache Listener Plug-Ins
 
-All standard Geode application plug-ins work with transactions. In addition, the transaction interface offers specialized plug-ins that support transactional operation.
+All standard <%=vars.product_name%> application plug-ins work with transactions. In addition, the transaction interface offers specialized plug-ins that support transactional operation.
 
-No direct interaction exists between client transactions and client application plug-ins. When a client runs a transaction, Geode calls the plug-ins that are installed on the transaction's server delegate and its server host. Client application plug-ins are not called for operations inside the transaction or for the transaction as a whole. When the transaction is committed, the changes to the server cache are sent to the client cache according to client interest registration. These events can result in calls to the client's `CacheListener`s, as with any other events received from the server.
+No direct interaction exists between client transactions and client application plug-ins. When a client runs a transaction, <%=vars.product_name%> calls the plug-ins that are installed on the transaction's server delegate and its server host. Client application plug-ins are not called for operations inside the transaction or for the transaction as a whole. When the transaction is committed, the changes to the server cache are sent to the client cache according to client interest registration. These events can result in calls to the client's `CacheListener`s, as with any other events received from the server.
 
-The `EntryEvent` that a callback receives has a unique Geode transaction ID, so the cache listener can associate each event, as it occurs, with a particular transaction. The transaction ID of an `EntryEvent` that is not part of a transaction is null to distinguish it from a transaction ID.
+The `EntryEvent` that a callback receives has a unique <%=vars.product_name%> transaction ID, so the cache listener can associate each event, as it occurs, with a particular transaction. The transaction ID of an `EntryEvent` that is not part of a transaction is null to distinguish it from a transaction ID.
 
 -   `CacheLoader`. When a cache loader is called by a transaction operation, values loaded by the cache loader may cause a write conflict when the transaction commits.
 -   `CacheWriter`. During a transaction, if a cache writer exists, its methods are invoked as usual for all operations, as the operations are called in the transactions. The `netWrite` operation is not used. The only cache writer used is the one in the member where the transactional data resides.
@@ -170,9 +168,9 @@ For more information on writing cache event handlers, see [Implementing Cache Ev
 
 ## <a id="concept_ocw_vf1_wk" class="no-quick-link"></a>Configuring Transaction Plug-In Event Handlers
 
-Geode has two types of transaction plug-ins: Transaction Writers and Transaction Listeners. You can optionally install one transaction writer and one or more transaction listeners per cache.
+<%=vars.product_name%> has two types of transaction plug-ins: Transaction Writers and Transaction Listeners. You can optionally install one transaction writer and one or more transaction listeners per cache.
 
-Like JTA global transactions, you can use transaction plug-in event handlers to coordinate Geode cache transaction activity with an external data store. However, you typically use JTA global transactions when Geode is running as a peer data store with your external data stores. Transaction writers and listeners are typically used when Geode is acting as a front end cache to your backend database.
+Like JTA global transactions, you can use transaction plug-in event handlers to coordinate <%=vars.product_name%> cache transaction activity with an external data store. However, you typically use JTA global transactions when <%=vars.product_name%> is running as a peer data store with your external data stores. Transaction writers and listeners are typically used when <%=vars.product_name%> is acting as a front end cache to your backend database.
 
 **Note:**
 You can also use transaction plug-in event handlers when running JTA global transactions.
@@ -181,7 +179,7 @@ You can also use transaction plug-in event handlers when running JTA global tran
 
 When you commit a transaction, if a transaction writer is installed in the cache where the data updates were performed, it is called. The writer can do whatever work you need, including aborting the transaction.
 
-The transaction writer is the last place that an application can rollback a transaction. If the transaction writer throws any exception, the transaction is rolled back. For example, you might use a transaction writer to update a backend data source before the Geode cache transaction completes the commit. If the backend data source update fails, the transaction writer implementation can throw a [TransactionWriterException](/releases/latest/javadoc/org/apache/geode/cache/TransactionWriterException.html) to veto the transaction.
+The transaction writer is the last place that an application can rollback a transaction. If the transaction writer throws any exception, the transaction is rolled back. For example, you might use a transaction writer to update a backend data source before the <%=vars.product_name%> cache transaction completes the commit. If the backend data source update fails, the transaction writer implementation can throw a [TransactionWriterException](/releases/latest/javadoc/org/apache/geode/cache/TransactionWriterException.html) to veto the transaction.
 
 A typical usage scenario would be to use the transaction writer to prepare the commit on the external database. Then in a transaction listener, you can apply the commit on the database.
 
@@ -193,7 +191,7 @@ Transaction listeners have access to the transactional view and thus are not aff
 
 A transaction listener can preserve the result of a transaction, perhaps to compare with other transactions, or for reference in case of a failed commit. When a commit fails and the transaction ends, the application cannot just retry the transaction, but must build up the data again. For most applications, the most efficient action is just to start a new transaction and go back through the application logic again.
 
-The rollback and failed commit operations are local to the member where the transactional operations are run. When a successful commit writes to a distributed or partitioned region, however, the transaction results are distributed to other members the same as other updates. The transaction listener on the receiving members reflect the changes the transaction makes in that member, not the originating member. Any exceptions thrown by the transaction listener are caught by Geode and logged.
+The rollback and failed commit operations are local to the member where the transactional operations are run. When a successful commit writes to a distributed or partitioned region, however, the transaction results are distributed to other members the same as other updates. The transaction listener on the receiving members reflect the changes the transaction makes in that member, not the originating member. Any exceptions thrown by the transaction listener are caught by <%=vars.product_name%> and logged.
 
 To configure a transaction listener, add a `cache-transaction-manager` configuration to the cache definition and define one or more instances of `transaction-listener` there. The only parameter to this `transaction-listener` is `URL`, which must be a string, as shown in the following cache.xml example.
 


[05/11] geode git commit: GEODE-3395 Variable-ize product version and name in user guide - Developing

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/rebalancing_pr_data.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/rebalancing_pr_data.html.md.erb b/geode-docs/developing/partitioned_regions/rebalancing_pr_data.html.md.erb
index d40a4f2..66057a4 100644
--- a/geode-docs/developing/partitioned_regions/rebalancing_pr_data.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/rebalancing_pr_data.html.md.erb
@@ -34,7 +34,7 @@ If you have transactions running in your system, be careful in planning your reb
 
 Kick off a rebalance using one of the following:
 
--   `gfsh` command. First, starting a `gfsh` prompt and connect to the Geode distributed system. Then type the following command:
+-   `gfsh` command. First, starting a `gfsh` prompt and connect to the <%=vars.product_name%> distributed system. Then type the following command:
 
     ``` pre
     gfsh>rebalance
@@ -70,11 +70,11 @@ The rebalancing operation runs asynchronously.
 
 By default, rebalancing is performed on one partitioned region at a time. For regions that have colocated data, the rebalancing works on the regions as a group, maintaining the data colocation between the regions.
 
-You can optionally rebalance multiple regions in parallel by setting the `gemfire.resource.manager.threads` system property. Setting this property to a value greater than 1 enables Geode to rebalance multiple regions in parallel, any time a rebalance operation is initiated using the API.
+You can optionally rebalance multiple regions in parallel by setting the `gemfire.resource.manager.threads` system property. Setting this property to a value greater than 1 enables <%=vars.product_name%> to rebalance multiple regions in parallel, any time a rebalance operation is initiated using the API.
 
 You can continue to use your partitioned regions normally while rebalancing is in progress. Read operations, write operations, and function executions continue while data is moving. If a function is executing on a local data set, you may see a performance degradation if that data moves to another host during function execution. Future function invocations are routed to the correct member.
 
-Geode tries to ensure that each member has the same percentage of its available space used for each partitioned region. The percentage is configured in the `partition-attributes` `local-max-memory` setting.
+<%=vars.product_name%> tries to ensure that each member has the same percentage of its available space used for each partitioned region. The percentage is configured in the `partition-attributes` `local-max-memory` setting.
 
 Partitioned region rebalancing:
 
@@ -89,7 +89,7 @@ You typically want to trigger rebalancing when capacity is increased or reduced
 
 You may also need to rebalance when:
 
--   You use redundancy for high availability and have configured your region to not automatically recover redundancy after a loss. In this case, Geode only restores redundancy when you invoke a rebalance. See [Configure High Availability for a Partitioned Region](configuring_ha_for_pr.html).
+-   You use redundancy for high availability and have configured your region to not automatically recover redundancy after a loss. In this case, <%=vars.product_name%> only restores redundancy when you invoke a rebalance. See [Configure High Availability for a Partitioned Region](configuring_ha_for_pr.html).
 -   You have uneven hashing of data. Uneven hashing can occur if your keys do not have a hash code method, which ensures uniform distribution, or if you use a `PartitionResolver` to colocate your partitioned region data (see [Colocate Data from Different Partitioned Regions](colocating_partitioned_region_data.html#colocating_partitioned_region_data)). In either case, some buckets may receive more data than others. Rebalancing can be used to even out the load between data stores by putting fewer buckets on members that are hosting large buckets.
 
 ## <a id="rebalancing_pr_data__section_495FEE48ED60433BADB7D36C73279C89" class="no-quick-link"></a>How to Simulate Region Rebalancing

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/set_enforce_unique_host.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/set_enforce_unique_host.html.md.erb b/geode-docs/developing/partitioned_regions/set_enforce_unique_host.html.md.erb
index 000216c..fdfaf5a 100644
--- a/geode-docs/developing/partitioned_regions/set_enforce_unique_host.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/set_enforce_unique_host.html.md.erb
@@ -19,11 +19,11 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Configure Geode to use only unique physical machines for redundant copies of partitioned region data.
+Configure <%=vars.product_name%> to use only unique physical machines for redundant copies of partitioned region data.
 
 Understand how to set a member's `gemfire.properties` settings. See [Reference](../../reference/book_intro.html#reference).
 
-Configure your members so Geode always uses different physical machines for redundant copies of partitioned region data using the `gemfire.properties` setting `enforce-unique-host`. The default for this setting is false. 
+Configure your members so <%=vars.product_name%> always uses different physical machines for redundant copies of partitioned region data using the `gemfire.properties` setting `enforce-unique-host`. The default for this setting is false. 
 
 Example:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/set_redundancy_zones.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/set_redundancy_zones.html.md.erb b/geode-docs/developing/partitioned_regions/set_redundancy_zones.html.md.erb
index e08be5d..d4d3838 100644
--- a/geode-docs/developing/partitioned_regions/set_redundancy_zones.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/set_redundancy_zones.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Group members into redundancy zones so Geode will separate redundant data copies into different zones.
+Group members into redundancy zones so <%=vars.product_name%> will separate redundant data copies into different zones.
 
 Understand how to set a member's `gemfire.properties` settings. See [Reference](../../reference/book_intro.html#reference).
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/partitioned_regions/using_custom_partition_resolvers.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/partitioned_regions/using_custom_partition_resolvers.html.md.erb b/geode-docs/developing/partitioned_regions/using_custom_partition_resolvers.html.md.erb
index 40b2237..44b45d8 100644
--- a/geode-docs/developing/partitioned_regions/using_custom_partition_resolvers.html.md.erb
+++ b/geode-docs/developing/partitioned_regions/using_custom_partition_resolvers.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-By default, Geode partitions each data entry into a bucket using a hashing policy on the key. Additionally, the physical location of the key-value pair is abstracted away from the application. You can change these policies for a partitioned region. You can provide your own data partitioning resolver and you can additionally specify which members host which data buckets.
+By default, <%=vars.product_name%> partitions each data entry into a bucket using a hashing policy on the key. Additionally, the physical location of the key-value pair is abstracted away from the application. You can change these policies for a partitioned region. You can provide your own data partitioning resolver and you can additionally specify which members host which data buckets.
 
 <a id="custom_partition_region_data__section_CF05CE974C9C4AF78430DA55601D2158"></a>
 **Note:**
@@ -40,7 +40,7 @@ For standard partitioning, use `org.apache.geode.cache.PartitionResolver`. To im
 
 **Procedure**
 
-1.  If using `org.apache.geode.cache.PartitionResolver` (standard partitioning) or `org.apache.geode.cache.FixedPartitionResolver` (fixed partitioning), implement the standard partitioning resolver or the fixed partitioning resolver in one of the following locations, listed here in the search order used by Geode:
+1.  If using `org.apache.geode.cache.PartitionResolver` (standard partitioning) or `org.apache.geode.cache.FixedPartitionResolver` (fixed partitioning), implement the standard partitioning resolver or the fixed partitioning resolver in one of the following locations, listed here in the search order used by <%=vars.product_name%>:
     -   **Custom class**. You provide this class as the partition resolver to the region creation.
     -   **Entry key**. You use the implementing key object for every operation on the region entries.
     -   **Cache callback argument**. This implementation restricts you to using methods that accept a cache callback argument to manage the region entries. For a full list of the methods that take a callback argument, see the `Region` Javadocs.
@@ -54,7 +54,7 @@ function that partitions the entry.
 2.  If you need the resolver's `getName` method, program that.
 3.  If *not* using the default implementation of the string-based
 partition resolver,
-program the resolver's `getRoutingObject` method to return the routing object for each entry, based on how you want to group the entries. Give the same routing object to entries you want to group together. Geode will place the entries in the same bucket.
+program the resolver's `getRoutingObject` method to return the routing object for each entry, based on how you want to group the entries. Give the same routing object to entries you want to group together. <%=vars.product_name%> will place the entries in the same bucket.
 
     **Note:**
     Only fields on the key should be used when creating the routing object. Do not use the value or additional metadata for this purpose.
@@ -130,7 +130,7 @@ program the resolver's `getRoutingObject` method to return the routing object fo
 
             You cannot specify a partition resolver using gfsh.
 
-    2.  Program the `FixedPartitionResolver` `getPartitionName` method to return the name of the partition for each entry, based on where you want the entries to reside. Geode uses `getPartitionName` and `getRoutingObject` to determine where an entry is placed.
+    2.  Program the `FixedPartitionResolver` `getPartitionName` method to return the name of the partition for each entry, based on where you want the entries to reside. <%=vars.product_name%> uses `getPartitionName` and `getRoutingObject` to determine where an entry is placed.
 
         **Note:**
         To group entries, assign every entry in the group the same routing object and the same partition name.
@@ -188,7 +188,7 @@ program the resolver's `getRoutingObject` method to return the routing object fo
         }
         ```
 
-5.  Configure or program the region so Geode finds your resolver for every operation that you perform on the region's entries. How you do this depends on where you chose to program your custom partitioning implementation (step 1).
+5.  Configure or program the region so <%=vars.product_name%> finds your resolver for every operation that you perform on the region's entries. How you do this depends on where you chose to program your custom partitioning implementation (step 1).
     -   **Custom class**. Define the class for the region at creation. The resolver will be used for every entry operation. Use one of these methods:
 
         **XML:**

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_additional/advanced_querying.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_additional/advanced_querying.html.md.erb b/geode-docs/developing/query_additional/advanced_querying.html.md.erb
index 7054868..ce758a2 100644
--- a/geode-docs/developing/query_additional/advanced_querying.html.md.erb
+++ b/geode-docs/developing/query_additional/advanced_querying.html.md.erb
@@ -21,27 +21,27 @@ limitations under the License.
 
 This section includes advanced querying topics such as using query indexes, using query bind parameters, querying partitioned regions and query debugging.
 
--   **[Performance Considerations](../../developing/querying_basics/performance_considerations.html)**
+-   **[Performance Considerations](../querying_basics/performance_considerations.html)**
 
     This topic covers considerations for improving query performance.
 
--   **[Monitoring Queries for Low Memory](../../developing/querying_basics/monitor_queries_for_low_memory.html)**
+-   **[Monitoring Queries for Low Memory](../querying_basics/monitor_queries_for_low_memory.html)**
 
     The query monitoring feature prevents out-of-memory exceptions from occurring when you execute queries or create indexes.
 
--   **[Using Query Bind Parameters](../../developing/query_additional/using_query_bind_parameters.html)**
+-   **[Using Query Bind Parameters](../query_additional/using_query_bind_parameters.html)**
 
-    Using query bind parameters in Geode queries is similar to using prepared statements in SQL where parameters can be set during query execution. This allows user to build a query once and execute it multiple times by passing the query conditions during run time.
+    Using query bind parameters in <%=vars.product_name%> queries is similar to using prepared statements in SQL where parameters can be set during query execution. This allows user to build a query once and execute it multiple times by passing the query conditions during run time.
 
--   **[Working with Indexes](../../developing/query_index/query_index.html)**
+-   **[Working with Indexes](../query_index/query_index.html)**
 
-    The Geode query engine supports indexing. An index can provide significant performance gains for query execution.
+    The <%=vars.product_name%> query engine supports indexing. An index can provide significant performance gains for query execution.
 
--   **[Querying Partitioned Regions](../../developing/querying_basics/querying_partitioned_regions.html)**
+-   **[Querying Partitioned Regions](../querying_basics/querying_partitioned_regions.html)**
 
-    Geode allows you to manage and store large amounts of data across distributed nodes using partitioned regions. The basic unit of storage for a partitioned region is a bucket, which resides on a Geode node and contains all the entries that map to a single hashcode. In a typical partitioned region query, the system distributes the query to all buckets across all nodes, then merges the result sets and sends back the query results.
+    <%=vars.product_name%> allows you to manage and store large amounts of data across distributed nodes using partitioned regions. The basic unit of storage for a partitioned region is a bucket, which resides on a <%=vars.product_name%> node and contains all the entries that map to a single hashcode. In a typical partitioned region query, the system distributes the query to all buckets across all nodes, then merges the result sets and sends back the query results.
 
--   **[Query Debugging](../../developing/query_additional/query_debugging.html)**
+-   **[Query Debugging](../query_additional/query_debugging.html)**
 
     You can debug a specific query at the query level by adding the `<trace>` keyword before the query string that you want to debug.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_additional/literals.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_additional/literals.html.md.erb b/geode-docs/developing/query_additional/literals.html.md.erb
index e86371c..40c4434 100644
--- a/geode-docs/developing/query_additional/literals.html.md.erb
+++ b/geode-docs/developing/query_additional/literals.html.md.erb
@@ -21,7 +21,7 @@ limitations under the License.
 
 ## <a id="literals__section_BA2D0AC444EB45088F00D9E2C8A1DD06" class="no-quick-link"></a>Comparing Values With java.util.Date
 
-Geode supports the following literal types:
+<%=vars.product_name%> supports the following literal types:
 
 <dt>**boolean**</dt>
 <dd>A `boolean` value, either TRUE or FALSE</dd>
@@ -51,7 +51,7 @@ You can compare temporal literal values `DATE`, `TIME`, and `TIMESTAMP` with `ja
 
 ## <a id="literals__section_9EE6CFC410D2409188EDEAA43AC85851" class="no-quick-link"></a>Type Conversion
 
-The Geode query processor performs implicit type conversions and promotions under certain cases in order to evaluate expressions that contain different types. The query processor performs binary numeric promotion, method invocation conversion, and temporal type conversion.
+The <%=vars.product_name%> query processor performs implicit type conversions and promotions under certain cases in order to evaluate expressions that contain different types. The query processor performs binary numeric promotion, method invocation conversion, and temporal type conversion.
 
 ## <a id="literals__section_F5A3FC509FD04E09B5468BA94B814701" class="no-quick-link"></a>Binary Numeric Promotion
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_additional/operators.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_additional/operators.html.md.erb b/geode-docs/developing/query_additional/operators.html.md.erb
index e8cca37..a4a3d8d 100644
--- a/geode-docs/developing/query_additional/operators.html.md.erb
+++ b/geode-docs/developing/query_additional/operators.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode supports comparison, logical, unary, map, index, dot, and right arrow operators.
+<%=vars.product_name%> supports comparison, logical, unary, map, index, dot, and right arrow operators.
 
 ## <a id="operators__section_A3FB372F85D840D7A49CB95BD7FCA7C6" class="no-quick-link"></a>Comparison Operators
 
@@ -42,7 +42,7 @@ The logical operators AND and OR allow you to create more complex expressions by
 
 ## <a id="operators__section_A970AE75B0D24E0B9E1B61BE2D9842D8" class="no-quick-link"></a>Unary Operators
 
-Unary operators operate on a single value or expression, and have lower precedence than comparison operators in expressions. Geode supports the unary operator NOT. NOT is the negation operator, which changes the value of the operand to its opposite. So if an expression evaluates to TRUE, NOT changes it to FALSE. The operand must be a boolean.
+Unary operators operate on a single value or expression, and have lower precedence than comparison operators in expressions. <%=vars.product_name%> supports the unary operator NOT. NOT is the negation operator, which changes the value of the operand to its opposite. So if an expression evaluates to TRUE, NOT changes it to FALSE. The operand must be a boolean.
 
 ## <a id="operators__section_E78FB4FB3703471C8186A0E26D25F01F" class="no-quick-link"></a>Map and Index Operators
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_additional/query_debugging.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_additional/query_debugging.html.md.erb b/geode-docs/developing/query_additional/query_debugging.html.md.erb
index c404d6b..8ec8703 100644
--- a/geode-docs/developing/query_additional/query_debugging.html.md.erb
+++ b/geode-docs/developing/query_additional/query_debugging.html.md.erb
@@ -33,7 +33,7 @@ You can also write:
 <TRACE> select * from /exampleRegion
 ```
 
-When the query is executed, Geode will log a message in `$GEMFIRE_DIR/system.log` with the following information:
+When the query is executed, <%=vars.product_name%> will log a message in `$GEMFIRE_DIR/system.log` with the following information:
 
 ``` pre
 [info 2011/08/29 11:24:35.472 PDT CqServer <main> tid=0x1] Query Executed in 9.619656 ms; rowCount = 99; indexesUsed(0) "select *  from /exampleRegion" 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_additional/query_language_features.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_additional/query_language_features.html.md.erb b/geode-docs/developing/query_additional/query_language_features.html.md.erb
index 10ab0c6..e9d3602 100644
--- a/geode-docs/developing/query_additional/query_language_features.html.md.erb
+++ b/geode-docs/developing/query_additional/query_language_features.html.md.erb
@@ -22,20 +22,20 @@ limitations under the License.
 <a id="concept_5B8BA904DF2A41BEAA057017777D4E90__section_33F0FD791A2448CB812E8397828B33C2"></a>
 This section covers the following querying language features:
 
--   **[Supported Character Sets](../../developing/querying_basics/supported_character_sets.html)**
+-   **[Supported Character Sets](../querying_basics/supported_character_sets.html)**
 
--   **[Supported Keywords](../../developing/query_additional/supported_keywords.html)**
+-   **[Supported Keywords](supported_keywords.html)**
 
--   **[Case Sensitivity](../../developing/query_additional/case_sensitivity.html)**
+-   **[Case Sensitivity](case_sensitivity.html)**
 
--   **[Comments in Query Strings](../../developing/querying_basics/comments_in_query_strings.html)**
+-   **[Comments in Query Strings](../querying_basics/comments_in_query_strings.html)**
 
--   **[Query Language Grammar](../../developing/querying_basics/query_grammar_and_reserved_words.html)**
+-   **[Query Language Grammar](../querying_basics/query_grammar_and_reserved_words.html)**
 
--   **[Operators](../../developing/query_additional/operators.html)**
+-   **[Operators](operators.html)**
 
--   **[Reserved Words](../../developing/querying_basics/reserved_words.html)**
+-   **[Reserved Words](../querying_basics/reserved_words.html)**
 
--   **[Supported Literals](../../developing/query_additional/literals.html)**
+-   **[Supported Literals](literals.html)**
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_additional/using_query_bind_parameters.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_additional/using_query_bind_parameters.html.md.erb b/geode-docs/developing/query_additional/using_query_bind_parameters.html.md.erb
index 8fee56b..880d186 100644
--- a/geode-docs/developing/query_additional/using_query_bind_parameters.html.md.erb
+++ b/geode-docs/developing/query_additional/using_query_bind_parameters.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Using query bind parameters in Geode queries is similar to using prepared statements in SQL where parameters can be set during query execution. This allows user to build a query once and execute it multiple times by passing the query conditions during run time.
+Using query bind parameters in <%=vars.product_name%> queries is similar to using prepared statements in SQL where parameters can be set during query execution. This allows user to build a query once and execute it multiple times by passing the query conditions during run time.
 
 Query objects are thread-safe.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_index/create_multiple_indexes.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_index/create_multiple_indexes.html.md.erb b/geode-docs/developing/query_index/create_multiple_indexes.html.md.erb
index e5a15c0..0f8f8af 100644
--- a/geode-docs/developing/query_index/create_multiple_indexes.html.md.erb
+++ b/geode-docs/developing/query_index/create_multiple_indexes.html.md.erb
@@ -61,7 +61,7 @@ Message : Region ' /r3' not found: from  /r3Occurred on following members
     List<Index> indexes = queryService.createDefinedIndexes();
 ```
 
-If one or more index population fails, Geode collect the Exceptions and continues to populate the rest of the indexes. The collected `Exceptions` are stored in a Map of index names and exceptions that can be accessed through `MultiIndexCreationException`.
+If one or more index population fails, <%=vars.product_name%> collect the Exceptions and continues to populate the rest of the indexes. The collected `Exceptions` are stored in a Map of index names and exceptions that can be accessed through `MultiIndexCreationException`.
 
 Index definitions are stored locally on the `gfsh` client. If you want to create a new set of indexes or if one or more of the index creations fail, you might want to clear the definitions stored by using `clear defined indexes`command. The defined indexes can be cleared by using the Java API:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_index/creating_an_index.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_index/creating_an_index.html.md.erb b/geode-docs/developing/query_index/creating_an_index.html.md.erb
index 2438447..abac6f3 100644
--- a/geode-docs/developing/query_index/creating_an_index.html.md.erb
+++ b/geode-docs/developing/query_index/creating_an_index.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-The Geode `QueryService` API provides methods to create, list and remove the index. You can also use `gfsh` command-line interface to create, list and remove indexes, and use cache.xml to create an index.
+The <%=vars.product_name%> `QueryService` API provides methods to create, list and remove the index. You can also use `gfsh` command-line interface to create, list and remove indexes, and use cache.xml to create an index.
 
 ## <a id="indexing__section_565C080FBDD0443C8504DF372E3C32C8" class="no-quick-link"></a>Creating Indexes
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_index/creating_hash_indexes.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_index/creating_hash_indexes.html.md.erb b/geode-docs/developing/query_index/creating_hash_indexes.html.md.erb
index bd97e6f..8ee8167 100644
--- a/geode-docs/developing/query_index/creating_hash_indexes.html.md.erb
+++ b/geode-docs/developing/query_index/creating_hash_indexes.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode supports the creation of hash indexes for the purposes of performing equality-based queries.
+<%=vars.product_name%> supports the creation of hash indexes for the purposes of performing equality-based queries.
 
 ## <a id="concept_5C7614F71F394C62ACA1BDC5684A7AC4__section_8A927DFB29364DA7856E7FE122FC1654" class="no-quick-link"></a>Why Create a HashIndex
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_index/indexing_guidelines.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_index/indexing_guidelines.html.md.erb b/geode-docs/developing/query_index/indexing_guidelines.html.md.erb
index 88c14b0..4470d97 100644
--- a/geode-docs/developing/query_index/indexing_guidelines.html.md.erb
+++ b/geode-docs/developing/query_index/indexing_guidelines.html.md.erb
@@ -31,7 +31,7 @@ When creating indexes, keep in mind the following:
 
 ## <a id="indexing_guidelines__section_A8AFAA243B5C43DD9BB9F9235A48AF53" class="no-quick-link"></a>Tips for Writing Queries that Use Indexes
 
-As with query processors that run against relational databases, the way a query is written can greatly affect execution performance. Among other things, whether indexes are used depends on how each query is stated. These are some of the things to consider when optimizing your Geode queries for performance:
+As with query processors that run against relational databases, the way a query is written can greatly affect execution performance. Among other things, whether indexes are used depends on how each query is stated. These are some of the things to consider when optimizing your <%=vars.product_name%> queries for performance:
 
 -   In general an index will improve query performance if the FROM clauses of the query and index match exactly.
 -   The query evaluation engine does not have a sophisticated cost-based optimizer. It has a simple optimizer which selects best index (one) or multiple indexes based on the index size and the operator that is being evaluated.

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_index/maintaining_indexes.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_index/maintaining_indexes.html.md.erb b/geode-docs/developing/query_index/maintaining_indexes.html.md.erb
index 838f380..8214076 100644
--- a/geode-docs/developing/query_index/maintaining_indexes.html.md.erb
+++ b/geode-docs/developing/query_index/maintaining_indexes.html.md.erb
@@ -52,7 +52,7 @@ Flight {
 }
 ```
 
-An index on the Passenger name field will have different memory space requirements in the cache than the Flight origin field even though they are both String field types. The internal data structure selected by Geode for index storage will depend on the field's level in the object. In this example, name is a top-level field and an index on name can be stored as a compact index. Since origin is a second-level field, any index that uses origin as the indexed expression will be stored as a non-compact index.
+An index on the Passenger name field will have different memory space requirements in the cache than the Flight origin field even though they are both String field types. The internal data structure selected by <%=vars.product_name%> for index storage will depend on the field's level in the object. In this example, name is a top-level field and an index on name can be stored as a compact index. Since origin is a second-level field, any index that uses origin as the indexed expression will be stored as a non-compact index.
 
 **Compact Index**
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_index/query_index.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_index/query_index.html.md.erb b/geode-docs/developing/query_index/query_index.html.md.erb
index 0f2c698..3d53e55 100644
--- a/geode-docs/developing/query_index/query_index.html.md.erb
+++ b/geode-docs/developing/query_index/query_index.html.md.erb
@@ -19,60 +19,60 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-The Geode query engine supports indexing. An index can provide significant performance gains for query execution.
+The <%=vars.product_name%> query engine supports indexing. An index can provide significant performance gains for query execution.
 
 <a id="indexing__section_565C080FBDD0443C8504DF372E3C32C8"></a>
 A query run without the aid of an index iterates through every object in the collection. If an index is available that matches part or all of the query specification, the query iterates only over the indexed set, and query processing time can be reduced.
 
--   **[Tips and Guidelines on Using Indexes](../../developing/query_index/indexing_guidelines.html)**
+-   **[Tips and Guidelines on Using Indexes](indexing_guidelines.html)**
 
     Optimizing your queries with indexes requires a cycle of careful planning, testing, and tuning. Poorly-defined indexes can degrade the performance of your queries instead of improving it. This section gives guidelines for index usage in the query service.
 
--   **[Creating, Listing and Removing Indexes](../../developing/query_index/creating_an_index.html)**
+-   **[Creating, Listing and Removing Indexes](creating_an_index.html)**
 
-    The Geode `QueryService` API provides methods to create, list and remove the index. You can also use `gfsh` command-line interface to create, list and remove indexes, and use cache.xml to create an index.
+    The <%=vars.product_name%> `QueryService` API provides methods to create, list and remove the index. You can also use `gfsh` command-line interface to create, list and remove indexes, and use cache.xml to create an index.
 
--   **[Creating Key Indexes](../../developing/query_index/creating_key_indexes.html)**
+-   **[Creating Key Indexes](creating_key_indexes.html)**
 
     Creating a key index is a good way to improve query performance when data is partitioned using a key or a field value. You can create key indexes by using the `createKeyIndex` method of the QueryService or by defining the index in `cache.xml`. Creating a key index makes the query service aware of the relationship between the values in the region and the keys in the region.
 
--   **[Creating Hash Indexes](../../developing/query_index/creating_hash_indexes.html)**
+-   **[Creating Hash Indexes](creating_hash_indexes.html)**
 
-    Geode supports the creation of hash indexes for the purposes of performing equality-based queries.
+    <%=vars.product_name%> supports the creation of hash indexes for the purposes of performing equality-based queries.
 
--   **[Creating Indexes on Map Fields ("Map Indexes")](../../developing/query_index/creating_map_indexes.html)**
+-   **[Creating Indexes on Map Fields ("Map Indexes")](creating_map_indexes.html)**
 
     To assist with the quick lookup of multiple values in a Map (or HashMap) type field, you can create an index (sometimes referred to as a "map index") on specific (or all) keys in that field.
 
--   **[Creating Multiple Indexes at Once](../../developing/query_index/create_multiple_indexes.html)**
+-   **[Creating Multiple Indexes at Once](create_multiple_indexes.html)**
 
     In order to speed and promote efficiency when creating indexes, you can define multiple indexes and then create them all at once.
 
--   **[Maintaining Indexes (Synchronously or Asynchronously) and Index Storage](../../developing/query_index/maintaining_indexes.html)**
+-   **[Maintaining Indexes (Synchronously or Asynchronously) and Index Storage](maintaining_indexes.html)**
 
     Indexes are automatically kept current with the region data they reference. The region attribute `IndexMaintenanceSynchronous` specifies whether the region indexes are updated synchronously when a region is modified or asynchronously in a background thread.
 
--   **[Using Query Index Hints](../../developing/query_index/query_index_hints.html)**
+-   **[Using Query Index Hints](query_index_hints.html)**
 
-    You can use the hint keyword to allow Geode's query engine to prefer certain indexes.
+    You can use the hint keyword to allow <%=vars.product_name%>'s query engine to prefer certain indexes.
 
--   **[Using Indexes on Single Region Queries](../../developing/query_index/indexes_on_single_region_queries.html)**
+-   **[Using Indexes on Single Region Queries](indexes_on_single_region_queries.html)**
 
     Queries with one comparison operation may be improved with either a key or range index, depending on whether the attribute being compared is also the primary key.
 
--   **[Using Indexes with Equi-Join Queries](../../developing/query_index/using_indexes_with_equijoin_queries.html)**
+-   **[Using Indexes with Equi-Join Queries](using_indexes_with_equijoin_queries.html)**
 
     Equi-join queries are queries in which two regions are joined through an equality condition in the WHERE clause.
 
--   **[Using Indexes with Overflow Regions](../../developing/query_index/indexes_with_overflow_regions.html)**
+-   **[Using Indexes with Overflow Regions](indexes_with_overflow_regions.html)**
 
     You can use indexes when querying on overflow regions; however, there are caveats.
 
--   **[Using Indexes on Equi-Join Queries using Multiple Regions](../../developing/query_index/using_indexes_with_equijoin_queries_multiple_regions.html)**
+-   **[Using Indexes on Equi-Join Queries using Multiple Regions](using_indexes_with_equijoin_queries_multiple_regions.html)**
 
     To query across multiple regions, identify all equi-join conditions. Then, create as few indexes for the equi-join conditions as you can while still joining all regions.
 
--   **[Index Samples](../../developing/query_index/index_samples.html)**
+-   **[Index Samples](index_samples.html)**
 
     This topic provides code samples for creating query indexes.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_index/query_index_hints.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_index/query_index_hints.html.md.erb b/geode-docs/developing/query_index/query_index_hints.html.md.erb
index 9911014..e461367 100644
--- a/geode-docs/developing/query_index/query_index_hints.html.md.erb
+++ b/geode-docs/developing/query_index/query_index_hints.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-You can use the hint keyword to allow Geode's query engine to prefer certain indexes.
+You can use the hint keyword to allow <%=vars.product_name%>'s query engine to prefer certain indexes.
 
 In cases where one index is hinted in a query, the query engine filters off the hinted index (if possible) and then iterates and filters from the resulting values.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_select/the_select_statement.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_select/the_select_statement.html.md.erb b/geode-docs/developing/query_select/the_select_statement.html.md.erb
index baaf6c1..d472889 100644
--- a/geode-docs/developing/query_select/the_select_statement.html.md.erb
+++ b/geode-docs/developing/query_select/the_select_statement.html.md.erb
@@ -80,7 +80,7 @@ When a struct is returned, the name of each field in the struct is determined fo
 
 ## <a id="concept_85AE7D6B1E2941ED8BD2A8310A81753E__section_972EE73A6F3E4427B6A99DB4EDF5860D" class="no-quick-link"></a>DISTINCT
 
-Use the DISTINCT keyword if you want to limit the results set to unique rows. Note that in the current version of Geode you are no longer required to use the DISTINCT keyword in your SELECT statement.
+Use the DISTINCT keyword if you want to limit the results set to unique rows. Note that in the current version of <%=vars.product_name%> you are no longer required to use the DISTINCT keyword in your SELECT statement.
 
 ``` pre
 SELECT DISTINCT * FROM /exampleRegion
@@ -124,7 +124,7 @@ If you are using ORDER BY queries, you must implement the equals and hashCode me
 
 ## <a id="concept_85AE7D6B1E2941ED8BD2A8310A81753E__section_69DCAD624E9640028BC86FD67649DEB2" class="no-quick-link"></a>Preset Query Functions
 
-Geode provides several built-in functions for evaluating or filtering data returned from a query. They include the following:
+<%=vars.product_name%> provides several built-in functions for evaluating or filtering data returned from a query. They include the following:
 
 <table>
 <colgroup>

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/query_select/the_where_clause.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/query_select/the_where_clause.html.md.erb b/geode-docs/developing/query_select/the_where_clause.html.md.erb
index 834bae9..6fe4498 100644
--- a/geode-docs/developing/query_select/the_where_clause.html.md.erb
+++ b/geode-docs/developing/query_select/the_where_clause.html.md.erb
@@ -162,7 +162,7 @@ SELECT * FROM /exampleRegion portfolio1, portfolio1.positions.values positions1,
 
 ## <a id="the_where_clause__section_D91E0B06FFF6431490CC0BFA369425AD" class="no-quick-link"></a>LIKE
 
-Geode offers limited support for the LIKE predicate. LIKE can be used to mean 'equals to'. If you terminate the string with a wildcard ('%'), it behaves like 'starts with'. You can also place a wildcard (either '%' or '\_') at any other position in the comparison string. You can escape the wildcard characters to represent the characters themselves.
+<%=vars.product_name%> offers limited support for the LIKE predicate. LIKE can be used to mean 'equals to'. If you terminate the string with a wildcard ('%'), it behaves like 'starts with'. You can also place a wildcard (either '%' or '\_') at any other position in the comparison string. You can escape the wildcard characters to represent the characters themselves.
 
 **Note:**
 The '\*' wildcard is not supported in OQL LIKE predicates.
@@ -318,7 +318,7 @@ One problem is that you cannot create indexes on Set or List types (collection t
 
 ## <a id="the_where_clause__section_E7206D045BEC4F67A8D2B793922BF213" class="no-quick-link"></a>Double.NaN and Float.NaN Comparisons
 
-The comparison behavior of Double.NaN and Float.NaN within Geode queries follow the semantics of the JDK methods Float.compareTo and Double.compareTo.
+The comparison behavior of Double.NaN and Float.NaN within <%=vars.product_name%> queries follow the semantics of the JDK methods Float.compareTo and Double.compareTo.
 
 In summary, the comparisons differ in the following ways from those performed by the Java language numerical comparison operators (<, <=, ==, >= >) when applied to primitive double [float] values:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/querying_basics/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/querying_basics/chapter_overview.html.md.erb b/geode-docs/developing/querying_basics/chapter_overview.html.md.erb
index 328cc46..b7291c8 100644
--- a/geode-docs/developing/querying_basics/chapter_overview.html.md.erb
+++ b/geode-docs/developing/querying_basics/chapter_overview.html.md.erb
@@ -27,11 +27,11 @@ Since Geode regions are key-value stores where values can range from simple byte
 
     This topic answers some frequently asked questions on querying functionality. It provides examples to help you get started with Geode querying.
 
--   **[Basic Querying](../../developing/querying_basics/query_basics.html)**
+-   **[Basic Querying](query_basics.html)**
 
     This section provides a high-level introduction to Geode querying such as building a query string and describes query language features.
 
--   **[Advanced Querying](../../developing/query_additional/advanced_querying.html)**
+-   **[Advanced Querying](../query_additional/advanced_querying.html)**
 
     This section includes advanced querying topics such as using query indexes, using query bind parameters, querying partitioned regions and query debugging.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/querying_basics/query_basics.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/querying_basics/query_basics.html.md.erb b/geode-docs/developing/querying_basics/query_basics.html.md.erb
index 4121140..b2928ff 100644
--- a/geode-docs/developing/querying_basics/query_basics.html.md.erb
+++ b/geode-docs/developing/querying_basics/query_basics.html.md.erb
@@ -19,12 +19,12 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-This section provides a high-level introduction to Geode querying such as building a query string and describes query language features.
+This section provides a high-level introduction to <%=vars.product_name%> querying such as building a query string and describes query language features.
 
 <a id="querying_with_oql__section_828A9660B5014DCAA883A58A45E6B51A"></a>
-Geode provides a SQL-like querying language that allows you to access data stored in Geode regions. Since Geode regions are key-value stores where values can range from simple byte arrays to complex nested objects, Geode uses a query syntax based on OQL (Object Query Language) to query region data. OQL and SQL have many syntactical similarities, however they have significant differences. For example, while OQL does not offer all of the capabilities of SQL like aggregates, OQL does allow you to execute queries on complex object graphs, query object attributes and invoke object methods.
+<%=vars.product_name%> provides a SQL-like querying language that allows you to access data stored in <%=vars.product_name%> regions. Since <%=vars.product_name%> regions are key-value stores where values can range from simple byte arrays to complex nested objects, <%=vars.product_name%> uses a query syntax based on OQL (Object Query Language) to query region data. OQL and SQL have many syntactical similarities, however they have significant differences. For example, while OQL does not offer all of the capabilities of SQL like aggregates, OQL does allow you to execute queries on complex object graphs, query object attributes and invoke object methods.
 
-The syntax of a typical Geode OQL query is:
+The syntax of a typical <%=vars.product_name%> OQL query is:
 
 ``` pre
 [IMPORT package]
@@ -34,24 +34,24 @@ FROM collection1, [collection2, …]
 [ORDER BY order_criteria [desc]]
 ```
 
-Therefore, a simple Geode OQL query resembles the following:
+Therefore, a simple <%=vars.product_name%> OQL query resembles the following:
 
 ``` pre
 SELECT DISTINCT * FROM /exampleRegion WHERE status = ‘active’
 ```
 
-An important characteristic of Geode querying to note is that by default, Geode queries on the values of a region and not on keys. To obtain keys from a region, you must use the keySet path expression on the queried region. For example, `/exampleRegion.keySet`.
+An important characteristic of <%=vars.product_name%> querying to note is that by default, <%=vars.product_name%> queries on the values of a region and not on keys. To obtain keys from a region, you must use the keySet path expression on the queried region. For example, `/exampleRegion.keySet`.
 
-For those new to the Geode querying, see also the [Geode Querying FAQ and Examples](../../getting_started/querying_quick_reference.html#reference_D5CE64F5FD6F4A808AEFB748C867189E).
+For those new to the <%=vars.product_name%> querying, see also the [<%=vars.product_name%> Querying FAQ and Examples](../../getting_started/querying_quick_reference.html#reference_D5CE64F5FD6F4A808AEFB748C867189E).
 
--   **[Advantages of OQL](../../developing/querying_basics/oql_compared_to_sql.html)**
+-   **[Advantages of OQL](oql_compared_to_sql.html)**
 
--   **[Writing and Executing a Query in Geode](../../developing/querying_basics/running_a_query.html)**
+-   **[Writing and Executing a Query in <%=vars.product_name%>](running_a_query.html)**
 
--   **[Building a Query String](../../developing/querying_basics/what_is_a_query_string.html)**
+-   **[Building a Query String](what_is_a_query_string.html)**
 
--   **[OQL Syntax and Semantics](../../developing/query_additional/query_language_features.html)**
+-   **[OQL Syntax and Semantics](../query_additional/query_language_features.html)**
 
--   **[Query Language Restrictions and Unsupported Features](../../developing/querying_basics/restrictions_and_unsupported_features.html)**
+-   **[Query Language Restrictions and Unsupported Features](restrictions_and_unsupported_features.html)**
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/querying_basics/querying_partitioned_regions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/querying_basics/querying_partitioned_regions.html.md.erb b/geode-docs/developing/querying_basics/querying_partitioned_regions.html.md.erb
index 0105d82..882fb9a 100644
--- a/geode-docs/developing/querying_basics/querying_partitioned_regions.html.md.erb
+++ b/geode-docs/developing/querying_basics/querying_partitioned_regions.html.md.erb
@@ -19,23 +19,23 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode allows you to manage and store large amounts of data across distributed nodes using partitioned regions. The basic unit of storage for a partitioned region is a bucket, which resides on a Geode node and contains all the entries that map to a single hashcode. In a typical partitioned region query, the system distributes the query to all buckets across all nodes, then merges the result sets and sends back the query results.
+<%=vars.product_name%> allows you to manage and store large amounts of data across distributed nodes using partitioned regions. The basic unit of storage for a partitioned region is a bucket, which resides on a <%=vars.product_name%> node and contains all the entries that map to a single hashcode. In a typical partitioned region query, the system distributes the query to all buckets across all nodes, then merges the result sets and sends back the query results.
 
 <a id="querying_partitioned_regions__section_4C603563DEDC4303818FB8F894470457"></a>
-The following list summarizes the querying functionality supported by Geode for partitioned regions:
+The following list summarizes the querying functionality supported by <%=vars.product_name%> for partitioned regions:
 
 -   **Ability to target specific nodes in a query**. If you know that a specific bucket contains the data that you want to query, you can use a function to ensure that your query only runs the specific node that holds the data. This can greatly improve query efficiency. The ability to query data on a specific node is only available if you are using functions and if the function is executed on one single region. In order to do this, you need to use `Query.execute(RegionFunctionContext context)`. See the [Java API](/releases/latest/javadoc/org/apache/geode/cache/query/Query.html) and [Querying a Partitioned Region on a Single Node](../query_additional/query_on_a_single_node.html#concept_30B18A6507534993BD55C2C9E0544A97) for more details.
--   **Ability to optimize partitioned region query performance using key indexes**. You can improve query performance on data that is partitioned by key or a field value by creating a key index and then executing the query using use `Query.execute(RegionFunctionContext                         context)` with the key or field value used as filter. See the [Java API](/releases/latest/javadoc/org/apache/geode/cache/query/Query.html) and [Optimizing Queries on Data Partitioned by a Key or Field Value](../query_additional/partitioned_region_key_or_field_value.html#concept_3010014DFBC9479783B2B45982014454) for more details.
+-   **Ability to optimize partitioned region query performance using key indexes**. You can improve query performance on data that is partitioned by key or a field value by creating a key index and then executing the query using use `Query.execute(RegionFunctionContext context)` with the key or field value used as filter. See the [Java API](/releases/latest/javadoc/org/apache/geode/cache/query/Query.html) and [Optimizing Queries on Data Partitioned by a Key or Field Value](../query_additional/partitioned_region_key_or_field_value.html#concept_3010014DFBC9479783B2B45982014454) for more details.
 -   **Ability to perform equi-join queries between partitioned regions and between partitioned regions and replicated regions**. Join queries between partitioned region and between partitioned regions and replicated regions are supported through the function service. In order to perform equi-join operations on partitioned regions or partitioned regions and replicated regions, the partitioned regions must be colocated, and you need to use the need to use `Query.execute(RegionFunctionContext                         context)`. See the [Java API](/releases/latest/javadoc/org/apache/geode/cache/query/Query.html) and [Performing an Equi-Join Query on Partitioned Regions](../partitioned_regions/join_query_partitioned_regions.html#concept_B930D276F49541F282A2CFE639F107DD) for more details.
 
--   **[Using ORDER BY on Partitioned Regions](../../developing/query_additional/order_by_on_partitioned_regions.html)**
+-   **[Using ORDER BY on Partitioned Regions](../query_additional/order_by_on_partitioned_regions.html)**
 
--   **[Querying a Partitioned Region on a Single Node](../../developing/query_additional/query_on_a_single_node.html)**
+-   **[Querying a Partitioned Region on a Single Node](../query_additional/query_on_a_single_node.html)**
 
--   **[Optimizing Queries on Data Partitioned by a Key or Field Value](../../developing/query_additional/partitioned_region_key_or_field_value.html)**
+-   **[Optimizing Queries on Data Partitioned by a Key or Field Value](../query_additional/partitioned_region_key_or_field_value.html)**
 
--   **[Performing an Equi-Join Query on Partitioned Regions](../../developing/partitioned_regions/join_query_partitioned_regions.html)**
+-   **[Performing an Equi-Join Query on Partitioned Regions](../partitioned_regions/join_query_partitioned_regions.html)**
 
--   **[Partitioned Region Query Restrictions](../../developing/query_additional/partitioned_region_query_restrictions.html)**
+-   **[Partitioned Region Query Restrictions](../query_additional/partitioned_region_query_restrictions.html)**
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/querying_basics/reserved_words.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/querying_basics/reserved_words.html.md.erb b/geode-docs/developing/querying_basics/reserved_words.html.md.erb
index 7a23f91..67829b9 100644
--- a/geode-docs/developing/querying_basics/reserved_words.html.md.erb
+++ b/geode-docs/developing/querying_basics/reserved_words.html.md.erb
@@ -21,7 +21,7 @@ limitations under the License.
 
 ## <a id="concept_4F288B1F9579422FA481FBE2C3ADD007__section_3415163C3EFB46A6BE873E2606C9DE0F" class="no-quick-link"></a>Reserved Words
 
-These words are reserved for the query language and may not be used as identifiers. The words with asterisk (`*`) after them are not currently used by Geode, but are reserved for future implementation.
+These words are reserved for the query language and may not be used as identifiers. The words with asterisk (`*`) after them are not currently used by <%=vars.product_name%>, but are reserved for future implementation.
 
 <table>
 <colgroup>

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/querying_basics/restrictions_and_unsupported_features.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/querying_basics/restrictions_and_unsupported_features.html.md.erb b/geode-docs/developing/querying_basics/restrictions_and_unsupported_features.html.md.erb
index d315461..0927a04 100644
--- a/geode-docs/developing/querying_basics/restrictions_and_unsupported_features.html.md.erb
+++ b/geode-docs/developing/querying_basics/restrictions_and_unsupported_features.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-At a high level, Geode does not support the following querying features:
+At a high level, <%=vars.product_name%> does not support the following querying features:
 
 -   Indexes targeted for joins across more than one region are not supported
 -   Static method invocations. For example, the following query is invalid:

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/querying_basics/running_a_query.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/querying_basics/running_a_query.html.md.erb b/geode-docs/developing/querying_basics/running_a_query.html.md.erb
index 6ddb1de..985a231 100644
--- a/geode-docs/developing/querying_basics/running_a_query.html.md.erb
+++ b/geode-docs/developing/querying_basics/running_a_query.html.md.erb
@@ -1,6 +1,4 @@
----
-title:  Writing and Executing a Query in Geode
----
+<% set_title("Writing and Executing a Query in", product_name) %>
 
 <!--
 Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,7 +18,7 @@ limitations under the License.
 -->
 
 <a id="running_a_querying__section_C285160AF91C4486A39444C3A22D6475"></a>
-The Geode QueryService provides methods to create the Query object. You can then use the Query object to perform query-related operations.
+The <%=vars.product_name%> QueryService provides methods to create the Query object. You can then use the Query object to perform query-related operations.
 
 The QueryService instance you should use depends on whether you are querying the local cache of an application or if you want your application to query the server cache.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/querying_basics/supported_character_sets.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/querying_basics/supported_character_sets.html.md.erb b/geode-docs/developing/querying_basics/supported_character_sets.html.md.erb
index 50b9c87..e5db399 100644
--- a/geode-docs/developing/querying_basics/supported_character_sets.html.md.erb
+++ b/geode-docs/developing/querying_basics/supported_character_sets.html.md.erb
@@ -19,6 +19,6 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode query language supports the full ASCII and Unicode character sets.
+<%=vars.product_name%> query language supports the full ASCII and Unicode character sets.
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/querying_basics/what_is_a_query_string.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/querying_basics/what_is_a_query_string.html.md.erb b/geode-docs/developing/querying_basics/what_is_a_query_string.html.md.erb
index eb79645..f12729b 100644
--- a/geode-docs/developing/querying_basics/what_is_a_query_string.html.md.erb
+++ b/geode-docs/developing/querying_basics/what_is_a_query_string.html.md.erb
@@ -37,14 +37,14 @@ A query string follows the rules specified by the query language and grammar. It
 
 The components listed above can all be part of the query string, but none of the components are required. At a minimum, a query string contains an expression that can be evaluated against specified data.
 
-The following sections provide guidelines for the query language building blocks that are used when writing typical Geode queries.
+The following sections provide guidelines for the query language building blocks that are used when writing typical <%=vars.product_name%> queries.
 
--   **[IMPORT Statement](../../developing/query_select/the_import_statement.html)**
+-   **[IMPORT Statement](../query_select/the_import_statement.html)**
 
--   **[FROM Clause](../../developing/query_select/the_from_clause.html)**
+-   **[FROM Clause](../query_select/the_from_clause.html)**
 
--   **[WHERE Clause](../../developing/query_select/the_where_clause.html)**
+-   **[WHERE Clause](../query_select/the_where_clause.html)**
 
--   **[SELECT Statement](../../developing/query_select/the_select_statement.html)**
+-   **[SELECT Statement](../query_select/the_select_statement.html)**
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/region_options/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/region_options/chapter_overview.html.md.erb b/geode-docs/developing/region_options/chapter_overview.html.md.erb
index e48ac79..be1ac36 100644
--- a/geode-docs/developing/region_options/chapter_overview.html.md.erb
+++ b/geode-docs/developing/region_options/chapter_overview.html.md.erb
@@ -19,21 +19,21 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-The Apache Geode data storage and distribution models put your data in the right place at the right time. You should understand all the options for data storage in Geode before you configure your data regions.
+The <%=vars.product_name_long%> data storage and distribution models put your data in the right place at the right time. You should understand all the options for data storage in <%=vars.product_name%> before you configure your data regions.
 
--   **[Storage and Distribution Options](../../developing/region_options/storage_distribution_options.html)**
+-   **[Storage and Distribution Options](storage_distribution_options.html)**
 
-    Geode provides several models for data storage and distribution, including partitioned or replicated regions as well as distributed or non-distributed regions (local cache storage).
+    <%=vars.product_name%> provides several models for data storage and distribution, including partitioned or replicated regions as well as distributed or non-distributed regions (local cache storage).
 
--   **[Region Types](../../developing/region_options/region_types.html)**
+-   **[Region Types](region_types.html)**
 
     Region types define region behavior within a single distributed system. You have various options for region data storage and distribution.
 
--   **[Region Data Stores and Data Accessors](../../developing/region_options/data_hosts_and_accessors.html)**
+-   **[Region Data Stores and Data Accessors](data_hosts_and_accessors.html)**
 
     Understand the difference between members that store data for a region and members that act only as data accessors to the region.
 
--   **[Creating Regions Dynamically](../../developing/region_options/dynamic_region_creation.html)**
+-   **[Creating Regions Dynamically](dynamic_region_creation.html)**
 
     You can dynamically create regions in your application code and automatically instantiate them on members of a distributed system.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/region_options/dynamic_region_creation.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/region_options/dynamic_region_creation.html.md.erb b/geode-docs/developing/region_options/dynamic_region_creation.html.md.erb
index 2652e3d..c26422f 100644
--- a/geode-docs/developing/region_options/dynamic_region_creation.html.md.erb
+++ b/geode-docs/developing/region_options/dynamic_region_creation.html.md.erb
@@ -25,7 +25,7 @@ If your application does not require partitioned regions, you can use the <span
 
 Due to the number of options involved, most developers use functions to create regions dynamically in their applications, as described in this topic. Dynamic regions can also be created from the `gfsh` command line.
 
-For a complete discussion of using Geode functions, see [Function Execution](../function_exec/chapter_overview.html). Functions use the <span class="keyword apiname">org.apache.geode.cache.execute.FunctionService</span> class.
+For a complete discussion of using <%=vars.product_name%> functions, see [Function Execution](../function_exec/chapter_overview.html). Functions use the <span class="keyword apiname">org.apache.geode.cache.execute.FunctionService</span> class.
 
 For example, the following Java classes define and use a function for dynamic region creation:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/region_options/region_types.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/region_options/region_types.html.md.erb b/geode-docs/developing/region_options/region_types.html.md.erb
index 57e565e..e12bb24 100644
--- a/geode-docs/developing/region_options/region_types.html.md.erb
+++ b/geode-docs/developing/region_options/region_types.html.md.erb
@@ -22,7 +22,7 @@ limitations under the License.
 Region types define region behavior within a single distributed system. You have various options for region data storage and distribution.
 
 <a id="region_types__section_E3435ED1D0D142538B99FA69A9E449EF"></a>
-Within a Geode distributed system, you can define distributed regions and non-distributed regions, and you can define regions whose data is spread across the distributed system, and regions whose data is entirely contained in a single member.
+Within a <%=vars.product_name%> distributed system, you can define distributed regions and non-distributed regions, and you can define regions whose data is spread across the distributed system, and regions whose data is entirely contained in a single member.
 
 Your choice of region type is governed in part by the type of application you are running. In particular, you need to use specific region types for your servers and clients for effective communication between the two tiers:
 
@@ -102,8 +102,8 @@ Partitioned regions group your data into buckets, each of which is stored on a s
 
 Use partitioning for:
 
--   **Large data sets**. Store data sets that are too large to fit into a single member, and all members will see the same logical data set. Partitioned regions divide the data into units of storage called buckets that are split across the members hosting the partitioned region data, so no member needs to host all of the region’s data. Geode provides dynamic redundancy recovery and rebalancing of partitioned regions, making them the choice for large-scale data containers. More members in the system can accommodate more uniform balancing of the data across all host members, allowing system throughput (both gets and puts) to scale as new members are added.
--   **High availability**. Partitioned regions allow you configure the number of copies of your data that Geode should make. If a member fails, your data will be available without interruption from the remaining members. Partitioned regions can also be persisted to disk for additional high availability.
+-   **Large data sets**. Store data sets that are too large to fit into a single member, and all members will see the same logical data set. Partitioned regions divide the data into units of storage called buckets that are split across the members hosting the partitioned region data, so no member needs to host all of the region’s data. <%=vars.product_name%> provides dynamic redundancy recovery and rebalancing of partitioned regions, making them the choice for large-scale data containers. More members in the system can accommodate more uniform balancing of the data across all host members, allowing system throughput (both gets and puts) to scale as new members are added.
+-   **High availability**. Partitioned regions allow you configure the number of copies of your data that <%=vars.product_name%> should make. If a member fails, your data will be available without interruption from the remaining members. Partitioned regions can also be persisted to disk for additional high availability.
 -   **Scalability**. Partitioned regions can scale to large amounts of data because the data is divided between the members available to host the region. Increase your data capacity dynamically by simply adding new members. Partitioned regions also allow you to scale your processing capacity. Because your entries are spread out across the members hosting the region, reads and writes to those entries are also spread out across those members.
 -   **Good write performance**. You can configure the number of copies of your data. The amount of data transmitted per write does not increase with the number of members. By contrast, with replicated regions, each write must be sent to every member that has the region replicated, so the amount of data transmitted per write increases with the number of members.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/region_options/storage_distribution_options.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/region_options/storage_distribution_options.html.md.erb b/geode-docs/developing/region_options/storage_distribution_options.html.md.erb
index f30135e..6cbafc6 100644
--- a/geode-docs/developing/region_options/storage_distribution_options.html.md.erb
+++ b/geode-docs/developing/region_options/storage_distribution_options.html.md.erb
@@ -19,11 +19,11 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-Geode provides several models for data storage and distribution, including partitioned or replicated regions as well as distributed or non-distributed regions (local cache storage).
+<%=vars.product_name%> provides several models for data storage and distribution, including partitioned or replicated regions as well as distributed or non-distributed regions (local cache storage).
 
 ## <a id="concept_B18B7754E7C7485BA6D66F2DDB7A11FB__section_787D674A64244871AE49CBB58475088E" class="no-quick-link"></a>Peer-to-Peer Region Storage and Distribution
 
-At its most general, data management means having current data available when and where your applications need it. In a properly configured Geode installation, you store your data in your local members and Geode automatically distributes it to the other members that need it according to your cache configuration settings. You may be storing very large data objects that require special consideration, or you may have a high volume of data requiring careful configuration to safeguard your application's performance or memory use. You may need to be able to explicitly lock some data during particular operations. Most data management features are available as configuration options, which you can specify either using the `gfsh` cluster configuration service, `cache.xml` file or the API. Once configured, Geode manages the data automatically. For example, this is how you manage data distribution, disk storage, data expiration activities, and data partitioning. A few features are managed at ru
 n-time through the API.
+At its most general, data management means having current data available when and where your applications need it. In a properly configured <%=vars.product_name%> installation, you store your data in your local members and <%=vars.product_name%> automatically distributes it to the other members that need it according to your cache configuration settings. You may be storing very large data objects that require special consideration, or you may have a high volume of data requiring careful configuration to safeguard your application's performance or memory use. You may need to be able to explicitly lock some data during particular operations. Most data management features are available as configuration options, which you can specify either using the `gfsh` cluster configuration service, `cache.xml` file or the API. Once configured, <%=vars.product_name%> manages the data automatically. For example, this is how you manage data distribution, disk storage, data expiration activities, and 
 data partitioning. A few features are managed at run-time through the API.
 
 At the architectural level, data distribution runs between peers in a single system and between clients and servers.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/storing_data_on_disk/chapter_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/storing_data_on_disk/chapter_overview.html.md.erb b/geode-docs/developing/storing_data_on_disk/chapter_overview.html.md.erb
index 002fb53..d41b009 100644
--- a/geode-docs/developing/storing_data_on_disk/chapter_overview.html.md.erb
+++ b/geode-docs/developing/storing_data_on_disk/chapter_overview.html.md.erb
@@ -24,17 +24,17 @@ You can persist data on disk for backup purposes and overflow it to disk to free
 **Note:**
 This supplements the general steps for managing data regions provided in [Basic Configuration and Programming](../../basic_config/book_intro.html).
 
-All disk storage uses Apache Geode[Disk Storage](../../managing/disk_storage/chapter_overview.html).
+All disk storage uses <%=vars.product_name_long%> [Disk Storage](../../managing/disk_storage/chapter_overview.html).
 
--   **[How Persistence and Overflow Work](../../developing/storing_data_on_disk/how_persist_overflow_work.html)**
+-   **[How Persistence and Overflow Work](how_persist_overflow_work.html)**
 
-    To use Geode persistence and overflow, you should understand how they work with your data.
+    To use <%=vars.product_name%> persistence and overflow, you should understand how they work with your data.
 
--   **[Configure Region Persistence and Overflow](../../developing/storing_data_on_disk/storing_data_on_disk.html)**
+-   **[Configure Region Persistence and Overflow](storing_data_on_disk.html)**
 
     Plan persistence and overflow for your data regions and configure them accordingly.
 
--   **[Overflow Configuration Examples](../../developing/storing_data_on_disk/overflow_config_examples.html)**
+-   **[Overflow Configuration Examples](overflow_config_examples.html)**
 
     The `cache.xml` examples show configuration of region and server subscription queue overflows.
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed9a8fd4/geode-docs/developing/storing_data_on_disk/how_persist_overflow_work.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/storing_data_on_disk/how_persist_overflow_work.html.md.erb b/geode-docs/developing/storing_data_on_disk/how_persist_overflow_work.html.md.erb
index 1a1cc10..89f63a3 100644
--- a/geode-docs/developing/storing_data_on_disk/how_persist_overflow_work.html.md.erb
+++ b/geode-docs/developing/storing_data_on_disk/how_persist_overflow_work.html.md.erb
@@ -19,14 +19,14 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-To use Geode persistence and overflow, you should understand how they work with your data.
+To use <%=vars.product_name%> persistence and overflow, you should understand how they work with your data.
 
 <a id="how_persist_overflow_work__section_jzl_wwb_pr"></a>
-Geode persists and overflows several types of data. You can persist or overflow the application data in your regions. In addition, Geode persists and overflows messaging queues between distributed systems, to manage memory consumption and provide high availability.
+<%=vars.product_name%> persists and overflows several types of data. You can persist or overflow the application data in your regions. In addition, <%=vars.product_name%> persists and overflows messaging queues between distributed systems, to manage memory consumption and provide high availability.
 
 Persistent data outlives the member where the region resides and can be used to initialize the region at creation. Overflow acts only as an extension of the region in memory.
 
-The data is written to disk according to the configuration of Geode disk stores. For any disk option, you can specify the name of the disk store to use or use the Geode default disk store. See [Disk Storage](../../managing/disk_storage/chapter_overview.html).
+The data is written to disk according to the configuration of <%=vars.product_name%> disk stores. For any disk option, you can specify the name of the disk store to use or use the <%=vars.product_name%> default disk store. See [Disk Storage](../../managing/disk_storage/chapter_overview.html).
 
 ## <a id="how_persist_overflow_work__section_78F2D1820B6C48859A0E5411CE360105" class="no-quick-link"></a>How Data Is Persisted and Overflowed
 


[09/11] geode git commit: Geode-3466 User Guide: Add WAN caveat to Delta Propagation section

Posted by zh...@apache.org.
Geode-3466 User Guide: Add WAN caveat to Delta Propagation section


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

Branch: refs/heads/feature/GEM-1601
Commit: 36daa9a14a4aca9d6418f7173a3293e2e4c687fd
Parents: ed9a8fd
Author: Dave Barnes <db...@pivotal.io>
Authored: Fri Aug 18 15:25:32 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Fri Aug 18 15:25:32 2017 -0700

----------------------------------------------------------------------
 .../delta_propagation/how_delta_propagation_works.html.md.erb      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/36daa9a1/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb b/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb
index 3609734..fa13a1c 100644
--- a/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb
+++ b/geode-docs/developing/delta_propagation/how_delta_propagation_works.html.md.erb
@@ -66,4 +66,4 @@ The following topologies support delta propagation (with some limitations):
     -   When the client's `gemfire.properties` setting `conflate-events` is set to true, the servers send full values for all regions.
     -   When the server region attribute `enable-subscription-conflation` is set to true and the client `gemfire.properties` setting `conflate-events` is set to `server`, the servers send full values for the region.
     -   When the client region is configured with the `PROXY` client region shortcut setting (empty client region), servers send full values.
-
+-   **Multi-site (WAN)**. Gateway senders do not send Deltas. The full value is always sent.