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

[01/19] incubator-geode git commit: GEODE-17: pulse not started with gemfire security enabled

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-1392 67d5e8de6 -> 22907b952


GEODE-17: pulse not started with gemfire security enabled


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

Branch: refs/heads/feature/GEODE-1392
Commit: fa2d963e0203bb1af6e8a760e9ef84e5fa80780b
Parents: c35f76f
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri May 13 07:20:14 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:22 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/management/internal/ManagementAgent.java | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa2d963e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
index adc69c4..af35c1b 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
@@ -201,10 +201,9 @@ public class ManagementAgent {
         if (logger.isDebugEnabled()) {
           logger.debug(message);
         }
-
-        if (isCustomAuthorizer()){
-          System.setProperty("spring.profiles.active", "pulse.authentication.gemfire");
-        }
+      }
+      else if (isCustomAuthenticator()){
+        System.setProperty("spring.profiles.active", "pulse.authentication.gemfire");
       }
 
       // Find developer REST WAR file


[14/19] incubator-geode git commit: GEODE-1222: remove InsufficientDiskSpaceException

Posted by kl...@apache.org.
GEODE-1222: remove InsufficientDiskSpaceException


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

Branch: refs/heads/feature/GEODE-1392
Commit: c54227c16d22f94c993136c399ec756f2b34a24c
Parents: 79b97aa
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed May 11 16:05:48 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:25 2016 -0700

----------------------------------------------------------------------
 .../InsufficientDiskSpaceException.java         | 53 --------------------
 .../gemstone/gemfire/internal/cache/Oplog.java  |  7 ++-
 .../codeAnalysis/sanctionedSerializables.txt    |  1 -
 3 files changed, 3 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c54227c1/geode-core/src/main/java/com/gemstone/gemfire/internal/InsufficientDiskSpaceException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/InsufficientDiskSpaceException.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/InsufficientDiskSpaceException.java
deleted file mode 100644
index be87c8b..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/InsufficientDiskSpaceException.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package com.gemstone.gemfire.internal;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import com.gemstone.gemfire.cache.DiskAccessException;
-import com.gemstone.gemfire.cache.DiskStore;
-import com.gemstone.gemfire.internal.Sendable;
-
-/**
- * @since gfxd 1.0.1
- */
-public class InsufficientDiskSpaceException extends DiskAccessException implements Sendable {
-  private static final long serialVersionUID = -6167707908956900841L;
-
-  public InsufficientDiskSpaceException(String msg, Throwable cause, DiskStore ds) {
-    super(msg, cause, ds);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void sendTo(DataOutput out) throws IOException {
-    // send base DiskAccessException to older versions
-    Version peerVersion = InternalDataSerializer.getVersionForDataStream(out);
-    if (Version.GFE_80.compareTo(peerVersion) > 0) {
-      DiskAccessException dae = new DiskAccessException(getMessage(),
-          getCause());
-      InternalDataSerializer.writeSerializableObject(dae, out);
-    }
-    else {
-      InternalDataSerializer.writeSerializableObject(this, out);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c54227c1/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
index 4728594..96f616b 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
@@ -77,7 +77,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.ByteArrayDataInput;
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.InsufficientDiskSpaceException;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalStatisticsDisabledException;
 import com.gemstone.gemfire.internal.Sendable;
@@ -603,7 +602,7 @@ public final class Oplog implements CompactableOplog, Flushable {
     long availableSpace = this.dirHolder.getAvailableSpace();
     if (availableSpace < maxOplogSizeParam) {
       if (DiskStoreImpl.PREALLOCATE_OPLOGS && !DiskStoreImpl.SET_IGNORE_PREALLOCATE) {
-        throw new InsufficientDiskSpaceException(LocalizedStrings.Oplog_PreAllocate_Failure_Init.toLocalizedString(this.dirHolder,
+        throw new DiskAccessException(LocalizedStrings.Oplog_PreAllocate_Failure_Init.toLocalizedString(this.dirHolder,
             maxOplogSizeParam), new IOException("not enough space left to create and pre grow oplog files, available="
             + availableSpace + ", required=" + maxOplogSizeParam), getParent());
       }
@@ -1070,13 +1069,13 @@ public final class Oplog implements CompactableOplog, Flushable {
          * false; } } }
          */
         closeAndDeleteAfterEx(ioe, olf);
-        throw new InsufficientDiskSpaceException(LocalizedStrings.Oplog_PreAllocate_Failure.toLocalizedString(olf.f
+        throw new DiskAccessException(LocalizedStrings.Oplog_PreAllocate_Failure.toLocalizedString(olf.f
             .getAbsolutePath(), maxSize), ioe, getParent());
       }
     }
     // TODO: Perhaps the test flag is not requierd here. Will re-visit.
     else if (DiskStoreImpl.PREALLOCATE_OPLOGS && !DiskStoreImpl.SET_IGNORE_PREALLOCATE) {
-      throw new InsufficientDiskSpaceException(LocalizedStrings.Oplog_PreAllocate_Failure.toLocalizedString(
+      throw new DiskAccessException(LocalizedStrings.Oplog_PreAllocate_Failure.toLocalizedString(
           olf.f.getAbsolutePath(), maxSize), new IOException("not enough space left to pre-blow, available=" + availableSpace
           + ", required=" + maxSize), getParent());
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c54227c1/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
index fa02ca3..1ee16d5 100755
--- a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
@@ -246,7 +246,6 @@ com/gemstone/gemfire/internal/ConfigSource$Type,false
 com/gemstone/gemfire/internal/CopyOnWriteHashSet,true,8591978652141659932
 com/gemstone/gemfire/internal/DSFIDFactory$SqlfSerializationException,true,5076687296705595933
 com/gemstone/gemfire/internal/DSFIDNotFoundException,true,130596009484324655,dsfid:int,versionOrdinal:short
-com/gemstone/gemfire/internal/InsufficientDiskSpaceException,true,-6167707908956900841
 com/gemstone/gemfire/internal/InternalDataSerializer$SERIALIZATION_VERSION,false
 com/gemstone/gemfire/internal/InternalStatisticsDisabledException,true,4146181546364258311
 com/gemstone/gemfire/internal/LinuxProcFsStatistics$CPU,false


[11/19] incubator-geode git commit: GEODE-1351: let the waitForFlush() to return false if timeout

Posted by kl...@apache.org.
GEODE-1351: let the waitForFlush() to return false if timeout

also apply it on some unit tests.


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

Branch: refs/heads/feature/GEODE-1392
Commit: 758643cefcbffb29b8964535c1e08522e60ac436
Parents: 8ea5320
Author: zhouxh <gz...@pivotal.io>
Authored: Mon May 16 17:12:15 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:25 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneIndex.java       |  4 +++-
 .../cache/lucene/internal/LuceneIndexImpl.java  | 11 ++++-----
 .../internal/xml/LuceneIndexCreation.java       |  3 ++-
 .../LuceneIndexCreationIntegrationTest.java     | 25 ++++++++++----------
 .../LuceneIndexRecoveryHAIntegrationTest.java   | 14 +++++------
 5 files changed, 29 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/758643ce/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
index be329f7..6b1a4b4 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
@@ -58,7 +58,9 @@ public interface LuceneIndex {
   
   /* 
    * wait until the current entries in cache are indexed
+   * @param maxWaitInMilliseconds max wait time in millisecond
+   * @return if entries are flushed within maxWait
    */
-  public void waitUntilFlushed(int maxWait);
+  public boolean waitUntilFlushed(int maxWaitInMillisecond);
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/758643ce/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
index 981d9e4..c165085 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
@@ -81,19 +81,17 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     searchableFieldNames = fields;
   }
 
-  /*
-   *  For test and demo purpose. To use it, the data region should stop feeding
-   *  A more advanced version is under-development
-   */
   @Override
-  public void waitUntilFlushed(int maxWait) {
+  public boolean waitUntilFlushed(int maxWaitInMillisecond) {
     String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath);
     AsyncEventQueue queue = (AsyncEventQueue)cache.getAsyncEventQueue(aeqId);
+    boolean flushed = false;
     if (queue != null) {
       long start = System.nanoTime();
-      while (System.nanoTime() - start < TimeUnit.MILLISECONDS.toNanos(maxWait)) {
+      while (System.nanoTime() - start < TimeUnit.MILLISECONDS.toNanos(maxWaitInMillisecond)) {
         if (0 == queue.size()) {
           logger.debug("waitUntilFlushed: Queue size is 0");
+          flushed = true;
           break;
         } else {
           try {
@@ -105,6 +103,7 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     } else { 
       throw new IllegalArgumentException("The AEQ does not exist for the index "+indexName+" region "+regionPath);
     }
+    return flushed;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/758643ce/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
index b54f51b..a3bdd24 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
@@ -116,6 +116,7 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
   }
 
   @Override
-  public void waitUntilFlushed(int maxWait) {
+  public boolean waitUntilFlushed(int maxWaitInMillisecond) {
+    return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/758643ce/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index 6429143..fe754a4 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -44,7 +44,6 @@ import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import com.jayway.awaitility.Awaitility;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
@@ -71,12 +70,10 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     Region region = createRegion();
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     region.put("key1", new TestObject());
-
+    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     assertEquals(analyzers, index.getFieldAnalyzers());
-    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
-      assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
-      assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
-    });
+    assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
+    assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
   }
 
   @Test
@@ -188,26 +185,28 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     cache.close();
     createCache();
     createIndex("field1", "field2");
+    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
     LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
       .create(INDEX_NAME, REGION_NAME,
         "field1:world");
-    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
-      assertEquals(1, query.search().size());
-    });
+    assertEquals(1, query.search().size());
   }
 
+  private void verifyIndexFinishFlushing(String indexName, String regionName) {
+    LuceneIndex index = luceneService.getIndex(indexName, regionName);
+    boolean flushed = index.waitUntilFlushed(60000);
+    assertTrue(flushed);
+  }
+  
   @Test
   public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
     createIndex("field1", "field2");
     Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
     dataRegion.put("A", new TestObject());
-    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
-
-    //Wait until the queue has drained
-    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, queue.size()));
+    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     cache.close();
     createCache();
     createIndex("text");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/758643ce/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
index 77d2a5c..d32e6d8 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
@@ -41,6 +41,7 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
@@ -55,7 +56,6 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import com.jayway.awaitility.Awaitility;
 
 @Category(IntegrationTest.class)
 public class LuceneIndexRecoveryHAIntegrationTest {
@@ -139,7 +139,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     value = new Type1("lucene world", 1, 2L, 3.0, 4.0f);
     userRegion.put("value3", value);
 
-    waitUntilQueueEmpty(aeqId);
+    verifyIndexFinishFlushing(INDEX, REGION);
 
     LuceneQuery<Integer, Type1> query = service.createLuceneQueryFactory().create(INDEX, REGION, "s:world");
     LuceneQueryResults<Integer, Type1> results = query.search();
@@ -190,7 +190,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     value = new Type1("lucene world", 1, 2L, 3.0, 4.0f);
     userRegion.put("value3", value);
 
-    waitUntilQueueEmpty(aeqId);
+    verifyIndexFinishFlushing(INDEX, REGION);
 
     PartitionedRegion fileRegion = (PartitionedRegion) cache.getRegion(aeqId + ".files");
     assertNotNull(fileRegion);
@@ -203,9 +203,9 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     Assert.assertEquals(3, results.size());
   }
 
-  private void waitUntilQueueEmpty(final String aeqId) {
-    // TODO flush queue
-    AsyncEventQueue queue = cache.getAsyncEventQueue(aeqId);
-    Awaitility.waitAtMost(1000, TimeUnit.MILLISECONDS).until(() -> assertEquals(0, queue.size()));
+  private void verifyIndexFinishFlushing(String indexName, String regionName) {
+    LuceneIndex index = LuceneServiceProvider.get(cache).getIndex(indexName, regionName);
+    boolean flushed = index.waitUntilFlushed(60000);
+    assertTrue(flushed);
   }
 }


[05/19] incubator-geode git commit: GEODE-1390: Disabling broken test

Posted by kl...@apache.org.
GEODE-1390: Disabling broken test


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

Branch: refs/heads/feature/GEODE-1392
Commit: 6a9b0aecf4578b95edc4daac905504da245a5369
Parents: a49978e
Author: Jens Deppe <jd...@pivotal.io>
Authored: Sat May 14 15:14:12 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:23 2016 -0700

----------------------------------------------------------------------
 .../cli/commands/HelpCommandsJUnitTest.java       | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6a9b0aec/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
index 7cbe85a..1d81d2e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
@@ -25,7 +25,9 @@ import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.shell.GfshConfig;
 import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -41,8 +43,10 @@ public class HelpCommandsJUnitTest extends JUnit4DistributedTestCase {
 
   private int jmxPort;
 
+  private Gfsh gfsh;
+
   @Before
-  public void setup() {
+  public void setup() throws Exception {
     jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
     Properties localProps = new Properties();
@@ -51,15 +55,22 @@ public class HelpCommandsJUnitTest extends JUnit4DistributedTestCase {
     localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
     getSystem(localProps);
 
+    gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
   }
 
+  @After
+  public void teardown() {
+    disconnectAllFromDS();
+
+    gfsh.executeCommand("disconnect");
+  }
+
+  @Ignore("Disconnect command doesn't appear to be working")
   @Test
   public void testOfflineHelp() throws Exception {
     Properties helpProps = new Properties();
     helpProps.load(HelpCommandsJUnitTest.class.getResourceAsStream("golden-help-offline.properties"));
 
-    Gfsh gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
-
     CommandManager cm = CommandManager.getInstance();
     for (Map.Entry<String, CommandTarget> e : cm.getCommands().entrySet()) {
       // Mock commands may have been produced in the VM by other tests
@@ -88,7 +99,6 @@ public class HelpCommandsJUnitTest extends JUnit4DistributedTestCase {
     Properties helpProps = new Properties();
     helpProps.load(HelpCommandsJUnitTest.class.getResourceAsStream("golden-help-online.properties"));
 
-    Gfsh gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
     gfsh.executeCommand("connect --jmx-manager=localhost[" + jmxPort + "]");
 
     CommandManager cm = CommandManager.getInstance();


[19/19] incubator-geode git commit: GEODE-11: Split out lucene tests of persistence and off heap

Posted by kl...@apache.org.
GEODE-11: Split out lucene tests of persistence and off heap

Persistence and offheap require some special before and after code to
create and clean up the resources. Splitting these to separate
integration tests.


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

Branch: refs/heads/feature/GEODE-1392
Commit: 22907b952626d2b05a5447826315daecf1158567
Parents: 5576aca
Author: Dan Smith <up...@apache.org>
Authored: Fri May 13 09:51:43 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:27 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexCreationIntegrationTest.java     | 159 ++++---------------
 ...ceneIndexCreationOffHeapIntegrationTest.java |  72 +++++++++
 ...IndexCreationPersistenceIntegrationTest.java | 141 ++++++++++++++++
 .../cache/lucene/LuceneIntegrationTest.java     |  20 ++-
 .../cache/lucene/test/LuceneTestUtilities.java  |  64 ++++++++
 .../gemfire/cache/lucene/test/TestObject.java   |  27 ++++
 .../gemfire/cache/lucene/test/package-info.java |  23 +++
 7 files changed, 365 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22907b95/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index ef7f808..4c28938 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -19,9 +19,9 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -36,10 +36,8 @@ import com.gemstone.gemfire.cache.FixedPartitionAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
@@ -47,14 +45,26 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
-import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+/**
+ * Tests of creating lucene indexes on regions. All tests of index creation
+ * use cases should be in classes starting with LuceneIndexCreation*. Most
+ * tests belong in this class, except for:
+ * <ul>
+ * <li> Tests that use persistence are in {@link LuceneIndexCreationPersistenceIntegrationTest}  </li>
+ * <li> Tests that use offheap are in {@link LuceneIndexCreationOffHeapIntegrationTest}  </li>
+ * </ul>
+ */
 @Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
 public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
-  public static final String INDEX_NAME = "index";
-  public static final String REGION_NAME = "region";
+
 
   @Test
   public void shouldCreateIndexWriterWithAnalyzersWhenSettingPerFieldAnalyzers()
@@ -70,18 +80,22 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     Region region = createRegion();
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     region.put("key1", new TestObject());
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
     assertEquals(analyzers, index.getFieldAnalyzers());
     assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
     assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
   }
 
   @Test
-  public void shouldUseRedundancyForInternalRegionsWhenUserRegionHasRedundancy() {
+  @Parameters({"0", "1", "2"})
+  public void shouldUseRedundancyForInternalRegionsWhenUserRegionHasRedundancy(int redundancy) {
     createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_REDUNDANT).create(REGION_NAME);
+    PartitionAttributesFactory paf = new PartitionAttributesFactory();
+    paf.setRedundantCopies(redundancy);
+
+    cache.createRegionFactory(RegionShortcut.PARTITION_REDUNDANT).setPartitionAttributes(paf.create()).create(REGION_NAME);
     verifyInternalRegions(region -> {
-      assertEquals(1, region.getAttributes().getPartitionAttributes().getRedundantCopies());
+      assertEquals(redundancy, region.getAttributes().getPartitionAttributes().getRedundantCopies());
     });
   }
 
@@ -122,103 +136,6 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldNotUseOffHeapForInternalRegionsWhenUserRegionHasOffHeap() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION)
-      .setOffHeap(true)
-      .create(REGION_NAME);
-
-    verifyInternalRegions(region -> {
-      assertEquals(false, region.getOffHeap());
-    });
-  }
-
-  @Test
-  public void shouldNotUseOverflowForInternalRegionsWhenUserRegionHasOverflow() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_OVERFLOW).create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getAttributes().getEvictionAttributes().getAction().isNone());
-    });
-  }
-
-  @Test
-  public void shouldUseDiskSynchronousWhenUserRegionHasDiskSynchronous() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .setDiskSynchronous(true)
-      .create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getDataPolicy().withPersistence());
-      assertTrue(region.isDiskSynchronous());
-    });
-    AsyncEventQueue queue = getIndexQueue();
-    assertEquals(true, queue.isDiskSynchronous());
-    assertEquals(true, queue.isPersistent());
-  }
-
-  @Test
-  public void shouldUseDiskSyncFalseOnQueueWhenUserRegionHasDiskSynchronousFalse() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .setDiskSynchronous(false)
-      .create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getDataPolicy().withPersistence());
-      assertTrue(region.isDiskSynchronous());
-    });
-    AsyncEventQueue queue = getIndexQueue();
-    assertEquals(false, queue.isDiskSynchronous());
-    assertEquals(true, queue.isPersistent());
-  }
-
-  @Test
-  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
-    createIndex("field1", "field2");
-    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    //Pause the sender so that the entry stays in the queue
-    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
-    queue.getSender().pause();
-
-    dataRegion.put("A", new TestObject());
-    cache.close();
-    createCache();
-    createIndex("field1", "field2");
-    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
-    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
-      .create(INDEX_NAME, REGION_NAME,
-        "field1:world");
-    assertEquals(1, query.search().size());
-  }
-
-  private void verifyIndexFinishFlushing(String indexName, String regionName) {
-    LuceneIndex index = luceneService.getIndex(indexName, regionName);
-    boolean flushed = index.waitUntilFlushed(60000);
-    assertTrue(flushed);
-  }
-  
-  @Test
-  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
-    createIndex("field1", "field2");
-    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    dataRegion.put("A", new TestObject());
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
-    cache.close();
-    createCache();
-    createIndex("text");
-    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
-      .create(INDEX_NAME, REGION_NAME,
-      "field1:world");
-    assertEquals(1, query.search().size());
-  }
-
-  @Test
   public void shouldCreateInternalRegionsForIndex() {
     createIndex("field1", "field2");
 
@@ -252,19 +169,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
 
 
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
-    // Get index
-    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
-
-    // Verify the meta regions exist and are internal
-    LocalRegion chunkRegion = (LocalRegion) cache.getRegion(index.createChunkRegionName());
-    LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
-    verify.accept(chunkRegion);
-    verify.accept(fileRegion);
-  }
-
-  private AsyncEventQueue getIndexQueue() {
-    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
-    return cache.getAsyncEventQueue(aeqId);
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
   }
 
   private Region createRegion() {
@@ -272,13 +177,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   private void createIndex(String ... fieldNames) {
-    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
-  }
-
-  private static class TestObject implements Serializable {
-
-    String field1 = "hello world";
-    String field2 = "this is a field";
+    LuceneTestUtilities.createIndex(cache, fieldNames);
   }
 
   private static class RecordingAnalyzer extends Analyzer {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22907b95/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
new file mode 100644
index 0000000..8f10c26
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.gemstone.gemfire.cache.lucene;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.assertEquals;
+
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests of lucene index creation that use off heap memory
+ */
+@Category(IntegrationTest.class)
+public class LuceneIndexCreationOffHeapIntegrationTest extends LuceneIntegrationTest {
+
+  @Override
+  public void closeCache() {
+    super.closeCache();
+    MemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  @Override
+  protected CacheFactory getCacheFactory() {
+    CacheFactory factory = super.getCacheFactory();
+    factory.set("off-heap-memory-size", "100m");
+    return factory;
+  }
+
+  @Test
+  public void shouldNotUseOffHeapForInternalRegionsWhenUserRegionHasOffHeap() {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION)
+      .setOffHeap(true)
+      .create(REGION_NAME);
+
+    verifyInternalRegions(region -> {
+      assertEquals(false, region.getOffHeap());
+    });
+  }
+
+  private void verifyInternalRegions(Consumer<LocalRegion> verify) {
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22907b95/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
new file mode 100644
index 0000000..23983cb
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.gemstone.gemfire.cache.lucene;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.rules.DiskDirRule;
+import com.jayway.awaitility.Awaitility;
+
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+/**
+ * Tests of lucene index creation that use persistence
+ */
+@Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegrationTest {
+
+  @Rule
+  public DiskDirRule diskDirRule = new DiskDirRule();
+
+  public static final String INDEX_NAME = "index";
+  public static final String REGION_NAME = "region";
+
+  @Override
+  public void createCache() {
+    super.createCache();
+    cache.createDiskStoreFactory()
+      .setDiskDirs(new File[] {diskDirRule.get()})
+      .setMaxOplogSize(1)
+      .create(GemFireCacheImpl.getDefaultDiskStoreName());
+  }
+
+  @Test
+  public void shouldNotUseOverflowForInternalRegionsWhenUserRegionHasOverflow() {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_OVERFLOW).create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getAttributes().getEvictionAttributes().getAction().isNone());
+    });
+  }
+
+  @Test
+  @Parameters({"true", "false"})
+  public void shouldUseDiskSynchronousWhenUserRegionHasDiskSynchronous(boolean synchronous) {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .setDiskSynchronous(synchronous)
+      .create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getDataPolicy().withPersistence());
+      //Underlying region should always be synchronous
+      assertTrue(region.isDiskSynchronous());
+    });
+    AsyncEventQueue queue = getIndexQueue(cache);
+    assertEquals(synchronous, queue.isDiskSynchronous());
+    assertEquals(true, queue.isPersistent());
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
+    createIndex(cache, "field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    //Pause the sender so that the entry stays in the queue
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue(cache);
+    queue.getSender().pause();
+
+    dataRegion.put("A", new TestObject());
+    cache.close();
+    createCache();
+    createIndex(cache, "field1", "field2");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+        "field1:world");
+    assertEquals(1, query.search().size());
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
+    createIndex(cache, "field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    dataRegion.put("A", new TestObject());
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
+    cache.close();
+    createCache();
+    createIndex(cache, "field1", "field2");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+      "field1:world");
+    assertEquals(1, query.search().size());
+  }
+
+  private void verifyInternalRegions(Consumer<LocalRegion> verify) {
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22907b95/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
index 67775d2..c302460 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
@@ -38,28 +38,26 @@ public class LuceneIntegrationTest {
 
   protected Cache cache;
   protected LuceneService luceneService;
-  @Rule
-  public DiskDirRule diskDirRule = new DiskDirRule();
 
   @After
-  public void tearDown() {
+  public void closeCache() {
     if(this.cache != null) {
       this.cache.close();
     }
-    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Before
   public void createCache() {
+    CacheFactory cf = getCacheFactory();
+    this.cache = cf.create();
+
+    luceneService = LuceneServiceProvider.get(this.cache);
+  }
+
+  protected CacheFactory getCacheFactory() {
     CacheFactory cf = new CacheFactory();
     cf.set("mcast-port", "0");
     cf.set("locators", "");
-    cf.set("off-heap-memory-size", "100m");
-    this.cache = cf.create();
-    cache.createDiskStoreFactory()
-      .setDiskDirs(new File[] {diskDirRule.get()})
-      .setMaxOplogSize(1)
-      .create(GemFireCacheImpl.getDefaultDiskStoreName());
-    luceneService = LuceneServiceProvider.get(this.cache);
+    return cf;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22907b95/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
new file mode 100644
index 0000000..61355c4
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.test;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+
+public class LuceneTestUtilities {
+  public static final String INDEX_NAME = "index";
+  public static final String REGION_NAME = "region";
+
+  public static void verifyInternalRegions(LuceneService luceneService, Cache cache, Consumer<LocalRegion> verify) {
+    // Get index
+    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    // Verify the meta regions exist and are internal
+    LocalRegion chunkRegion = (LocalRegion) cache.getRegion(index.createChunkRegionName());
+    LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
+    verify.accept(chunkRegion);
+    verify.accept(fileRegion);
+  }
+
+  public static AsyncEventQueue getIndexQueue(Cache cache) {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
+    return cache.getAsyncEventQueue(aeqId);
+  }
+
+  public static void createIndex(Cache cache, String... fieldNames) {
+    LuceneServiceProvider.get(cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
+  }
+
+  public static void verifyIndexFinishFlushing(Cache cache, String indexName, String regionName) {
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
+    LuceneIndex index = luceneService.getIndex(indexName, regionName);
+    boolean flushed = index.waitUntilFlushed(60000);
+    assertTrue(flushed);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22907b95/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
new file mode 100644
index 0000000..3287b9f
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.test;
+
+import java.io.Serializable;
+
+public class TestObject implements Serializable {
+
+  String field1 = "hello world";
+  String field2 = "this is a field";
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22907b95/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
new file mode 100644
index 0000000..6345804
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * Utility classes for testing the lucene functionality.
+ */
+
+package com.gemstone.gemfire.cache.lucene.test;
\ No newline at end of file


[17/19] incubator-geode git commit: GEODE-1402: use preTearDownAssertions to call checkOrphans

Posted by kl...@apache.org.
GEODE-1402: use preTearDownAssertions to call checkOrphans


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

Branch: refs/heads/feature/GEODE-1392
Commit: bb6414944ced9b73c5f68c8b42e54f059eb1b744
Parents: 1179c08
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon May 16 17:37:27 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:26 2016 -0700

----------------------------------------------------------------------
 .../cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java   | 2 +-
 .../cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java | 2 +-
 .../gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java   | 2 +-
 .../gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java      | 2 +-
 .../gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java | 2 +-
 .../gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java    | 2 +-
 .../gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java  | 2 +-
 .../com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java | 2 +-
 .../gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java     | 2 +-
 .../gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java         | 2 +-
 .../internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java    | 2 +-
 .../gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java  | 2 +-
 .../gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java      | 2 +-
 .../cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java     | 2 +-
 .../internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java  | 2 +-
 .../tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java      | 2 +-
 .../compression/CompressionCacheListenerOffHeapDUnitTest.java      | 2 +-
 .../compression/CompressionRegionOperationsOffHeapDUnitTest.java   | 2 +-
 .../gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java      | 2 +-
 19 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java
index 1f0fbd9..908bc96 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedAckOverflowRegionCCEOffHeapDUnitTest extends Distributed
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java
index 431116a..c1bd906 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedAckPersistentRegionCCEOffHeapDUnitTest extends Distribut
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java
index 3950dca..ffba1d3 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedAckRegionCCEOffHeapDUnitTest extends DistributedAckRegio
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java
index 1b2c6be..08486d4 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedAckRegionOffHeapDUnitTest extends DistributedAckRegionDU
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java
index 98e92c0..08a1fcb 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedNoAckRegionCCEOffHeapDUnitTest extends DistributedNoAckR
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java
index 8778069..60b0fdf 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedNoAckRegionOffHeapDUnitTest extends DistributedNoAckRegi
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java
index 4311196..6846aa8 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class GlobalRegionCCEOffHeapDUnitTest extends GlobalRegionCCEDUnitTest {
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java
index dda4f7c..cb0e7ec 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class GlobalRegionOffHeapDUnitTest extends GlobalRegionDUnitTest {
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java
index 8af7b2a..af25980 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java
@@ -40,7 +40,7 @@ public class OffHeapLRUEvictionControllerDUnitTest extends
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java
index b00d9c0..897e9b9 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java
@@ -37,7 +37,7 @@ public class PartitionedRegionOffHeapDUnitTest extends PartitionedRegionDUnitTes
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java
index b9c90dc..33513b9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java
@@ -40,7 +40,7 @@ public class DistrbutedRegionProfileOffHeapDUnitTest extends CacheTestCase {
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java
index 97477fe..d93aa89 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java
@@ -42,7 +42,7 @@ public class OffHeapEvictionDUnitTest extends EvictionDUnitTest {
   }  
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java
index 1f410db..6920e05 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java
@@ -38,7 +38,7 @@ public class OffHeapEvictionStatsDUnitTest extends EvictionStatsDUnitTest {
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java
index b204453..55e52cb 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class PartitionedRegionLocalMaxMemoryOffHeapDUnitTest extends Partitioned
   }
   
   @Override
-  protected final void preTearDownPartitionedRegionDUnitTest() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
index 046d787..48f88e8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
@@ -34,7 +34,7 @@ public class PartitionedRegionOffHeapEvictionDUnitTest extends
   }  
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java
index c62c593..c4fdded 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java
@@ -35,7 +35,7 @@ public class ForceInvalidateOffHeapEvictionDUnitTest extends
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
index ec828fd..0100a05 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
@@ -40,7 +40,7 @@ public class CompressionCacheListenerOffHeapDUnitTest extends
   }
 
   @Override
-  protected final void preTearDownCompressionCacheListenerDUnitTest() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
index d76976f..0cc3bdf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
@@ -33,7 +33,7 @@ public class CompressionRegionOperationsOffHeapDUnitTest extends
   }
   
   @Override
-  protected final void preTearDownCompressionRegionOperationsDUnitTest() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb641494/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
index a1ce952..99fadfc 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
@@ -77,7 +77,7 @@ public class OutOfOffHeapMemoryDUnitTest extends CacheTestCase {
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     final SerializableRunnable checkOrphans = new SerializableRunnable() {
       @Override
       public void run() {


[06/19] incubator-geode git commit: GEODE-17: make sure commands tests are run in useHttp mode.

Posted by kl...@apache.org.
GEODE-17: make sure commands tests are run in useHttp mode.

* added the CommandOverHttpDUnitTest in geode-web
* added GfshCommandsOverHttpSecurityTest in geode-web
* move ConnectCommandWithHttpAndSSLDUnitTest to geode-web
* make sure ConnectCommandWithHttpAndSSLDUnitTest is running with the correct connection method
* updated the trusted.keystore with the supported algorithm
* create a SuiteRunner so that we can run tests in a suite yet have them report to its own xml


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

Branch: refs/heads/feature/GEODE-1392
Commit: 96b3df9cffaca4620d83bfafad91575c01cf490d
Parents: fa2d963
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue May 10 13:03:41 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:23 2016 -0700

----------------------------------------------------------------------
 .../SharedConfigurationEndToEndDUnitTest.java   |  12 +-
 .../internal/security/shiro/ShiroPrincipal.java |   2 +-
 .../cli/commands/CliCommandTestBase.java        |  47 +--
 .../ConnectCommandWithHttpAndSSLDUnitTest.java  | 305 -------------------
 .../SharedConfigurationCommandsDUnitTest.java   |   8 +-
 .../security/GfshCommandsSecurityTest.java      |   9 +-
 .../src/test/resources/ssl/trusted.keystore     | Bin 1078 -> 2241 bytes
 .../test/junit/runner/SuiteBlockRunner.java     |  46 +++
 .../gemfire/test/junit/runner/SuiteRunner.java  |  53 ++++
 .../ClusterConfigurationDUnitTest.java          |  10 +-
 geode-web/build.gradle                          |  11 +
 .../cli/commands/CommandOverHttpDUnitTest.java  |  58 ++++
 .../ConnectCommandWithHttpAndSSLDUnitTest.java  | 305 +++++++++++++++++++
 .../GfshCommandsOverHttpSecurityTest.java       |  29 ++
 14 files changed, 528 insertions(+), 367 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
index caee9ea..a6221e9 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
@@ -36,10 +36,6 @@ import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.commons.io.FileUtils;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
@@ -61,6 +57,10 @@ import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 @Category(DistributedTest.class)
 public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
 
@@ -116,7 +116,7 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
   public void testStartServerAndExecuteCommands() throws Exception {
     final HeadlessGfsh gfsh = new HeadlessGfsh("gfsh2", 300, this.gfshDir);
     assertNotNull(gfsh);
-    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+    connect(jmxHost, jmxPort, httpPort, gfsh);
 
     serverNames.addAll(startServers(gfsh, locatorString, 2, "Server", 1));
     doCreateCommands();
@@ -380,7 +380,7 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
     int jmxPort = (Integer)result[2];
     int httpPort = (Integer)result[3];
 
-    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+    connect(jmxHost, jmxPort, httpPort, gfsh);
 
     // Create a cache in VM 1
     VM dataMember = getHost(0).getVM(1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java
index 8413ebb..621eb87 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java
@@ -30,7 +30,7 @@ public class ShiroPrincipal implements Principal {
 
   @Override
   public String getName() {
-    return subject.toString();
+    return subject.getPrincipal().toString();
   }
 
   public Subject getSubject(){

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
index 4b25165..a0966f9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@ -39,7 +39,6 @@ import com.gemstone.gemfire.management.internal.cli.parser.CommandTarget;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.management.internal.security.JSONAuthorization;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
 
@@ -57,11 +56,8 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
 
   private transient HeadlessGfsh shell;
 
-  protected boolean useHttpOnConnect = false;
-  protected boolean enableAuth = false;
-  protected String jsonAuthorization = "cacheServer.json";
-  protected String username = "super-user";
-  protected String password = "1234567";
+  public static final String USE_HTTP_SYSTEM_PROPERTY = "useHTTP";
+  private boolean useHttpOnConnect = Boolean.getBoolean(USE_HTTP_SYSTEM_PROPERTY);
 
   private transient int httpPort;
   private transient int jmxPort;
@@ -107,12 +103,12 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
   @SuppressWarnings("serial")
   protected HeadlessGfsh setUpJmxManagerOnVm0ThenConnect(final Properties props) {
     setUpJMXManagerOnVM(0, props);
-    shellConnect();
+    connect(this.jmxHost, this.jmxPort, this.httpPort, getDefaultShell());
     return shell;
   }
 
   protected void setUpJMXManagerOnVM(int vm, final Properties props) {
-    Object[] result = (Object[]) Host.getHost(0).getVM(vm).invoke("setUpJmxManagerOnVm0ThenConnect", () -> {
+    Object[] result = (Object[]) Host.getHost(0).getVM(vm).invoke("setUpJmxManagerOnVm"+vm, () -> {
       final Object[] results = new Object[3];
       final Properties localProps = (props != null ? props : new Properties());
 
@@ -138,14 +134,6 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
       localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
       localProps.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
 
-      if (enableAuth) {
-        localProps.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME,
-          JSONAuthorization.class.getName() + ".create");
-        localProps.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
-
-        JSONAuthorization.setUpWithJsonFile(jsonAuthorization);
-      }
-
       getSystem(localProps);
       verifyManagementServiceStarted(getCache());
 
@@ -214,33 +202,9 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
     }
   }
 
-  /**
-   * Connect a shell to the JMX server at the given host and port
-   *
-   *
-   * @param host    Host of the JMX server
-   * @param jmxPort Port of the JMX server
-   * @param shell   Shell to connect
-   */
-  protected void shellConnect(final String host, final int jmxPort, final int httpPort, HeadlessGfsh shell) {
-    assertTrue(host != null);
-    assertTrue(shell != null);
-
-    connect(host, jmxPort, httpPort, shell);
-  }
-
-  protected CommandResult shellConnect(){
-    return connect(this.jmxHost, this.jmxPort, this.httpPort, getDefaultShell());
-  }
-
-  protected CommandResult connect(final String host, final int jmxPort, final int httpPort, HeadlessGfsh shell){
+  protected void connect(final String host, final int jmxPort, final int httpPort, HeadlessGfsh shell){
     final CommandStringBuilder command = new CommandStringBuilder(CliStrings.CONNECT);
 
-    if(enableAuth) {
-      command.addOption(CliStrings.CONNECT__USERNAME, username);
-      command.addOption(CliStrings.CONNECT__PASSWORD, password);
-    }
-
     String endpoint;
     if (useHttpOnConnect) {
       endpoint = "http://" + host + ":" + httpPort + "/gemfire/v1";
@@ -261,7 +225,6 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
 
     info("Successfully connected to managing node using " + (useHttpOnConnect ? "HTTP" : "JMX"));
     assertEquals(true, shell.isConnectedAndReady());
-    return result;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
deleted file mode 100644
index 2e0897d..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.management.internal.cli.commands;
-
-import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
-import static com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.*;
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-import static com.gemstone.gemfire.util.test.TestUtil.*;
-
-import java.io.File;
-import java.util.Properties;
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.SSLSession;
-
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
-import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
-import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import com.gemstone.gemfire.test.junit.categories.SecurityTest;
-
-/**
- * @since  8.1
- */
-@Category({ DistributedTest.class, SecurityTest.class })
-public class ConnectCommandWithHttpAndSSLDUnitTest extends CliCommandTestBase {
-
-  private static final ThreadLocal<Properties> sslInfoHolder = new ThreadLocal<>();
-
-  private File jks;
-
-  // TODO: should this test use @RunWith(Parameterized.class)?
-
-  @Override
-  public final void postSetUpCliCommandTestBase() throws Exception {
-    this.jks = new File(getResourcePath(getClass(), "/ssl/trusted.keystore"));
-  }
-  
-  @Override
-  protected final void preTearDownCliCommandTestBase() throws Exception {
-    destroyDefaultSetup();
-  }
-  
-  @Override
-  public final void postTearDownCacheTestCase() throws Exception {
-    sslInfoHolder.set(null);
-  }
-
-  @Test
-  public void testMutualAuthentication() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME, "SSL");
-    localProps.setProperty(HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__KEY_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__KEY_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "SSL");
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSimpleSSL() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithoutKeyStoreType() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-  
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithSSLProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLS");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSv11Protocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.1");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSv12Protocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testWithMultipleProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL,TLSv1.2");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Ignore("disabled for unknown reason")
-  @Test
-  public void testSSLWithCipherSuite() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLSv1.2");
-
-    //Its bad to hard code here. But using SocketFactory.getDefaultCiphers() somehow is not working with the option 
-    //"https.cipherSuites" which is required to restrict cipher suite with HttpsURLConnection
-    //Keeping the below code for further investigation on different Java versions ( 7 & 8) @TODO
-    
-   /*SSLContext sslContext = SSLContext.getInstance("TLSv1.2");
-    
-    sslContext.init(null, null, new java.security.SecureRandom());
-    String[] cipherSuites = sslContext.getSocketFactory().getSupportedCipherSuites();*/
-    
-    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS_NAME,"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_CIPHERS, "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Ignore("disabled for unknown reason")
-  @Test
-  public void testSSLWithMultipleCipherSuite() throws Exception {
-    System.setProperty("javax.net.debug", "ssl,handshake,failure");
-    
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
-    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS_NAME,"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256,TLS_EMPTY_RENEGOTIATION_INFO_SCSV");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Override
-  protected void shellConnect(final String host, final int jmxPort, final int httpPort, final HeadlessGfsh shell) {
-    assertNotNull(host);
-    assertNotNull(shell);
-
-    final CommandStringBuilder command = new CommandStringBuilder(CONNECT);
-    String endpoint;
-
-    // This is for testing purpose only. If we remove this piece of code we will
-    // get a java.security.cert.CertificateException
-    // as matching hostname can not be obtained in all test environment.
-    HttpsURLConnection.setDefaultHostnameVerifier(new HostnameVerifier() {
-      @Override
-      public boolean verify(String string, SSLSession ssls) {
-        return true;
-      }
-    });
-    
-    endpoint = "https://" + host + ":" + httpPort + "/gemfire/v1";
-    
-    command.addOption(CONNECT__USE_HTTP, Boolean.TRUE.toString());
-    command.addOption(CONNECT__URL, endpoint);
-    command.addOption(CONNECT__USE_SSL,Boolean.TRUE.toString());
-
-    if(sslInfoHolder.get().getProperty(CONNECT__KEY_STORE) != null){
-      command.addOption(CONNECT__KEY_STORE, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD) != null){
-      command.addOption(CONNECT__KEY_STORE_PASSWORD, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE) != null){
-      command.addOption(CONNECT__TRUST_STORE, sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD) != null){
-      command.addOption(CONNECT__TRUST_STORE_PASSWORD, sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS) != null){
-      command.addOption(CONNECT__SSL_PROTOCOLS, sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS) != null){
-      command.addOption(CONNECT__SSL_CIPHERS, sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS));
-    }
-
-    CommandResult result = executeCommand(shell, command.toString());
-
-    if (!shell.isConnectedAndReady()) {
-      fail("Connect command failed to connect to manager " + endpoint + " result=" + commandResultToString(result));
-    }
-
-    info("Successfully connected to managing node using HTTPS");
-    assertEquals(true, shell.isConnectedAndReady());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
index 781ed98..334d2e1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
@@ -31,9 +31,6 @@ import java.io.IOException;
 import java.util.Properties;
 import java.util.Set;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.DistributedMember;
@@ -54,6 +51,9 @@ import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 /**
  * DUnit test to test export and import of shared configuration.
  */
@@ -167,7 +167,7 @@ public class SharedConfigurationCommandsDUnitTest extends CliCommandTestBase {
     });
 
     HeadlessGfsh gfsh = getDefaultShell();
-    shellConnect(locator1JmxHost, locator1JmxPort, locator1HttpPort, gfsh);
+    connect(locator1JmxHost, locator1JmxPort, locator1HttpPort, gfsh);
 
     // Create a cache in VM 1
     VM dataMember = getHost(0).getVM(1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
index d96c96c..8eaaf6a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
@@ -39,9 +39,9 @@ import org.junit.experimental.categories.Category;
 
 @Category(IntegrationTest.class)
 public class GfshCommandsSecurityTest {
-  private static int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
-  private static int jmxPort = ports[0];
-  private static int httpPort = ports[1];
+  protected static int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+  protected static int jmxPort = ports[0];
+  protected static int httpPort = ports[1];
 
   private HeadlessGfsh gfsh = null;
 
@@ -50,12 +50,13 @@ public class GfshCommandsSecurityTest {
       jmxPort, httpPort, "cacheServer.json");
 
   @Rule
-  public GfshShellConnectionRule gfshConnection = null;
+  public GfshShellConnectionRule gfshConnection;
 
   public GfshCommandsSecurityTest(){
     gfshConnection = new GfshShellConnectionRule(jmxPort, httpPort, false);
   }
 
+
   @Before
   public void before(){
     gfsh = gfshConnection.getGfsh();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-core/src/test/resources/ssl/trusted.keystore
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/ssl/trusted.keystore b/geode-core/src/test/resources/ssl/trusted.keystore
old mode 100755
new mode 100644
index 6057e3c..bd75039
Binary files a/geode-core/src/test/resources/ssl/trusted.keystore and b/geode-core/src/test/resources/ssl/trusted.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteBlockRunner.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteBlockRunner.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteBlockRunner.java
new file mode 100644
index 0000000..60fed84
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteBlockRunner.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.test.junit.runner;
+
+import org.junit.runners.BlockJUnit4ClassRunner;
+import org.junit.runners.model.FrameworkMethod;
+import org.junit.runners.model.InitializationError;
+
+/**
+ * used by SuiteRunner to override the test method name
+ */
+public class SuiteBlockRunner extends BlockJUnit4ClassRunner {
+
+  private Class<?> suiteClass;
+
+  /**
+   * Creates a BlockJUnit4ClassRunner to run {@code klass}
+   * @param klass
+   * @throws InitializationError if the test class is malformed.
+   */
+  public SuiteBlockRunner(final Class parentClass,  final Class<?> klass) throws InitializationError {
+    super(klass);
+    this.suiteClass = parentClass;
+  }
+
+  @Override
+  protected String testName(FrameworkMethod method) {
+    return method.getName()+"@"+ suiteClass.getName();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteRunner.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteRunner.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteRunner.java
new file mode 100644
index 0000000..2a860c0
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteRunner.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.test.junit.runner;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.runner.Runner;
+import org.junit.runners.Suite;
+import org.junit.runners.model.InitializationError;
+import org.junit.runners.model.RunnerBuilder;
+
+/**
+ * SuiteRunner is like Junit Suite, it's used in conjunction with <code>SuiteClass({xx.class, yy.class})</code>
+ * It's different from Suite in two ways:
+ * 1. it should only contain contain Junit4 test classes
+ * 2. the test method names inside each test class are appended with the suiteClass name so that the result will show up different
+ * as when you run these tests alone.
+ */
+public class SuiteRunner extends Suite {
+
+  public SuiteRunner(final Class<?> klass, final RunnerBuilder builder) throws InitializationError {
+    super(klass, getRunners(klass));
+  }
+
+  private static List<Runner> getRunners(final Class<?> klass) throws InitializationError{
+    SuiteClasses annotation = klass.getAnnotation(SuiteClasses.class);
+    if (annotation == null) {
+      throw new InitializationError(String.format("class '%s' must have a SuiteClasses annotation", klass.getName()));
+    }
+    Class<?>[] childClasses = annotation.value();
+    List<Runner> runners = new ArrayList<>();
+    for(Class childClass:childClasses){
+      runners.add(new SuiteBlockRunner(klass, childClass));
+    }
+    return runners;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
index 62bb793..d8692ce 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
@@ -40,10 +40,6 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Region;
@@ -80,6 +76,10 @@ import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 @Category(DistributedTest.class)
 public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
 
@@ -663,7 +663,7 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
     int jmxPort = (Integer)result[2];
     int httpPort = (Integer)result[3];
     
-    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+    connect(jmxHost, jmxPort, httpPort, gfsh);
 
     final String dataMemberWorkingDir = this.temporaryFolder.getRoot().getCanonicalPath() + File.separator + dataMember;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-web/build.gradle
----------------------------------------------------------------------
diff --git a/geode-web/build.gradle b/geode-web/build.gradle
index 5bd1d89..228751b 100755
--- a/geode-web/build.gradle
+++ b/geode-web/build.gradle
@@ -52,6 +52,9 @@ dependencies {
   testCompile project(':geode-junit')
   testRuntime project(':geode-common')
   testRuntime project(':geode-core')
+  testCompile files(project(':geode-core').sourceSets.test.output)
+
+  testRuntime files(war.destinationDir)
 }
 
 //Remove the gradle output directories from the eclipse classpath. These
@@ -69,6 +72,14 @@ idea.module.iml {
   }
 }
 
+distributedTest {
+  dependsOn ':geode-core:webJar', 'war'
+}
+
+integrationTest {
+  dependsOn ':geode-core:webJar', 'war'
+}
+
 war {
   classpath configurations.runtime 
   classpath project(':geode-core').webJar.archivePath

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
new file mode 100644
index 0000000..9687eb7
--- /dev/null
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.management.internal.cli.commands;
+
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.runner.SuiteRunner;
+
+import org.junit.ClassRule;
+import org.junit.contrib.java.lang.system.ProvideSystemProperty;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@Category(DistributedTest.class)
+@RunWith(SuiteRunner.class)
+@Suite.SuiteClasses({
+  ConfigCommandsDUnitTest.class,
+  DeployCommandsDUnitTest.class,
+  DiskStoreCommandsDUnitTest.class,
+  FunctionCommandsDUnitTest.class,
+  GemfireDataCommandsDUnitTest.class,
+  GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.class,
+  IndexCommandsDUnitTest.class,
+  ListAndDescribeDiskStoreCommandsDUnitTest.class,
+  ListIndexCommandDUnitTest.class,
+  MemberCommandsDUnitTest.class,
+  MiscellaneousCommandsDUnitTest.class,
+  MiscellaneousCommandsExportLogsPart1DUnitTest.class,
+  MiscellaneousCommandsExportLogsPart2DUnitTest.class,
+  MiscellaneousCommandsExportLogsPart3DUnitTest.class,
+  MiscellaneousCommandsExportLogsPart4DUnitTest.class,
+  QueueCommandsDUnitTest.class,
+  SharedConfigurationCommandsDUnitTest.class,
+  ShellCommandsDUnitTest.class,
+  ShowDeadlockDUnitTest.class,
+  ShowMetricsDUnitTest.class,
+  ShowStackTraceDUnitTest.class,
+  UserCommandsDUnitTest.class
+})
+public class CommandOverHttpDUnitTest {
+  @ClassRule
+  public static ProvideSystemProperty provideSystemProperty = new ProvideSystemProperty(CliCommandTestBase.USE_HTTP_SYSTEM_PROPERTY, "true");
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
new file mode 100644
index 0000000..a8f209b
--- /dev/null
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.cli.commands;
+
+import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
+import static com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.*;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.util.test.TestUtil.*;
+
+import java.io.File;
+import java.util.Properties;
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSession;
+
+import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * @since  8.1
+ */
+@Category({ DistributedTest.class, SecurityTest.class })
+public class ConnectCommandWithHttpAndSSLDUnitTest extends CliCommandTestBase {
+
+  private static final ThreadLocal<Properties> sslInfoHolder = new ThreadLocal<>();
+
+  private File jks;
+
+  // TODO: should this test use @RunWith(Parameterized.class)?
+
+  @Override
+  public final void postSetUpCliCommandTestBase() throws Exception {
+    this.jks = new File(getResourcePath(getClass(), "/ssl/trusted.keystore"));
+  }
+  
+  @Override
+  protected final void preTearDownCliCommandTestBase() throws Exception {
+    destroyDefaultSetup();
+  }
+  
+  @Override
+  public final void postTearDownCacheTestCase() throws Exception {
+    sslInfoHolder.set(null);
+  }
+
+  @Test
+  public void testMutualAuthentication() throws Exception {
+    Properties serverProps = new Properties();
+    serverProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
+    serverProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME, "SSL");
+    serverProps.setProperty(HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME, "true");
+    serverProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_NAME, jks.getCanonicalPath());
+    serverProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
+
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__KEY_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__KEY_STORE_PASSWORD, "password");
+    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "SSL");
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(serverProps);
+  }
+
+  @Test
+  public void testSimpleSSL() throws Exception {
+    Properties serverProps = new Properties();
+    serverProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
+
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(serverProps);
+  }
+
+  @Test
+  public void testSSLWithoutKeyStoreType() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+  
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testSSLWithSSLProtocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testSSLWithTLSProtocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLS");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testSSLWithTLSv11Protocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.1");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testSSLWithTLSv12Protocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testWithMultipleProtocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL,TLSv1.2");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testSSLWithCipherSuite() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLSv1.2");
+
+    //Its bad to hard code here. But using SocketFactory.getDefaultCiphers() somehow is not working with the option 
+    //"https.cipherSuites" which is required to restrict cipher suite with HttpsURLConnection
+    //Keeping the below code for further investigation on different Java versions ( 7 & 8) @TODO
+    
+   /*SSLContext sslContext = SSLContext.getInstance("TLSv1.2");
+    
+    sslContext.init(null, null, new java.security.SecureRandom());
+    String[] cipherSuites = sslContext.getSocketFactory().getSupportedCipherSuites();*/
+    
+    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS_NAME,"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
+
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    clientProps.setProperty(CONNECT__SSL_CIPHERS, "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
+    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testSSLWithMultipleCipherSuite() throws Exception {
+    System.setProperty("javax.net.debug", "ssl,handshake,failure");
+    
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
+    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS_NAME,"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256,TLS_EMPTY_RENEGOTIATION_INFO_SCSV");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Override
+  protected void connect(final String host, final int jmxPort, final int httpPort, final HeadlessGfsh shell) {
+    assertNotNull(host);
+    assertNotNull(shell);
+
+    final CommandStringBuilder command = new CommandStringBuilder(CONNECT);
+    String endpoint;
+
+    // This is for testing purpose only. If we remove this piece of code we will
+    // get a java.security.cert.CertificateException
+    // as matching hostname can not be obtained in all test environment.
+    HttpsURLConnection.setDefaultHostnameVerifier(new HostnameVerifier() {
+      @Override
+      public boolean verify(String string, SSLSession ssls) {
+        return true;
+      }
+    });
+    
+    endpoint = "https://" + host + ":" + httpPort + "/gemfire/v1";
+    
+    command.addOption(CONNECT__USE_HTTP, Boolean.TRUE.toString());
+    command.addOption(CONNECT__URL, endpoint);
+    command.addOption(CONNECT__USE_SSL,Boolean.TRUE.toString());
+
+    if(sslInfoHolder.get().getProperty(CONNECT__KEY_STORE) != null){
+      command.addOption(CONNECT__KEY_STORE, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD) != null){
+      command.addOption(CONNECT__KEY_STORE_PASSWORD, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE) != null){
+      command.addOption(CONNECT__TRUST_STORE, sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD) != null){
+      command.addOption(CONNECT__TRUST_STORE_PASSWORD, sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS) != null){
+      command.addOption(CONNECT__SSL_PROTOCOLS, sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS) != null){
+      command.addOption(CONNECT__SSL_CIPHERS, sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS));
+    }
+
+    CommandResult result = executeCommand(shell, command.toString());
+
+    if (!shell.isConnectedAndReady()) {
+      fail("Connect command failed to connect to manager " + endpoint + " result=" + commandResultToString(result));
+    }
+
+    info("Successfully connected to managing node using HTTPS");
+    assertEquals(true, shell.isConnectedAndReady());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96b3df9c/geode-web/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsOverHttpSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsOverHttpSecurityTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsOverHttpSecurityTest.java
new file mode 100644
index 0000000..af0cfad
--- /dev/null
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsOverHttpSecurityTest.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.management.internal.security;
+
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class GfshCommandsOverHttpSecurityTest extends GfshCommandsSecurityTest {
+  public GfshCommandsOverHttpSecurityTest(){
+    gfshConnection = new GfshShellConnectionRule(jmxPort, httpPort, true);
+  }
+}


[15/19] incubator-geode git commit: GEODE-1256 Alter rat.gradle to exclude copies of website sources

Posted by kl...@apache.org.
GEODE-1256  Alter rat.gradle to exclude copies of website sources

  Added `geode-site/content/**` to the rat.gradle excludes list
  such that there are no longer 10 Unknown Licenses.

This closes #144


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

Branch: refs/heads/feature/GEODE-1392
Commit: c5796a9fbd6dcaf2c0e352cad9532254e305cb47
Parents: 147c1f0
Author: Karen Miller <km...@pivotal.io>
Authored: Mon May 16 18:22:01 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:26 2016 -0700

----------------------------------------------------------------------
 gradle/rat.gradle | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c5796a9f/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 3d0826a..f507993 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -117,6 +117,7 @@ rat {
     'geode-site/website/content/css/font-awesome.min.css',
     'geode-site/website/lib/pandoc.template',
     'geode-site/website/content/font/**',
+    'geode-site/content/**',
     // compiled logs and locks
     'geode-site/website/tmp/**',
     'geode-site/website/layouts/**',


[07/19] incubator-geode git commit: GEODE-17: wording change

Posted by kl...@apache.org.
GEODE-17: wording change


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

Branch: refs/heads/feature/GEODE-1392
Commit: 79b97aac5412e103f423e2d972cc751c2eeb8011
Parents: 4845dd0
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon May 16 11:19:03 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:24 2016 -0700

----------------------------------------------------------------------
 .../gemfire/management/internal/cli/commands/ShellCommands.java  | 4 ++--
 .../gemfire/management/internal/cli/i18n/CliStrings.java         | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/79b97aac/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
index a9712a1..baf3049 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
@@ -200,7 +200,7 @@ public class ShellCommands implements CommandMarker {
 
         if (userName != null && userName.length() > 0) {
           if (passwordToUse == null || passwordToUse.length() == 0) {
-            passwordToUse = gemfireShell.readWithMask("http password: ", '*');
+            passwordToUse = gemfireShell.readWithMask("password: ", '*');
           }
           if (passwordToUse == null || passwordToUse.length() == 0) {
             throw new IllegalArgumentException(CliStrings.CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED);
@@ -266,7 +266,7 @@ public class ShellCommands implements CommandMarker {
         // JMX Authentication Config
         if (userName != null && userName.length() > 0) {
           if (passwordToUse == null || passwordToUse.length() == 0) {
-            passwordToUse = gfshInstance.readWithMask("jmx password: ", '*');
+            passwordToUse = gfshInstance.readWithMask("password: ", '*');
           }
           if (passwordToUse == null || passwordToUse.length() == 0) {
               throw new IllegalArgumentException(CliStrings.CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/79b97aac/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
index 9bb573b..28ba856 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
@@ -394,7 +394,7 @@ public class CliStrings {
   public static final String CONNECT__MSG__CONNECTING_TO_MANAGER_AT_0 = "Connecting to Manager at {0} ..";
   public static final String CONNECT__MSG__CONNECTING_TO_MANAGER_HTTP_SERVICE_AT_0 = "Connecting to Manager's HTTP service at {0} ..";
   public static final String CONNECT__MSG__LOCATOR_COULD_NOT_FIND_MANAGER = "Locator could not find a JMX Manager";
-  public static final String CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED = "jmx password must be specified.";
+  public static final String CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED = "password must be specified.";
   public static final String CONNECT__MSG__ALREADY_CONNECTED = "Already connected to: {0}";
   public static final String CONNECT__MSG__SUCCESS = "Successfully connected to: {0}";
   public static final String CONNECT__MSG__ERROR = "Could not connect to : {0}. {1}";


[09/19] incubator-geode git commit: GEODE-493 CI failure: PartitionedRegionSingleHopDUnitTest.test_MetadataContents GEODE-699 PartitionedRegionSingleHopDUnitTest.test_MetadataServiceCallAccuracy

Posted by kl...@apache.org.
GEODE-493 CI failure:
PartitionedRegionSingleHopDUnitTest.test_MetadataContents
GEODE-699
PartitionedRegionSingleHopDUnitTest.test_MetadataServiceCallAccuracy

Add a counter fetchTaskCount in ClientMetadataService to count the
number fetch task threads. Make sure the counter is increased before a
fetch task thread is scheduled. And decrease the counter when a thread
is done.

In the test, use Awaitility to make all fetch tasks threads are
completed, i.e. fetchTaskCount == 0, before the next step.


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

Branch: refs/heads/feature/GEODE-1392
Commit: 8beb8af8eb3c7780cf42e266d77b41dfddfaa287
Parents: 6a9b0ae
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Sat May 14 15:59:32 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:24 2016 -0700

----------------------------------------------------------------------
 .../client/internal/ClientMetadataService.java  | 31 ++++++++++++++++++--
 .../PartitionedRegionSingleHopDUnitTest.java    |  8 +++--
 2 files changed, 34 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8beb8af8/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java
index ed26708..60318b0 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java
@@ -72,7 +72,13 @@ public final class ClientMetadataService {
   /** random number generator used in pruning */
   private final Random rand = new Random();
   
-  private volatile boolean isMetadataStable = true; 
+  private volatile boolean isMetadataStable = true;
+
+  private boolean isMetadataRefreshed_TEST_ONLY = false;
+  
+  private int fetchTaskCount = 0;
+  
+  private final Object fetchTaskCountLock = new Object();
   
   public ClientMetadataService(Cache cache) {
     this.cache = cache;
@@ -528,6 +534,9 @@ public final class ClientMetadataService {
       }
     }
     else {
+      synchronized (fetchTaskCountLock){
+        fetchTaskCount++;
+      }
       Runnable fetchTask = new Runnable() {
         @SuppressWarnings("synthetic-access")
         public void run() {
@@ -544,6 +553,11 @@ public final class ClientMetadataService {
               logger.debug("An exception occurred while fetching metadata", e);
             }
           }
+          finally {
+            synchronized (fetchTaskCountLock){
+              fetchTaskCount--;
+            }
+          }
         }
       };
       SingleHopClientExecutor.submitTask(fetchTask);
@@ -630,6 +644,9 @@ public final class ClientMetadataService {
         }
       }
     } else {
+      synchronized (fetchTaskCountLock){
+        fetchTaskCount++;
+      }
       Runnable fetchTask = new Runnable() {
         @SuppressWarnings("synthetic-access")
         public void run() {
@@ -644,6 +661,11 @@ public final class ClientMetadataService {
               logger.debug("An exception occurred while fetching metadata", e);
             }
           }
+          finally {
+            synchronized (fetchTaskCountLock){
+              fetchTaskCount--;
+            }
+          }
         }
       };
       SingleHopClientExecutor.submitTask(fetchTask);
@@ -841,6 +863,9 @@ public final class ClientMetadataService {
     this.isMetadataStable = isMetadataStable;
   }
 
-  private boolean isMetadataRefreshed_TEST_ONLY = false;
-
+  public int getFetchTaskCount() {
+    synchronized(fetchTaskCountLock) {
+      return fetchTaskCount;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8beb8af8/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
index f79d6c6..65cc7eb 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
@@ -342,7 +342,6 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
   // Put data, get data and make the metadata stable.
   // Now verify that metadata has all 8 buckets info.
   // Now update and ensure the fetch service is never called.
-  @Category(FlakyTest.class) // GEODE-493: random ports, waitForCriterions
   public void test_MetadataContents() {
     Integer port0 = (Integer)member0.invoke(() -> PartitionedRegionSingleHopDUnitTest.createServer( 1, 4 ));
     Integer port1 = (Integer)member1.invoke(() -> PartitionedRegionSingleHopDUnitTest.createServer( 1, 4 ));
@@ -368,7 +367,6 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
   // once,
   // fetchservice has to be triggered.
   // Now put again from c2.There should be no hop at all.
-  @Category(FlakyTest.class) // GEODE-699: random ports, async actions, time sensitive
   public void test_MetadataServiceCallAccuracy() {
     Integer port0 = (Integer)member0.invoke(() -> PartitionedRegionSingleHopDUnitTest.createServer( 1, 4 ));
     Integer port1 = (Integer)member1.invoke(() -> PartitionedRegionSingleHopDUnitTest.createServer( 1, 4 ));
@@ -391,6 +389,9 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
 
     Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> cms.isRefreshMetadataTestOnly() == true);
 
+    //make sure all fetch tasks are completed
+    Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> cms.getFetchTaskCount() == 0);
+
     cms.satisfyRefreshMetadata_TEST_ONLY(false);
     region.put(new Integer(0), "create0");
     region.put(new Integer(1), "create1");
@@ -1973,6 +1974,9 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
 
   private void verifyMetadata() {
     ClientMetadataService cms = ((GemFireCacheImpl)cache).getClientMetadataService();
+    //make sure all fetch tasks are completed
+    Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> cms.getFetchTaskCount() == 0);
+
     final Map<String, ClientPartitionAdvisor> regionMetaData = cms
         .getClientPRMetadata_TEST_ONLY();
     Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> (regionMetaData.size() == 4));


[18/19] incubator-geode git commit: GEODE-1405: Fixing shouldRecoverPersistentIndexWhenDataStillInQueue

Posted by kl...@apache.org.
GEODE-1405: Fixing shouldRecoverPersistentIndexWhenDataStillInQueue

The changes in 3e8a610e99bc005bcc56c378e3d1e2274a3d468c broke this test.
It just needs to wait for the flush *after* the data region is created.


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

Branch: refs/heads/feature/GEODE-1392
Commit: 5576acad2b4d22be6b604f689838a802226ac490
Parents: c5796a9
Author: Dan Smith <up...@apache.org>
Authored: Tue May 17 11:43:53 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:27 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5576acad/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index fe754a4..ef7f808 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -185,9 +185,9 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     cache.close();
     createCache();
     createIndex("field1", "field2");
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
+    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
       .create(INDEX_NAME, REGION_NAME,
         "field1:world");


[02/19] incubator-geode git commit: Fixing compile errors in eclipse

Posted by kl...@apache.org.
Fixing compile errors in eclipse

We were seeing compile errors in eclipse due to:
1) Missing gradle output dirs in the extensions module. I removed those
dirs from the eclipse classpath

2) Access restrictions on things like Unsafe. This was caused by a
change in gradle 2.12 to add the java 1.8 execution environment, rather
than the direct JDK dependency, to the project. That turned on access
restrictions in eclipse for  the execution environment.


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

Branch: refs/heads/feature/GEODE-1392
Commit: c35f76f96aeb5cf1c674be440953bb10521de8bd
Parents: 3490f37
Author: Dan Smith <up...@apache.org>
Authored: Thu May 12 17:10:29 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:22 2016 -0700

----------------------------------------------------------------------
 extensions/geode-modules-tomcat7/build.gradle |  6 ++++++
 gradle/ide.gradle                             | 15 ++++++++++++---
 2 files changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c35f76f9/extensions/geode-modules-tomcat7/build.gradle
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-tomcat7/build.gradle b/extensions/geode-modules-tomcat7/build.gradle
index 975c976..3c75e56 100644
--- a/extensions/geode-modules-tomcat7/build.gradle
+++ b/extensions/geode-modules-tomcat7/build.gradle
@@ -49,4 +49,10 @@ dependencies {
   testCompile project(path: ':geode-junit')
   testCompile files(project(':geode-core').sourceSets.test.output)
   testCompile files(project(':extensions/geode-modules').sourceSets.test.output)
+
+  eclipse.classpath.file {
+    whenMerged { classpath ->
+      classpath.entries.removeAll { entry -> entry.path.contains('geode-modules/build')}
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c35f76f9/gradle/ide.gradle
----------------------------------------------------------------------
diff --git a/gradle/ide.gradle b/gradle/ide.gradle
index adaf8d9..03f6565 100644
--- a/gradle/ide.gradle
+++ b/gradle/ide.gradle
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import org.gradle.plugins.ide.eclipse.model.Container
 allprojects {
   apply plugin: 'idea'
   apply plugin: 'eclipse'
@@ -26,11 +27,19 @@ subprojects {
       downloadSources = true
       plusConfigurations += [ configurations.provided ]
       file {
-        // Remove the gradle output directories from the eclipse classpath.
-        // Unfortunately, using minusConfigurations does not work here, because
-        // it removes the entire geode-core project.
         whenMerged { classpath ->
+            // Remove the gradle output directories from the eclipse classpath.
+            // Unfortunately, using minusConfigurations does not work here, because
+            // it removes the entire geode-core project.
             classpath.entries.removeAll { entry -> entry.path.contains('geode-core/build')}
+
+            //By default, gradle adds the java 1.8 *execution environment*, which has access restrictions on
+            //things like Unsafe. Change it to a direct dependency on the workspace JDK
+            classpath.entries = classpath.entries.collect { entry -> 
+              entry.path.contains('org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE')
+                ? new Container('org.eclipse.jdt.launching.JRE_CONTAINER')
+                : entry
+            }
         }
       }
     }


[03/19] incubator-geode git commit: GEODE-11: Adding tests of fixed partitions and persistent lucene indexes

Posted by kl...@apache.org.
GEODE-11: Adding tests of fixed partitions and persistent lucene indexes

In addition to the tests, I fixed some code to properly propagate the
disk synchronous flag to the index.


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

Branch: refs/heads/feature/GEODE-1392
Commit: 3490f37a01d75b92ca36bbe2aaae1e31a763021b
Parents: 67d5e8d
Author: Dan Smith <up...@apache.org>
Authored: Thu May 12 15:36:38 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:22 2016 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/test/fake/Fakes.java   |  19 +++
 .../gemfire/test/junit/rules/DiskDirRule.java   |  54 ++++++++
 .../test/junit/rules/DiskDirRuleTest.java       |  49 +++++++
 .../LuceneIndexForPartitionedRegion.java        |  28 ++--
 .../lucene/internal/LuceneQueryFactoryImpl.java |   3 +
 .../internal/PartitionedRepositoryManager.java  |   1 +
 .../LuceneIndexCreationIntegrationTest.java     | 131 +++++++++++++++++--
 .../cache/lucene/LuceneIntegrationTest.java     |  14 +-
 .../LuceneIndexForPartitionedRegionTest.java    |  58 ++++----
 .../LuceneQueryFactoryImplJUnitTest.java        |  21 ++-
 10 files changed, 312 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
index 99644b7..2a1fd8e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
@@ -23,6 +23,10 @@ import java.net.UnknownHostException;
 import org.junit.Assert;
 
 import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.distributed.internal.DSClock;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -96,6 +100,21 @@ public class Fakes {
     return cache().getDistributedSystem();
   }
 
+  /**
+   * A fake region, which contains a fake cache and some other
+   * fake attributes
+   */
+  public static Region region(String name, Cache cache) {
+    Region region = mock(Region.class);
+    RegionAttributes attributes = mock(RegionAttributes.class);
+    DataPolicy policy = mock(DataPolicy.class);
+    when(region.getAttributes()).thenReturn(attributes);
+    when(attributes.getDataPolicy()).thenReturn(policy);
+    when(region.getCache()).thenReturn(cache);
+    when(region.getRegionService()).thenReturn(cache);
+    return region;
+  }
+
   private Fakes() {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
new file mode 100644
index 0000000..184619f
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.test.junit.rules;
+
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+
+import org.junit.rules.ExternalResource;
+
+public class DiskDirRule extends ExternalResource {
+  private File diskDir;
+
+  @Override protected void before() throws Throwable {
+    diskDir = new File(".", "DiskDirRule-" + System.nanoTime());
+  }
+
+  @Override protected void after() {
+    try {
+      Files.walk(diskDir.toPath()).forEach((path) -> {
+        try {
+          Files.delete(path);
+        }
+        catch (IOException e) {
+          //Ignore
+        }
+      });
+    } catch(IOException e) {
+      throw new RuntimeException("Could not delete disk dir: " + diskDir, e);
+    }
+    diskDir.delete();
+  }
+
+  public File get() {
+    diskDir.mkdirs();
+    return diskDir;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/DiskDirRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/DiskDirRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/DiskDirRuleTest.java
new file mode 100644
index 0000000..8c322a4
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/DiskDirRuleTest.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.test.junit.rules;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class DiskDirRuleTest {
+  @Test
+  public void shouldDeleteDirInAfter() throws Throwable {
+    DiskDirRule diskDirRule = new DiskDirRule();
+    diskDirRule.before();
+    final File dir = diskDirRule.get();
+    assertTrue(dir.exists());
+    final File file1 = new File(dir, "file1");
+    final File subdir = new File(dir, "subdir");
+    final File file2 = new File(dir, "file2");
+    subdir.mkdir();
+    Files.write(file1.toPath(), Arrays.asList("abc"));
+    Files.write(file2.toPath(), Arrays.asList("stuff"));
+    diskDirRule.after();
+    assertFalse(dir.exists());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index dd9d384..d22ca4a 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -29,7 +29,6 @@ import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
@@ -88,46 +87,37 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       repositoryManager = new PartitionedRepositoryManager(dataRegion, (PartitionedRegion)fileRegion, (PartitionedRegion)chunkRegion, mapper, analyzer);
       
       // create AEQ, AEQ listener and specify the listener to repositoryManager
-      if (withPersistence) {
-        createAEQWithPersistence();
-      }
-      else {
-        createAEQ();
-      }
+      createAEQ(dataRegion);
 
       addExtension(dataRegion);
       hasInitialized = true;
     }
   }
 
-  private AsyncEventQueueFactoryImpl createAEQFactory() {
+  private AsyncEventQueueFactoryImpl createAEQFactory(final Region dataRegion) {
     AsyncEventQueueFactoryImpl factory = (AsyncEventQueueFactoryImpl) cache.createAsyncEventQueueFactory();
     factory.setParallel(true); // parallel AEQ for PR
     factory.setMaximumQueueMemory(1000);
     factory.setDispatcherThreads(1);
     factory.setIsMetaQueue(true);
+    if(dataRegion.getAttributes().getDataPolicy().withPersistence()) {
+      factory.setPersistent(true);
+    }
+    factory.setDiskSynchronous(dataRegion.getAttributes().isDiskSynchronous());
     return factory;
   }
 
-  AsyncEventQueue createAEQWithPersistence() {
-    AsyncEventQueueFactoryImpl factory = createAEQFactory();
-    factory.setPersistent(true);
-    return createAEQ(factory);
-  }
-
-  AsyncEventQueue createAEQ() {
-    return createAEQ(createAEQFactory());
+  AsyncEventQueue createAEQ(Region dataRegion) {
+    return createAEQ(createAEQFactory(dataRegion));
   }
 
   private AsyncEventQueue createAEQ(AsyncEventQueueFactoryImpl factory) {
     LuceneEventListener listener = new LuceneEventListener(repositoryManager);
     String aeqId = LuceneServiceImpl.getUniqueIndexName(getName(), regionPath);
-    AsyncEventQueueImpl aeq = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
     AsyncEventQueue indexQueue = factory.create(aeqId, listener);
     return indexQueue;
   }
 
-
   boolean fileRegionExists(String fileRegionName) {
     return cache.<String, File> getRegion(fileRegionName) != null;
   }
@@ -156,7 +146,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".chunks";
   }
 
-  private PartitionAttributesFactory configureLuceneRegionAttributesFactory(PartitionAttributesFactory attributesFactory, PartitionAttributes dataRegionAttributes) {
+  private PartitionAttributesFactory configureLuceneRegionAttributesFactory(PartitionAttributesFactory attributesFactory, PartitionAttributes<?,?> dataRegionAttributes) {
     attributesFactory.setTotalNumBuckets(dataRegionAttributes.getTotalNumBuckets());
     attributesFactory.setRedundantCopies(dataRegionAttributes.getRedundantCopies());
     return attributesFactory;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java
index c6087ea..385b226 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java
@@ -54,6 +54,9 @@ public class LuceneQueryFactoryImpl implements LuceneQueryFactory {
   
   public <K, V> LuceneQuery<K, V> create(String indexName, String regionName, LuceneQueryProvider provider) {
     Region<K, V> region = cache.getRegion(regionName);
+    if(region == null) {
+      throw new IllegalArgumentException("Region not found: " + regionName);
+    }
     LuceneQueryImpl<K, V> luceneQuery = new LuceneQueryImpl<K, V>(indexName, region, provider, projectionFields, limit, pageSize);
     return luceneQuery;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
index 07050e2..57b8862 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 
 import com.gemstone.gemfire.InternalGemFireError;
 import com.gemstone.gemfire.cache.Region;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index 2d16e32..6429143 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -29,23 +29,26 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
-import java.util.function.Function;
 
 import com.gemstone.gemfire.cache.EvictionAttributes;
 import com.gemstone.gemfire.cache.ExpirationAttributes;
+import com.gemstone.gemfire.cache.FixedPartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.lucene.LuceneIndex;
-import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
-import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.jayway.awaitility.Awaitility;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -122,6 +125,18 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
+  public void shouldNotUseOffHeapForInternalRegionsWhenUserRegionHasOffHeap() {
+    createIndex("text");
+    cache.createRegionFactory(RegionShortcut.PARTITION)
+      .setOffHeap(true)
+      .create(REGION_NAME);
+
+    verifyInternalRegions(region -> {
+      assertEquals(false, region.getOffHeap());
+    });
+  }
+
+  @Test
   public void shouldNotUseOverflowForInternalRegionsWhenUserRegionHasOverflow() {
     createIndex("text");
     cache.createRegionFactory(RegionShortcut.PARTITION_OVERFLOW).create(REGION_NAME);
@@ -131,18 +146,112 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldCreateInternalRegionsForIndex() {
+  public void shouldUseDiskSynchronousWhenUserRegionHasDiskSynchronous() {
+    createIndex("text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .setDiskSynchronous(true)
+      .create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getDataPolicy().withPersistence());
+      assertTrue(region.isDiskSynchronous());
+    });
+    AsyncEventQueue queue = getIndexQueue();
+    assertEquals(true, queue.isDiskSynchronous());
+    assertEquals(true, queue.isPersistent());
+  }
+
+  @Test
+  public void shouldUseDiskSyncFalseOnQueueWhenUserRegionHasDiskSynchronousFalse() {
     createIndex("text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .setDiskSynchronous(false)
+      .create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getDataPolicy().withPersistence());
+      assertTrue(region.isDiskSynchronous());
+    });
+    AsyncEventQueue queue = getIndexQueue();
+    assertEquals(false, queue.isDiskSynchronous());
+    assertEquals(true, queue.isPersistent());
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
+    createIndex("field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    //Pause the sender so that the entry stays in the queue
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
+    queue.getSender().pause();
+
+    dataRegion.put("A", new TestObject());
+    cache.close();
+    createCache();
+    createIndex("field1", "field2");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+        "field1:world");
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      assertEquals(1, query.search().size());
+    });
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
+    createIndex("field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    dataRegion.put("A", new TestObject());
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
+
+    //Wait until the queue has drained
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, queue.size()));
+    cache.close();
+    createCache();
+    createIndex("text");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+      "field1:world");
+    assertEquals(1, query.search().size());
+  }
+
+  @Test
+  public void shouldCreateInternalRegionsForIndex() {
+    createIndex("field1", "field2");
 
     // Create partitioned region
     createRegion();
 
     verifyInternalRegions(region -> {
       region.isInternalRegion();
+      assertNotNull(region.getAttributes().getPartitionAttributes().getColocatedWith());
       cache.rootRegions().contains(region);
     });
   }
 
+  @Test
+  public void shouldUseFixedPartitionsForInternalRegions() {
+    createIndex("text");
+
+    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory<>();
+    final FixedPartitionAttributes fixedAttributes = FixedPartitionAttributes.createFixedPartition("A", true, 1);
+    partitionAttributesFactory.addFixedPartitionAttributes(fixedAttributes);
+    cache.createRegionFactory(RegionShortcut.PARTITION)
+      .setPartitionAttributes(partitionAttributesFactory.create())
+      .create(REGION_NAME);
+
+    verifyInternalRegions(region -> {
+      //Fixed partitioned regions don't allow you to specify the partitions on the colocated region
+      assertNull(region.getAttributes().getPartitionAttributes().getFixedPartitionAttributes());
+      assertTrue(((PartitionedRegion) region).isFixedPartitionedRegion());
+    });
+  }
+
+
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
     // Get index
     LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
@@ -152,21 +261,25 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
     verify.accept(chunkRegion);
     verify.accept(fileRegion);
+  }
 
+  private AsyncEventQueue getIndexQueue() {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
+    return cache.getAsyncEventQueue(aeqId);
   }
 
   private Region createRegion() {
     return this.cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
   }
 
-  private void createIndex(String fieldName) {
-    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME, REGION_NAME, fieldName);
+  private void createIndex(String ... fieldNames) {
+    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
   }
 
   private static class TestObject implements Serializable {
 
-    String field1 = "a b c d";
-    String field2 = "f g h";
+    String field1 = "hello world";
+    String field2 = "this is a field";
   }
 
   private static class RecordingAnalyzer extends Analyzer {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
index 8c6f59e..67775d2 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
@@ -19,10 +19,15 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import java.io.File;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
+import com.gemstone.gemfire.test.junit.rules.DiskDirRule;
 
 import org.junit.After;
 import org.junit.Before;
@@ -33,15 +38,15 @@ public class LuceneIntegrationTest {
 
   protected Cache cache;
   protected LuceneService luceneService;
-
   @Rule
-  public TestName name = new TestName();
+  public DiskDirRule diskDirRule = new DiskDirRule();
 
   @After
   public void tearDown() {
     if(this.cache != null) {
       this.cache.close();
     }
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Before
@@ -49,7 +54,12 @@ public class LuceneIntegrationTest {
     CacheFactory cf = new CacheFactory();
     cf.set("mcast-port", "0");
     cf.set("locators", "");
+    cf.set("off-heap-memory-size", "100m");
     this.cache = cf.create();
+    cache.createDiskStoreFactory()
+      .setDiskDirs(new File[] {diskDirRule.get()})
+      .setMaxOplogSize(1)
+      .create(GemFireCacheImpl.getDefaultDiskStoreName());
     luceneService = LuceneServiceProvider.get(this.cache);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
index 984b221..e35de5e 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
@@ -32,6 +32,7 @@ import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.ExpirationAttributes;
 import com.gemstone.gemfire.cache.MembershipAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributes;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
@@ -120,12 +121,16 @@ public class LuceneIndexForPartitionedRegionTest {
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
+    final Region region =Fakes.region(regionPath, cache);
+    RegionAttributes attributes  = region.getAttributes();
+    when(attributes.getDataPolicy()).thenReturn(DataPolicy.PERSISTENT_PARTITION);
     AsyncEventQueueFactoryImpl aeqFactory = mock(AsyncEventQueueFactoryImpl.class);
     when(cache.createAsyncEventQueueFactory()).thenReturn(aeqFactory);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    index.createAEQWithPersistence();
+    index.createAEQ(region);
 
+    verify(aeqFactory).setPersistent(eq(true));
     verify(aeqFactory).create(any(), any());
   }
 
@@ -134,21 +139,23 @@ public class LuceneIndexForPartitionedRegionTest {
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
+    final Region region =Fakes.region(regionPath, cache);
     AsyncEventQueueFactoryImpl aeqFactory = mock(AsyncEventQueueFactoryImpl.class);
     when(cache.createAsyncEventQueueFactory()).thenReturn(aeqFactory);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    index.createAEQ();
+    index.createAEQ(region);
 
+    verify(aeqFactory, never()).setPersistent(eq(true));
     verify(aeqFactory).create(any(), any());
   }
 
-  private void initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache) {
+  private Region initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache) {
     int defaultLocalMemory = 100;
-    initializeScenario(withPersistence, regionPath, cache, defaultLocalMemory);
+    return initializeScenario(withPersistence, regionPath, cache, defaultLocalMemory);
   }
 
-  private void initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache, int localMaxMemory) {
+  private Region initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache, int localMaxMemory) {
     PartitionedRegion region = mock(PartitionedRegion.class);
     RegionAttributes regionAttributes = mock(RegionAttributes.class);
     PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
@@ -162,6 +169,8 @@ public class LuceneIndexForPartitionedRegionTest {
     when(partitionAttributes.getTotalNumBuckets()).thenReturn(113);
     when(dataPolicy.withPersistence()).thenReturn(withPersistence);
     when(region.getExtensionPoint()).thenReturn(extensionPoint);
+
+    return region;
   }
 
   private PartitionAttributes initializeAttributes(final Cache cache) {
@@ -192,41 +201,22 @@ public class LuceneIndexForPartitionedRegionTest {
   }
 
   @Test
-  public void initializeWithPersistenceShouldCreateAEQWithPersistence() {
-    boolean withPersistence = true;
-    String name = "indexName";
-    String regionPath = "regionName";
-    Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
-
-    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    index.setSearchableFields(new String[]{"field"});
-    LuceneIndexForPartitionedRegion spy = spy(index);
-    doReturn(null).when(spy).createFileRegion(any(), any(), any());
-    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQWithPersistence();
-    spy.initialize();
-
-    verify(spy).createAEQWithPersistence();
-  }
-
-  @Test
   public void initializeWithoutPersistenceShouldCreateAEQ() {
     boolean withPersistence = false;
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
+    Region region = initializeScenario(withPersistence, regionPath, cache);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     index.setSearchableFields(new String[]{"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQ();
+    doReturn(null).when(spy).createAEQ(eq(region));
     spy.initialize();
 
-    verify(spy).createAEQ();
+    verify(spy).createAEQ(eq(region));
   }
 
   @Test
@@ -235,14 +225,14 @@ public class LuceneIndexForPartitionedRegionTest {
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
+    Region region = initializeScenario(withPersistence, regionPath, cache);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     index.setSearchableFields(new String[]{"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQ();
+    doReturn(null).when(spy).createAEQ(eq(region));
     spy.initialize();
 
     verify(spy).createChunkRegion(eq(RegionShortcut.PARTITION), eq(index.createFileRegionName()), any(), eq(index.createChunkRegionName()));
@@ -254,14 +244,14 @@ public class LuceneIndexForPartitionedRegionTest {
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
+    Region region = initializeScenario(withPersistence, regionPath, cache);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     index.setSearchableFields(new String[]{"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQ();
+    doReturn(null).when(spy).createAEQ(eq(region));
     spy.initialize();
 
     verify(spy).createFileRegion(eq(RegionShortcut.PARTITION), eq(index.createFileRegionName()), any());
@@ -309,7 +299,7 @@ public class LuceneIndexForPartitionedRegionTest {
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQWithPersistence();
+    doReturn(null).when(spy).createAEQ(any());
     spy.initialize();
 
     verify(spy).createChunkRegion(eq(RegionShortcut.PARTITION_PERSISTENT), eq(index.createFileRegionName()), any(), eq(index.createChunkRegionName()));
@@ -328,7 +318,7 @@ public class LuceneIndexForPartitionedRegionTest {
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQWithPersistence();
+    doReturn(null).when(spy).createAEQ(any());
     spy.initialize();
 
     verify(spy).createFileRegion(eq(RegionShortcut.PARTITION_PERSISTENT), eq(index.createFileRegionName()), any());
@@ -347,7 +337,7 @@ public class LuceneIndexForPartitionedRegionTest {
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQWithPersistence();
+    doReturn(null).when(spy).createAEQ(any());
     spy.initialize();
     spy.initialize();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3490f37a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java
index 0614e62..975b92f 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java
@@ -19,21 +19,30 @@
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.*;
 
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 import org.mockito.Mockito;
 
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class LuceneQueryFactoryImplJUnitTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
 
   @Test
-  public void test() {
-    Cache cache = Mockito.mock(Cache.class);
+  public void shouldCreateQueryWithCorrectAttributes() {
+    Cache cache = mock(Cache.class);
+    Region region = mock(Region.class);
+    when(cache.getRegion(any())).thenReturn(region);
     LuceneQueryFactoryImpl f = new LuceneQueryFactoryImpl(cache);
     f.setPageSize(5);
     f.setResultLimit(25);
@@ -47,4 +56,12 @@ public class LuceneQueryFactoryImplJUnitTest {
     Mockito.verify(cache).getRegion(Mockito.eq("region"));
   }
 
+  @Test
+  public void shouldThrowIllegalArgumentWhenRegionIsMissing() {
+    Cache cache = mock(Cache.class);
+    LuceneQueryFactoryImpl f = new LuceneQueryFactoryImpl(cache);
+    thrown.expect(IllegalArgumentException.class);
+    LuceneQuery<Object, Object> query = f.create("index", "region", new StringQueryProvider("test"));
+  }
+
 }


[10/19] incubator-geode git commit: GEODE-1252: modify bits field atomically

Posted by kl...@apache.org.
GEODE-1252: modify bits field atomically

- AtomicIntegerFieldUpdater now used to modify bits field
- added unit test for bit methods
- removed unused constructors


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

Branch: refs/heads/feature/GEODE-1392
Commit: 4845dd0886d4c72b8e91808d35b8041e64a7f50a
Parents: d08a047
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed May 11 14:38:09 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:24 2016 -0700

----------------------------------------------------------------------
 .../internal/cache/versions/VersionTag.java     | 83 +++++++++++-------
 .../versions/AbstractVersionTagTestBase.java    | 92 ++++++++++++++++++++
 .../cache/versions/VMVersionTagTest.java        | 32 +++++++
 .../sanctionedDataSerializables.txt             |  4 +-
 4 files changed, 179 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4845dd08/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java
index 60e4299..7910996 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java
@@ -19,6 +19,7 @@ package com.gemstone.gemfire.internal.cache.versions;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
 import org.apache.logging.log4j.Logger;
 
@@ -74,6 +75,7 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
   private static final int BITS_TIMESTAMP_APPLIED = 0x20;
 
   private static final int BITS_ALLOWED_BY_RESOLVER = 0x40;
+  // Note: the only valid BITS_* are 0xFFFF.
   
   /**
    * the per-entry version number for the operation
@@ -100,10 +102,19 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    */
   private byte distributedSystemId;
 
+  // In GEODE-1252 we found that the bits field
+  // was concurrently modified by calls to
+  // setPreviousMemberID and setRecorded.
+  // So bits has been changed to volatile and
+  // all modification to it happens using AtomicIntegerFieldUpdater.
+  private static final AtomicIntegerFieldUpdater<VersionTag> bitsUpdater =
+      AtomicIntegerFieldUpdater.newUpdater(VersionTag.class, "bits");
   /**
    * boolean bits
+   * Note: this is an int field so it has 32 bits BUT only the lower 16 bits are serialized.
+   * So all our code should treat this an an unsigned short field.
    */
-  private int bits;
+  private volatile int bits;
 
   /**
    * the initiator of the operation.  If null, the initiator was the sender
@@ -128,7 +139,11 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
 
   /** record that the timestamp from this tag was applied to the cache */
   public void setTimeStampApplied(boolean isTimeStampUpdated) {
-    this.bits |= BITS_TIMESTAMP_APPLIED;
+    if (isTimeStampUpdated) {
+      setBits(BITS_TIMESTAMP_APPLIED);
+    } else {
+      clearBits(~BITS_TIMESTAMP_APPLIED);
+    }
   }
 
   /**
@@ -152,9 +167,9 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
 
   public void setIsGatewayTag(boolean isGateway) {
     if (isGateway) {
-      this.bits = this.bits | BITS_GATEWAY_TAG;
+      setBits(BITS_GATEWAY_TAG);
     } else {
-      this.bits = this.bits & ~BITS_GATEWAY_TAG;
+      clearBits(~BITS_GATEWAY_TAG);
     }
   }
 
@@ -193,7 +208,7 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    * set that this tag has been recorded in a receiver's RVV
    */
   public void setRecorded() {
-    this.bits |= BITS_RECORDED;
+    setBits(BITS_RECORDED);
   }
 
   /**
@@ -236,7 +251,7 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    * @param previousMemberID the previousMemberID to set
    */
   public void setPreviousMemberID(T previousMemberID) {
-    this.bits |= BITS_HAS_PREVIOUS_ID;
+    setBits(BITS_HAS_PREVIOUS_ID);
     this.previousMemberID = previousMemberID;
   }
 
@@ -249,9 +264,9 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    */
   public VersionTag setPosDup(boolean flag) {
     if (flag) {
-      this.bits |= BITS_POSDUP;
+      setBits(BITS_POSDUP);
     } else {
-      this.bits &= ~BITS_POSDUP;
+      clearBits(~BITS_POSDUP);
     }
     return this;
   }
@@ -268,9 +283,9 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    */
   public VersionTag setAllowedByResolver(boolean flag) {
     if (flag) {
-      this.bits |= BITS_ALLOWED_BY_RESOLVER;
+      setBits(BITS_ALLOWED_BY_RESOLVER);
     } else {
-      this.bits &= ~BITS_ALLOWED_BY_RESOLVER;
+      clearBits(~BITS_ALLOWED_BY_RESOLVER);
     }
     return this;
   }
@@ -319,21 +334,6 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
     return !(this.entryVersion == 0 && this.regionVersionHighBytes == 0 && this.regionVersionLowBytes == 0);
   }
 
-  public VersionTag() {
-  }
-
-  /**
-   * creates a version tag for a WAN gateway event
-   *
-   * @param timestamp
-   * @param dsid
-   */
-  public VersionTag(long timestamp, int dsid) {
-    this.timeStamp = timestamp;
-    this.distributedSystemId = (byte) (dsid & 0xFF);
-    this.bits = BITS_GATEWAY_TAG + BITS_IS_REMOTE_TAG;
-  }
-
   public void toData(DataOutput out) throws IOException {
     toData(out, true);
   }
@@ -386,7 +386,7 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
     if (logger.isTraceEnabled(LogMarker.VERSION_TAG)) {
       logger.info(LogMarker.VERSION_TAG, "deserializing {} with flags 0x{}", this.getClass(), Integer.toHexString(flags));
     }
-    this.bits = in.readUnsignedShort();
+    bitsUpdater.set(this, in.readUnsignedShort());
     this.distributedSystemId = in.readByte();
     if ((flags & VERSION_TWO_BYTES) != 0) {
       this.entryVersion = in.readShort() & 0xffff;
@@ -408,11 +408,11 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
         this.previousMemberID = readMember(in);
       }
     }
-    this.bits |= BITS_IS_REMOTE_TAG;
+    setIsRemoteForTesting();
   }
   
   public void setIsRemoteForTesting() {
-    this.bits |= BITS_IS_REMOTE_TAG;
+    setBits(BITS_IS_REMOTE_TAG);
   }
 
   public abstract T readMember(DataInput in) throws IOException, ClassNotFoundException;
@@ -440,14 +440,14 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
       if (this.memberID != null) {
         s.append("; mbr=").append(this.memberID);
       }
-      if ((this.bits & BITS_HAS_PREVIOUS_ID) != 0) {
+      if (hasPreviousMemberID()) {
         s.append("; prev=").append(this.previousMemberID);
       }
       if (this.distributedSystemId >= 0) {
         s.append("; ds=").append(this.distributedSystemId);
       }
       s.append("; time=").append(getVersionTimeStamp());
-      if ((this.bits & BITS_IS_REMOTE_TAG) != 0) {
+      if (isFromOtherMember()) {
         s.append("; remote");
       }
       if (this.isAllowedByResolver()) {
@@ -544,4 +544,27 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
     }
     return true;
   }
+  
+  /**
+   * Set any bits in the given bitMask on the bits field
+   */
+  private void setBits(int bitMask) {
+    int oldBits;
+    int newBits;
+    do {
+      oldBits = this.bits;
+      newBits = oldBits | bitMask;
+    } while (!bitsUpdater.compareAndSet(this, oldBits, newBits));
+  }
+  /**
+   * Clear any bits not in the given bitMask from the bits field
+   */
+  private void clearBits(int bitMask) {
+    int oldBits;
+    int newBits;
+    do {
+      oldBits = this.bits;
+      newBits = oldBits & bitMask;
+    } while (!bitsUpdater.compareAndSet(this, oldBits, newBits));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4845dd08/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/AbstractVersionTagTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/AbstractVersionTagTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/AbstractVersionTagTestBase.java
new file mode 100644
index 0000000..bf0ce43
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/AbstractVersionTagTestBase.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache.versions;
+
+import static org.junit.Assert.*;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public abstract class AbstractVersionTagTestBase {
+  @SuppressWarnings("rawtypes")
+  protected abstract VersionTag createVersionTag();
+  
+  @SuppressWarnings("rawtypes")
+  private VersionTag vt;
+  
+  @Before
+  public void setup() {
+    this.vt = createVersionTag();
+  }
+  @Test
+  public void testFromOtherMemberBit() {
+    assertEquals(false, vt.isFromOtherMember());
+    vt.setIsRemoteForTesting();
+    assertEquals(true, vt.isFromOtherMember());
+  }
+  
+  @Test
+  public void testTimeStampUpdatedBit() {
+    assertEquals(false, vt.isTimeStampUpdated());
+    vt.setTimeStampApplied(true);
+    assertEquals(true, vt.isTimeStampUpdated());
+    vt.setTimeStampApplied(false);
+    assertEquals(false, vt.isTimeStampUpdated());
+  }
+  
+  @Test
+  public void testGatewayTagBit() {
+    assertEquals(false, vt.isGatewayTag());
+    vt.setIsGatewayTag(true);
+    assertEquals(true, vt.isGatewayTag());
+    vt.setIsGatewayTag(false);
+    assertEquals(false, vt.isGatewayTag());
+  }
+  
+  @Test
+  public void testRecordedBit() {
+    assertEquals(false, vt.isRecorded());
+    vt.setRecorded();
+    assertEquals(true, vt.isRecorded());
+  }
+  
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPreviousMemberIDBit() {
+    assertEquals(false, vt.hasPreviousMemberID());
+    vt.setPreviousMemberID(null);
+    assertEquals(true, vt.hasPreviousMemberID());
+  }
+  
+  @Test
+  public void testPosDupBit() {
+    assertEquals(false, vt.isPosDup());
+    vt.setPosDup(true);
+    assertEquals(true, vt.isPosDup());
+    vt.setPosDup(false);
+    assertEquals(false, vt.isPosDup());
+  }
+  
+  @Test
+  public void testAllowedByResolverBit() {
+    assertEquals(false, vt.isAllowedByResolver());
+    vt.setAllowedByResolver(true);
+    assertEquals(true, vt.isAllowedByResolver());
+    vt.setAllowedByResolver(false);
+    assertEquals(false, vt.isAllowedByResolver());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4845dd08/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/VMVersionTagTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/VMVersionTagTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/VMVersionTagTest.java
new file mode 100644
index 0000000..4e39f3d
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/VMVersionTagTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache.versions;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class VMVersionTagTest extends AbstractVersionTagTestBase {
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  protected VersionTag createVersionTag() {
+    return new VMVersionTag();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4845dd08/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index fe21fbf..d2204a0 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -1975,8 +1975,8 @@ fromData,212,2a2a2bb600bdb500132bb900be01003d2a1c047e04a0000704a7000403b500972ab
 toData,239,2ab4001499000dbb00b05912b8b700b2bf2a2ab400132bb600b9033d2ab400979900071c04803d2b1cb900ba02002b2ab4000bb60035b900bb03002b2ab4000cb60035b900bb03002b2ab40025b60033b900ba02002ab40025b60099b9002b01004e2db9002c010099002b2db9002d0100c0002e3a042a1904b9002f0100c0006e2bb600b91904b9003001002bb800bca7ffd22b2ab40026b60033b900ba02002ab40026b60099b9002b01004e2db9002c01009900332db9002d0100c0002e3a042a1904b9002f0100c0006e2bb600b92b1904b900300100c0003eb6003fb900bb0300a7ffca2ab4001f2bb800bcb1
 
 com/gemstone/gemfire/internal/cache/versions/VersionTag,2
-fromData,201,2bb9002001003db20012b20013b900140200990022b20012b20013122105bd001659032ab600175359041cb8001853b9001904002a2bb900200100b500012a2bb900220100b5000d1c077e9900132a2bb900230100121c7eb50003a7000f2a2bb900240100027eb500031c10107e99000d2a2bb900230100b500072a2bb900240100b5000a2a2bb80025b500041c047e99000c2a2a2bb60026b5000b1c057e99001e1c10087e99000e2a2ab4000bb5000ca7000c2a2a2bb60026b5000c2a59b40001101080b50001b1
-toData,269,033e0336042ab400031211a2000a0436041d07803e2ab400079900081d1010803e2ab4000bc6000b1c9900071d04803e2ab4000cc6001b1d05803e2ab4000c2ab4000ba6000c1c9900081d1008803eb20012b20013b900140200990022b20012b20013121505bd001659032ab600175359041db8001853b9001904002b1db9001a02002b2ab40001b9001a02002b2ab4000db9001b020015049900132b2ab40003121c7eb9001a0200a7000d2b2ab40003b9001d02002ab4000799000d2b2ab40007b9001a02002b2ab4000ab9001d02002ab400042bb8001e2ab4000bc600101c99000c2a2ab4000b2bb6001f2ab4000cc6001b2ab4000c2ab4000ba600071c9a000c2a2ab4000c2bb6001fb1
+fromData,197,2bb9002201003db20014b20015b900160200990022b20014b20015122305bd001859032ab600195359041cb8001a53b9001b0400b200242a2bb900220100b600252a2bb900260100b500101c077e9900132a2bb900270100121e7eb50006a7000f2a2bb900280100027eb500061c10107e99000d2a2bb900270100b5000a2a2bb900280100b5000d2a2bb80029b500071c047e99000c2a2a2bb6002ab5000e1c057e99001e1c10087e99000e2a2ab4000eb5000fa7000c2a2a2bb6002ab5000f2ab6002bb1
+toData,269,033e0336042ab400061213a2000a0436041d07803e2ab4000a9900081d1010803e2ab4000ec6000b1c9900071d04803e2ab4000fc6001b1d05803e2ab4000f2ab4000ea6000c1c9900081d1008803eb20014b20015b900160200990022b20014b20015121705bd001859032ab600195359041db8001a53b9001b04002b1db9001c02002b2ab40002b9001c02002b2ab40010b9001d020015049900132b2ab40006121e7eb9001c0200a7000d2b2ab40006b9001f02002ab4000a99000d2b2ab4000ab9001c02002b2ab4000db9001f02002ab400072bb800202ab4000ec600101c99000c2a2ab4000e2bb600212ab4000fc6001b2ab4000f2ab4000ea600071c9a000c2a2ab4000f2bb60021b1
 
 com/gemstone/gemfire/internal/cache/wan/GatewaySenderAdvisor$GatewaySenderProfile,4
 fromData,282,2a2bb700082a2bb80009b5000a2a2bb9000b0100b5000c2a2bb9000d0100b5000e2a2bb9000f0100b500102a2bb9000f0100b500112a2bb9000f0100b500122a2bb9000f0100b500132a2bb9000f0100b500142a2bb9000d0100b500152a2bb9000f0100b500162a2bb80017b500042a2bb80017b500052a2bb80017b500062a2bb9000f0100b500182a2bb9000d0100b500192bb8001ab2001bb6001c9c00552bb8001dc0001e4d2cc600412cb6001fb20020b60021b6002299000d2ab20020b50023a7002c2cb6001fb20024b60021b6002299000d2ab20024b50023a700122ab20025b50023a700082a01b50023a7000e2a2bb8001dc00026b500232bb800273d1c9900162abb002859b70029b5002a2ab4002a2bb8002bb1


[16/19] incubator-geode git commit: GEODE-1401: handle NOT_AVAILABLE in callers of getRawOldValue

Posted by kl...@apache.org.
GEODE-1401: handle NOT_AVAILABLE in callers of getRawOldValue


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

Branch: refs/heads/feature/GEODE-1392
Commit: 147c1f01c73516195ad0a7e77f26169d5053de9d
Parents: bb64149
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon May 16 16:57:56 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:26 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/BucketRegionQueue.java         | 10 ++--------
 .../gemstone/gemfire/internal/cache/EntryEventImpl.java   |  2 +-
 .../gemfire/internal/cache/RemoteDestroyMessage.java      |  2 +-
 3 files changed, 4 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/147c1f01/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java
index 0243cde..a3d9d97 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java
@@ -269,10 +269,7 @@ public class BucketRegionQueue extends AbstractBucketRegionQueue {
         requireOldValue, lastModified, overwriteDestroyed);
 
     if (success) {
-      Object ov = event.getRawOldValue();
-      if (ov instanceof GatewaySenderEventImpl) {
-        ((GatewaySenderEventImpl) ov).release();
-      }
+      GatewaySenderEventImpl.release(event.getRawOldValue());
 
       if (getPartitionedRegion().getColocatedWith() == null) {
         return success;
@@ -370,10 +367,7 @@ public class BucketRegionQueue extends AbstractBucketRegionQueue {
     }
     super.basicDestroy(event, cacheWrite, expectedOldValue);
 
-    Object rov = event.getRawOldValue();
-    if (rov instanceof GatewaySenderEventImpl) {
-      ((GatewaySenderEventImpl) rov).release();
-    }
+    GatewaySenderEventImpl.release(event.getRawOldValue());
     // Primary buckets should already remove the key while peeking
     if (!this.getBucketAdvisor().isPrimary()) {
       if (logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/147c1f01/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
index e241622..b833ecb 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
@@ -1407,7 +1407,7 @@ public class EntryEventImpl
         importer.importOldObject(cdV, true);
       }
     } else {
-      importer.importOldObject(ov, true);
+      importer.importOldObject(AbstractRegion.handleNotAvailable(ov), true);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/147c1f01/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
index bb97d9c..f7109bb 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
@@ -232,7 +232,7 @@ public class RemoteDestroyMessage extends RemoteOperationMessageWithDirectReply
           setOldValBytes((byte[]) old);
         } else {
           this.oldValueIsSerialized = true;
-          setOldValObj(old);
+          setOldValObj(AbstractRegion.handleNotAvailable(old));
         }
       }
     }


[08/19] incubator-geode git commit: GEODE-1393 locator returns incorrect server information when starting up

Posted by kl...@apache.org.
GEODE-1393 locator returns incorrect server information when starting up

When a locator auto-reconnects its ServerLocator needs to initialize its
ControllerAdvisor so that it has server information to give to clients.
The ServerLocator was creating a new ControllerAdvisor but didn't ask it
to perform a handshake to fill in its profiles.

ReconnectDUnitTest had an existing testReconnectWithQuorum test that
wasn't doing what it was supposed to.  I've removed the TODO from that
test and modified it to force-disconnect the tests Locator.  The
locator must restart its TcpServer component before it can start
a DistributedSystem, so this exercises the path in
InternalLocator.attemptReconnect() that boots the TcpServer prior to
connecting the DistributedSystem.  After the DistributedSystem
finishes reconnecting the ServerLocator's distribution advisor
should have been initialized by performing the handshake.


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

Branch: refs/heads/feature/GEODE-1392
Commit: d08a047c231d7f2820b4c9d41789caf1b1167be2
Parents: 8beb8af
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon May 16 08:05:00 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:24 2016 -0700

----------------------------------------------------------------------
 .../distributed/internal/InternalLocator.java   |  1 +
 .../distributed/internal/LocatorStats.java      | 31 -----------
 .../distributed/internal/ServerLocator.java     | 44 +++++++--------
 .../gemfire/cache30/ReconnectDUnitTest.java     | 57 ++++++++++----------
 .../internal/ServerLocatorJUnitTest.java        |  0
 5 files changed, 53 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d08a047c/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
index 7effa3d..7ad57ad 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
@@ -1350,6 +1350,7 @@ public class InternalLocator extends Locator implements ConnectListener {
         return response;
       }
     }
+
     private JmxManagerLocatorResponse findJmxManager(JmxManagerLocatorRequest request) {
       JmxManagerLocatorResponse result = null;
       // NYI

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d08a047c/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java
old mode 100644
new mode 100755
index d42a2b4..1140b1f
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java
@@ -117,13 +117,6 @@ public class LocatorStats {
   }
   
   
-  /**
-   * Used by tests to create an instance given its already existings stats.
-   */
-  public LocatorStats(Statistics stats) {
-    this._stats = stats;
-  }
-
   public final void setServerCount(int sc) {
     if(this._stats==null) {
       this.endpoints_known.set(sc);
@@ -140,14 +133,6 @@ public class LocatorStats {
     }
   }
   
-  public final void incLocatorRequests() {
-    if(this._stats==null) {
-      this.requests_to_locator.incrementAndGet();
-    } else {
-      this._stats.incLong(_REQUESTS_TO_LOCATOR, 1);
-    }
-  }  
-  
   public final void endLocatorRequest(long startTime) {
     long took = DistributionStats.getStatTime()-startTime;
     if(this._stats==null) {
@@ -180,14 +165,6 @@ public class LocatorStats {
   
   
   
-  public final void incLocatorResponses() {
-    if(this._stats==null) {
-      this.responses_from_locator.incrementAndGet();
-    } else {
-      this._stats.incLong(_RESPONSES_FROM_LOCATOR, 1);
-    }
-  }  
-  
   public final void setLocatorRequests(long rl) {
     if(this._stats==null) {
       this.requests_to_locator.set(rl);
@@ -218,14 +195,6 @@ public class LocatorStats {
     } else {
       this._stats.incLong(_SERVER_LOAD_UPDATES, 1);
     }
-  }  
-  
-  public void setRequestInProgress(int threads) {
-    if(this._stats!=null) {
-      this._stats.setInt(_REQUESTS_IN_PROGRESS, threads);
-    } else {
-      requestsInProgress.set(threads);
-    }
   }
   
   public void incRequestInProgress(int threads) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d08a047c/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
old mode 100644
new mode 100755
index e535b97..b37a50b
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
@@ -24,12 +24,12 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.CancelCriterion;
@@ -42,7 +42,6 @@ import com.gemstone.gemfire.cache.client.internal.locator.GetAllServersRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.GetAllServersResponse;
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorListRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorListResponse;
-import com.gemstone.gemfire.cache.client.internal.locator.LocatorStatusRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorStatusResponse;
 import com.gemstone.gemfire.cache.client.internal.locator.QueueConnectionRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.QueueConnectionResponse;
@@ -176,34 +175,34 @@ public class ServerLocator implements TcpHandler, DistributionAdvisee {
       logger.debug("ServerLocator: Received request {}", request);
     }
 
-    Object response;
+    if ( ! (request instanceof ServerLocationRequest) ) {
+      throw new InternalGemFireException("Expected ServerLocationRequest, got " + request.getClass());
+    }
 
-    if (request instanceof ServerLocationRequest) {
-      if (request instanceof LocatorStatusRequest) {
+    Object response;
+    int id = ((DataSerializableFixedID)request).getDSFID();
+    switch (id) {
+      case DataSerializableFixedID.LOCATOR_STATUS_REQUEST:
         response = new LocatorStatusResponse()
           .initialize(this.port, this.hostName, this.logFile, this.memberName);
-      }
-      else if (request instanceof LocatorListRequest) {
+        break;
+      case DataSerializableFixedID.LOCATOR_LIST_REQUEST:
         response = getLocatorListResponse((LocatorListRequest) request);
-      }
-      else if (request instanceof ClientReplacementRequest) {
+        break;
+      case DataSerializableFixedID.CLIENT_REPLACEMENT_REQUEST:
         response = pickReplacementServer((ClientReplacementRequest) request);
-      }
-      else if (request instanceof GetAllServersRequest) {
+        break;
+      case DataSerializableFixedID.GET_ALL_SERVERS_REQUEST:
         response = pickAllServers((GetAllServersRequest) request);
-      }
-      else if (request instanceof ClientConnectionRequest) {
+        break;
+      case DataSerializableFixedID.CLIENT_CONNECTION_REQUEST:
         response = pickServer((ClientConnectionRequest) request);
-      }
-      else if (request instanceof QueueConnectionRequest) {
+        break;
+      case DataSerializableFixedID.QUEUE_CONNECTION_REQUEST:
         response = pickQueueServers((QueueConnectionRequest) request);
-      }
-      else {
+        break;
+      default:
         throw new InternalGemFireException("Unknown ServerLocationRequest: " + request.getClass());
-      }
-    }
-    else {
-      throw new InternalGemFireException("Expected ServerLocationRequest, got " + request.getClass());
     }
 
     if(logger.isDebugEnabled()) {
@@ -290,6 +289,9 @@ public class ServerLocator implements TcpHandler, DistributionAdvisee {
       this.loadSnapshot = new LocatorLoadSnapshot();
       this.ds = (InternalDistributedSystem)ds;
       this.advisor = ControllerAdvisor.createControllerAdvisor(this); // escapes constructor but allows field to be final
+      if (ds.isConnected()) {
+        this.advisor.handshake();  // GEODE-1393: need to get server information during restart
+      }
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d08a047c/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
index ca2c17b..6c63def 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
@@ -27,6 +27,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem.ReconnectListener;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.ServerLocator;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManagerHelper;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
@@ -78,6 +79,7 @@ public class ReconnectDUnitTest extends CacheTestCase
           locatorPort = locPort;
           Properties props = getDistributedSystemProperties();
           locator = Locator.startLocatorAndDS(locatorPort, new File(""), props);
+          ReconnectDUnitTest.savedSystem = InternalDistributedSystem.getConnectedInstance();
           IgnoredException.addIgnoredException("com.gemstone.gemfire.ForcedDisconnectException||Possible loss of quorum");
 //          MembershipManagerHelper.getMembershipManager(InternalDistributedSystem.getConnectedInstance()).setDebugJGroups(true);
         } catch (IOException e) {
@@ -163,10 +165,6 @@ public class ReconnectDUnitTest extends CacheTestCase
     return factory.create();
   }
 
-  /*
-  TODO this test is not actually using quorum checks.  To do that it needs to
-  have the locator disconnect & reconnect
-   */
 
   public void testReconnectWithQuorum() throws Exception {
     // quorum check fails, then succeeds
@@ -174,7 +172,7 @@ public class ReconnectDUnitTest extends CacheTestCase
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
-    VM vm2 = host.getVM(2);
+    VM locatorVm = host.getVM(locatorVMNumber);
     
     final int locPort = locatorPort;
 
@@ -210,33 +208,23 @@ public class ReconnectDUnitTest extends CacheTestCase
       }
     };
     
-    System.out.println("creating caches in vm0, vm1 and vm2");
+    System.out.println("creating caches in vm0 and vm1");
     vm0.invoke(create);
     vm1.invoke(create);
-    vm2.invoke(create);
-    
+
     // view is [locator(3), vm0(15), vm1(10), vm2(10)]
     
-    /* now we want to cause vm0 and vm1 to force-disconnect.  This may cause the other
-     * non-locator member to also disconnect, depending on the timing
+    /* now we want to kick out the locator and observe that it reconnects
+     * using its rebooted location service
      */
-    System.out.println("disconnecting vm0");
-    forceDisconnect(vm0);
-    Wait.pause(10000);
-    System.out.println("disconnecting vm1");
-    forceDisconnect(vm1);
+    System.out.println("disconnecting locator");
+    forceDisconnect(locatorVm);
+    waitForReconnect(locatorVm);
+
+    // if the locator reconnected it did so with its own location
+    // service since it doesn't know about any other locators
+    ensureLocationServiceRunning(locatorVm);
 
-    /* now we wait for them to auto-reconnect */
-    try {
-      System.out.println("waiting for vm0 to reconnect");
-      waitForReconnect(vm0);
-      System.out.println("waiting for vm1 to reconnect");
-      waitForReconnect(vm1);
-      System.out.println("done reconnecting vm0 and vm1");
-    } catch (Exception e) {
-      ThreadUtils.dumpAllStacks();
-      throw e;
-    }
   }
   
   public void testReconnectOnForcedDisconnect() throws Exception  {
@@ -418,6 +406,19 @@ public class ReconnectDUnitTest extends CacheTestCase
       }
     });
   }
+
+  /** this will throw an exception if location services aren't running */
+  private void ensureLocationServiceRunning(VM vm) {
+    vm.invoke(new SerializableRunnable("ensureLocationServiceRunning") {
+      public void run() {
+        InternalLocator intloc = (InternalLocator)locator;
+        ServerLocator serverLocator = intloc.getServerLocatorAdvisee();
+        // the initialization flag in the locator's ControllerAdvisor will
+        // be set if a handshake has been performed
+        assertTrue(serverLocator.getDistributionAdvisor().isInitialized());
+      }
+    });
+  }
   
   private DistributedMember waitForReconnect(VM vm) {
     return (DistributedMember)vm.invoke(new SerializableCallable("wait for Reconnect and return ID") {
@@ -456,7 +457,7 @@ public class ReconnectDUnitTest extends CacheTestCase
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
-    VM vm3 = host.getVM(3);
+    VM locatorVm = host.getVM(3);
     DistributedMember dm, newdm;
     
     final int locPort = locatorPort;
@@ -467,7 +468,7 @@ public class ReconnectDUnitTest extends CacheTestCase
     final String xmlFileLoc = (new File(".")).getAbsolutePath();
 
     //This locator was started in setUp.
-    File locatorViewLog = new File(vm3.getWorkingDirectory(), "locator"+locatorPort+"views.log");
+    File locatorViewLog = new File(locatorVm.getWorkingDirectory(), "locator"+locatorPort+"views.log");
     assertTrue("Expected to find " + locatorViewLog.getPath() + " file", locatorViewLog.exists());
     long logSize = locatorViewLog.length();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d08a047c/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java
old mode 100644
new mode 100755


[13/19] incubator-geode git commit: GEODE-1351: add waitUntilFlush() into luceneIndexImpl

Posted by kl...@apache.org.
GEODE-1351: add waitUntilFlush() into luceneIndexImpl

This function is very useful in test and demo.
It will wait for the AEQ size to be 0

Also added junit test and integration tests.


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

Branch: refs/heads/feature/GEODE-1392
Commit: 8ea532002fc1bc3e0d442986a108f719a61757e2
Parents: c54227c
Author: zhouxh <gz...@pivotal.io>
Authored: Sun May 15 21:36:34 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:25 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneIndex.java       |  5 ++
 .../lucene/internal/LuceneEventListener.java    | 11 +++
 .../cache/lucene/internal/LuceneIndexImpl.java  | 43 +++++++++--
 .../internal/xml/LuceneIndexCreation.java       |  4 +
 .../gemfire/cache/lucene/LuceneQueriesBase.java | 69 ++++++++++++++++-
 .../internal/LuceneIndexImplJUnitTest.java      | 78 ++++++++++++++++++++
 6 files changed, 202 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8ea53200/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
index 743045b..be329f7 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
@@ -56,4 +56,9 @@ public interface LuceneIndex {
    */
   public Map<String, Analyzer> getFieldAnalyzers();
   
+  /* 
+   * wait until the current entries in cache are indexed
+   */
+  public void waitUntilFlushed(int maxWait);
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8ea53200/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
index 9fdfd43..2dae4ee 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
@@ -35,6 +35,8 @@ import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
+import com.gemstone.gemfire.internal.cache.CacheObserverHolder;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy.TestHook;
 import com.gemstone.gemfire.internal.logging.LogService;
 
 /**
@@ -70,6 +72,10 @@ public class LuceneEventListener implements AsyncEventListener {
         IndexRepository repository = repositoryManager.getRepository(region, key, callbackArgument);
 
         Operation op = event.getOperation();
+        
+        if (testHook != null) {
+          testHook.doTestHook("FOUND_AND_BEFORE_PROCESSING_A_EVENT");
+        }
 
         if (op.isCreate()) {
           repository.create(key, event.getDeserializedValue());
@@ -96,4 +102,9 @@ public class LuceneEventListener implements AsyncEventListener {
       DefaultQuery.setPdxReadSerialized(false);
     }
   }
+  
+  public interface TestHook {
+    public void doTestHook(String spot);
+  }
+  public static TestHook testHook;
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8ea53200/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
index 0b5f8fa..981d9e4 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
@@ -21,23 +21,26 @@ package com.gemstone.gemfire.cache.lucene.internal;
 
 import java.util.Collections;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
-import com.gemstone.gemfire.InternalGemFireError;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
 import com.gemstone.gemfire.cache.lucene.internal.xml.LuceneIndexCreation;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 
 public abstract class LuceneIndexImpl implements InternalLuceneIndex {
@@ -77,7 +80,33 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
   protected void setSearchableFields(String[] fields) {
     searchableFieldNames = fields;
   }
-  
+
+  /*
+   *  For test and demo purpose. To use it, the data region should stop feeding
+   *  A more advanced version is under-development
+   */
+  @Override
+  public void waitUntilFlushed(int maxWait) {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath);
+    AsyncEventQueue queue = (AsyncEventQueue)cache.getAsyncEventQueue(aeqId);
+    if (queue != null) {
+      long start = System.nanoTime();
+      while (System.nanoTime() - start < TimeUnit.MILLISECONDS.toNanos(maxWait)) {
+        if (0 == queue.size()) {
+          logger.debug("waitUntilFlushed: Queue size is 0");
+          break;
+        } else {
+          try {
+            Thread.sleep(200);
+          } catch (InterruptedException e) {
+          }
+        }
+      }
+    } else { 
+      throw new IllegalArgumentException("The AEQ does not exist for the index "+indexName+" region "+regionPath);
+    }
+  }
+
   @Override
   public String[] getFieldNames() {
     return searchableFieldNames;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8ea53200/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
index 86a10e4..b54f51b 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
@@ -114,4 +114,8 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
   public void addFieldNames(String[] fieldNames) {
     this.fieldNames.addAll(Arrays.asList(fieldNames));
   }
+
+  @Override
+  public void waitUntilFlushed(int maxWait) {
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8ea53200/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
index c467a18..c7567f3 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -18,7 +18,7 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
 import java.io.Serializable;
 import java.util.HashMap;
@@ -27,6 +27,12 @@ import java.util.Map;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneEventListener;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexImpl;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
+import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.VM;
@@ -75,6 +81,67 @@ public abstract class LuceneQueriesBase extends JUnit4CacheTestCase {
     executeTextSearch(accessor);
   }
 
+  @Test
+  public void entriesFlushedToIndexAfterWaitForFlushCalled() {
+    SerializableRunnableIF createIndex = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+    };
+    dataStore1.invoke(() -> initDataStore(createIndex));
+    dataStore2.invoke(() -> initDataStore(createIndex));
+    accessor.invoke(() -> initAccessor(createIndex));
+
+    try {
+      dataStore1.invoke(() -> setTestHook());
+      putDataInRegion(accessor);
+      waitForFlushBeforeExecuteTextSearch(accessor, 10);
+      executeTextSearch(accessor);
+    } finally {
+      dataStore1.invoke(() -> checkResultAndresetTestHook());
+    }
+  }
+
+  protected void waitForFlushBeforeExecuteTextSearch(VM vm, final int expectKeyNum) {
+    vm.invoke(() -> {
+      Cache cache = getCache();
+      Region<Object, Object> region = cache.getRegion(REGION_NAME);
+
+      LuceneService service = LuceneServiceProvider.get(cache);
+      LuceneIndexImpl index = (LuceneIndexImpl)service.getIndex(INDEX_NAME, REGION_NAME);
+      assertNotNull(index);
+      LuceneQuery<Integer, TestObject> query;
+
+      String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
+      AsyncEventQueue queue = cache.getAsyncEventQueue(aeqId);
+      assertNotNull(queue);
+      assertTrue(queue.size()>0);
+      index.waitUntilFlushed(30000);
+      return null;
+    });
+  }
+
+  public static void setTestHook() {
+    LuceneEventListener.testHook = new LuceneEventListener.TestHook() {
+
+      @Override
+      public void doTestHook(String spot) {
+        if (spot.equals("FOUND_AND_BEFORE_PROCESSING_A_EVENT")) {
+          try {
+            Thread.sleep(1000);
+            LogService.getLogger().debug("Waited in test hook");
+          }
+          catch (InterruptedException e) {
+          }
+        }
+      }
+    };
+  }
+  
+  public static void checkResultAndresetTestHook()
+  {
+    LuceneEventListener.testHook = null;
+  }
+
   protected void executeTextSearch(VM vm) {
     vm.invoke(() -> {
       Cache cache = getCache();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8ea53200/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
new file mode 100755
index 0000000..edecc66
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import static org.mockito.Mockito.*;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.test.fake.Fakes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class LuceneIndexImplJUnitTest {
+  public static final String REGION = "region";
+  public static final String INDEX = "index";
+  public static final int MAX_WAIT = 30000;
+  private Cache cache;
+  LuceneIndex index;
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+  
+  @Before
+  public void createLuceneIndex() {
+    cache = Fakes.cache();
+    index = new LuceneIndexForPartitionedRegion(INDEX, REGION, cache);
+  }
+  
+  @Test
+  public void waitUnitFlushedWithMissingAEQThrowsIllegalArgument() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    index.waitUntilFlushed(MAX_WAIT);
+  }
+  
+  @Test
+  public void waitUnitFlushedWaitsForFlush() throws Exception {
+    final String expectedIndexName = LuceneServiceImpl.getUniqueIndexName(INDEX, REGION);
+    final AsyncEventQueue queue = mock(AsyncEventQueue.class);
+    when(cache.getAsyncEventQueue(eq(expectedIndexName))).thenReturn(queue);
+    
+    AtomicInteger callCount = new AtomicInteger();
+    when(queue.size()).thenAnswer(invocation -> {
+      if (callCount.get() == 0) {
+        // when the waitUnitFlushed() called the 2nd time, queue.size() will return 0
+        callCount.incrementAndGet();
+        return 2;
+      } else {
+        // when the waitUnitFlushed() called the 2nd time, queue.size() will return 0
+        return 0;
+      }
+    });
+    index.waitUntilFlushed(MAX_WAIT);
+    verify(cache).getAsyncEventQueue(eq(expectedIndexName));
+  }
+
+}


[04/19] incubator-geode git commit: GEODE-430: Fixing races in UpdatePropagationDUnitTest

Posted by kl...@apache.org.
GEODE-430: Fixing races in UpdatePropagationDUnitTest

This test was failing with a couple of different race conditions

1) It was not waiting for updates to make it to the second client,
causing assertion failures. I added a wait.

2) It was occasionally using a connection that was previously made to
the server before it was killed, causing an unexpected socket closed. I
refactored the test to not use internal APIs to do the put, but instead
verify that the event was not sent to the client using a listener.

3) In the PR version of the test, the PR single hop code can return
different addresses for the server, resulting in the pool thinking it
has two different servers when it only has one. I changed the wait to
wait for a server with a given port to go away.

I also did some cleanup of the test - extended JUnit4CacheTestCase,
removed the waits, used awaitility, removed an almost duplicate disabled
test method and added the one extra assertion from that duplicate test
to the enabled test method.


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

Branch: refs/heads/feature/GEODE-1392
Commit: a49978e3d45c101a2fc0e6ec4dca46ae48c5c83b
Parents: 96b3df9
Author: Dan Smith <up...@apache.org>
Authored: Wed May 11 13:04:57 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:23 2016 -0700

----------------------------------------------------------------------
 .../sockets/UpdatePropagationDUnitTest.java     | 489 +++++--------------
 .../sockets/UpdatePropagationPRDUnitTest.java   |   4 +-
 2 files changed, 136 insertions(+), 357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a49978e3/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
index 69781c4..08d964a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
@@ -16,44 +16,50 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
+import static junit.framework.TestCase.assertNotNull;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
-import org.junit.Ignore;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.CacheWriterException;
 import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.EntryEvent;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
 import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.client.internal.EndpointManager;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
 import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
-import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
 import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
-import com.gemstone.gemfire.test.dunit.WaitCriterion;
-import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.jayway.awaitility.Awaitility;
+import com.jayway.awaitility.core.ConditionTimeoutException;
 
 /**
  * Start client 1
@@ -71,7 +77,8 @@ import com.gemstone.gemfire.test.junit.categories.FlakyTest;
  * The key is to verify that the memberid being used by the client
  * to register with the server is the same across servers
  */
-public class UpdatePropagationDUnitTest extends DistributedTestCase {
+@Category(DistributedTest.class)
+public class UpdatePropagationDUnitTest extends JUnit4CacheTestCase {
 
   VM server1 = null;
 
@@ -87,13 +94,6 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
 
   private static final String REGION_NAME = "UpdatePropagationDUnitTest_region";
 
-  protected static Cache cache = null;
-  
-  /** constructor */
-  public UpdatePropagationDUnitTest(String name) {
-    super(name);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     disconnectAllFromDS();
@@ -111,382 +111,169 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
     //client 2 VM
     client2 = host.getVM(3);
     
-    PORT1 =  ((Integer)server1.invoke(() -> createServerCache())).intValue();
-    PORT2 =  ((Integer)server2.invoke(() -> createServerCache())).intValue();
+    PORT1 =  server1.invoke(() -> createServerCache());
+    PORT2 =  server2.invoke(() -> createServerCache());
 
     client1.invoke(() -> createClientCache(
-      NetworkUtils.getServerHostName(server1.getHost()), new Integer(PORT1),new Integer(PORT2)));
+      NetworkUtils.getServerHostName(server1.getHost()), PORT1, PORT2));
     client2.invoke(() -> createClientCache(
-      NetworkUtils.getServerHostName(server1.getHost()), new Integer(PORT1),new Integer(PORT2)));
+      NetworkUtils.getServerHostName(server1.getHost()), PORT1, PORT2));
     
     IgnoredException.addIgnoredException("java.net.SocketException");
     IgnoredException.addIgnoredException("Unexpected IOException");
   }
 
-  private void createCache(Properties props) throws Exception
-  {
-    DistributedSystem ds = getSystem(props);
-    cache = CacheFactory.create(ds);
-    assertNotNull(cache);
-  }
-
-  static private final String WAIT_PROPERTY =
-    "UpdatePropagationDUnitTest.maxWaitTime";
-  static private final int WAIT_DEFAULT = 60000;
-
-
-  /**
-   * This tests whether the updates are received by the sender or not if
-   * there is an Interest List failover
-   *
-   */
-  @Ignore("Bug 50405")
-  public void DISABLED_testVerifyUpdatesNotReceivedBySender()
-  {
-    final int maxWaitTime = Integer.getInteger(WAIT_PROPERTY, WAIT_DEFAULT).intValue();
-
-    //First create entries on both servers via the two client
-    client1.invoke(() -> createEntriesK1andK2());
-    client2.invoke(() -> createEntriesK1andK2());
-    client1.invoke(() -> registerKeysK1andK2());
-    client2.invoke(() -> registerKeysK1andK2());
-    //Induce fail over of InteretsList Endpoint to Server 2 by killing server1
-    
-    server1.invoke(() -> UpdatePropagationDUnitTest.killServer(new Integer(PORT1)));
-    //Wait for 10 seconds to allow fail over. This would mean that Interest
-    // has failed over to Server2.
-    client1.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong1"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
-
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() != 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
-
-    client2.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong3"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
-
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() != 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
-
-    //Start Server1 again so that both clients1 & Client 2 will establish connection to server1 too.
-    server1.invoke(() -> UpdatePropagationDUnitTest.startServer(new Integer(PORT1)));
-
-    client1.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong2"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
-
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() == 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
-
-    //Do a put on Server1 via Connection object from client1.
-    // Client1 should not receive updated value while client2 should receive
-    client1.invoke(() -> acquireConnectionsAndPutonK1andK2( NetworkUtils.getServerHostName(client1.getHost())));
-    //pause(5000);
-    //Check if both the puts ( on key1 & key2 ) have reached the servers
-    server1.invoke(() -> verifyUpdates());
-    server2.invoke(() -> verifyUpdates());
-    // verify no updates for update originator
-    client1.invoke(() -> verifyNoUpdates());
-
-  }
-
-
   /**
    * This tests whether the updates are received by other clients or not , if there are
    * situation of Interest List fail over
    */
-  @Category(FlakyTest.class) // GEODE-430: time sensitive, random ports, port reuse, thread sleeps (5 seconds), eats exceptions (fixed 1), async actions, waitForCriterion
-  public void testVerifyUpdatesReceivedByOtherClients() {
-    final int maxWaitTime = Integer.getInteger(WAIT_PROPERTY, WAIT_DEFAULT).intValue();
+  @Test
+  public void updatesAreProgegatedAfterFailover() {
     //  First create entries on both servers via the two client
     client1.invoke(() -> createEntriesK1andK2());
     client2.invoke(() -> createEntriesK1andK2());
     client1.invoke(() -> registerKeysK1andK2());
     client2.invoke(() -> registerKeysK1andK2());
     //Induce fail over of InteretsList Endpoint to Server 2 by killing server1
-    server1.invoke(() -> UpdatePropagationDUnitTest.killServer(new Integer(PORT1)));
+    server1.invoke(() -> killServer(new Integer(PORT1)));
     //Wait for 10 seconds to allow fail over. This would mean that Interstist has failed
     // over to Server2.
-    client1.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong3"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
-
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() != 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
-    client2.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
+    final CacheSerializableRunnable waitToDetectDeadServer = new CacheSerializableRunnable("Wait for server on port1 to be dead") {
       public void run2() throws CacheException
       {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong3"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
+        Region r = getCache().getRegion(REGION_NAME);
 
         String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() != 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
+        final PoolImpl pool = (PoolImpl) PoolManager.find(poolName);
+        Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> !hasEndPointWithPort(pool, PORT1));
       }
-    });
+    };
+    client1.invoke(waitToDetectDeadServer);
+    client2.invoke(waitToDetectDeadServer);
 
     //Start Server1 again so that both clients1 & Client 2 will establish connection to server1 too.
-    server1.invoke(() -> UpdatePropagationDUnitTest.startServer(new Integer(PORT1)));
-
-    client1.invoke(new CacheSerializableRunnable("Wait for servers to be alive") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() == 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
+    server1.invoke(() -> startServer(new Integer(PORT1)));
 
-    client2.invoke(new CacheSerializableRunnable("Wait for servers to be alive") {
+    final CacheSerializableRunnable waitToDetectLiveServer = new CacheSerializableRunnable("Wait for servers to be alive") {
       public void run2() throws CacheException
       {
-        Region r = cache.getRegion(REGION_NAME);
+        Region r = getCache().getRegion(REGION_NAME);
         String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() == 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
+        final PoolImpl pool = (PoolImpl) PoolManager.find(poolName);
+        Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> hasEndPointWithPort(pool, PORT1));
       }
-    });
-    
-    Wait.pause(5000);
+    };
+    client1.invoke(waitToDetectLiveServer);
+    client2.invoke(waitToDetectLiveServer);
 
     //Do a put on Server1 via Connection object from client1.
     // Client1 should not receive updated value while client2 should receive
     client1.invoke(() -> acquireConnectionsAndPutonK1andK2( NetworkUtils.getServerHostName(client1.getHost())));
-    Wait.pause(5000);
     //Check if both the puts ( on key1 & key2 ) have reached the servers
     server1.invoke(() -> verifyUpdates());
     server2.invoke(() -> verifyUpdates());
     // verify updates to other client
     client2.invoke(() -> verifyUpdates());
+
+    // verify no updates for update originator
+    client1.invoke(() -> verifySenderUpdateCount());
+  }
+
+  /**
+   * Check to see if a client is connected to an endpoint with a specific port
+   */
+  private boolean hasEndPointWithPort(final PoolImpl pool, final int port) {
+    EndpointManager endpointManager = pool.getEndpointManager();
+    final Set<ServerLocation> servers = endpointManager
+      .getEndpointMap().keySet();
+    return servers.stream().anyMatch(location -> location.getPort() == port);
   }
 
   public void acquireConnectionsAndPutonK1andK2(String host)
   {
-    try {
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-      assertNotNull(r1);
-      String poolName = r1.getAttributes().getPoolName();
-      assertNotNull(poolName);
-      PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-      assertNotNull(pool);
-
-      Connection conn = pool.acquireConnection(new ServerLocation(host,PORT1));
-      assertNotNull(conn);
-      assertEquals(PORT1, conn.getServer().getPort());
-      ServerRegionProxy srp = new ServerRegionProxy(Region.SEPARATOR+ REGION_NAME, pool);
-      srp.putOnForTestsOnly(conn, "key1", "server-value1", new EventID(new byte[] {1},159632,1), null);
-      srp.putOnForTestsOnly(conn, "key2", "server-value2", new EventID(new byte[] {1},159632,2), null);
-    }
-    catch (Exception ex) {
-      Assert.fail("while setting acquireConnections", ex);
-    }
+    Region r1 = getCache().getRegion(Region.SEPARATOR + REGION_NAME);
+    r1.put("key1", "server-value1");
+    r1.put("key2", "server-value2");
   }
 
-  public static void killServer(Integer port )
+  public void killServer(Integer port )
   {
-    try {
-      Iterator iter = cache.getCacheServers().iterator();
-      if (iter.hasNext()) {
-        CacheServer server = (CacheServer)iter.next();
-        if(server.getPort() == port.intValue()){
-          server.stop();
-        }
-
+    Iterator iter = getCache().getCacheServers().iterator();
+    if (iter.hasNext()) {
+      CacheServer server = (CacheServer)iter.next();
+      if(server.getPort() == port.intValue()){
+        server.stop();
       }
-    }
-    catch (Exception ex) {
-      fail("while killing Server  " + ex);
+
     }
   }
 
-  public static void startServer(Integer port)
+  public void startServer(Integer port) throws IOException
   {
-    try {
-      CacheServer server1 = cache.addCacheServer();
-      server1.setPort(port.intValue());
-      server1.setNotifyBySubscription(true);
-      server1.start();
-    }
-    catch (Exception ex) {
-      fail("while killServer  " + ex);
-    }
+    CacheServer server1 = getCache().addCacheServer();
+    server1.setPort(port.intValue());
+    server1.setNotifyBySubscription(true);
+    server1.start();
   }
 
   /**
    * Creates entries on the server
    *
    */
-  public static void createEntriesK1andK2()
+  public void createEntriesK1andK2()
   {
-    try {
-      Region r1 = cache.getRegion(Region.SEPARATOR+REGION_NAME);
-      assertNotNull(r1);
-      if (!r1.containsKey("key1")) {
-        r1.put("key1", "key-1");
-      }
-      if (!r1.containsKey("key2")) {
-        r1.put("key2", "key-2");
-      }
-      assertEquals(r1.get("key1"), "key-1");
-      if (r1.getAttributes().getPartitionAttributes() == null) {
-        assertEquals(r1.getEntry("key1").getValue(), "key-1");
-        assertEquals(r1.getEntry("key2").getValue(), "key-2");
-      }
-      else {
-        assertEquals(r1.get("key1"), "key-1");
-        assertEquals(r1.get("key2"), "key-2");
-      }
+    Region r1 = getCache().getRegion(Region.SEPARATOR+REGION_NAME);
+    assertNotNull(r1);
+    if (!r1.containsKey("key1")) {
+      r1.put("key1", "key-1");
     }
-    catch (Exception ex) {
-      Assert.fail("failed while createEntriesK1andK2()", ex);
+    if (!r1.containsKey("key2")) {
+      r1.put("key2", "key-2");
+    }
+    assertEquals(r1.get("key1"), "key-1");
+    if (r1.getAttributes().getPartitionAttributes() == null) {
+      assertEquals(r1.getEntry("key1").getValue(), "key-1");
+      assertEquals(r1.getEntry("key2").getValue(), "key-2");
+    }
+    else {
+      assertEquals(r1.get("key1"), "key-1");
+      assertEquals(r1.get("key2"), "key-2");
     }
   }
 
-  public static void createClientCache(String host, Integer port1 , Integer port2 ) throws Exception
+  public void createClientCache(String host, Integer port1 , Integer port2 ) throws Exception
   {
-    int PORT1 = port1.intValue() ;
-    int PORT2 = port2.intValue();
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new UpdatePropagationDUnitTest("temp").createCache(props);
-    Pool p;
+    ClientCache cache;
     try {
       System.setProperty("gemfire.PoolImpl.DISABLE_RANDOM", "true");
-      p = PoolManager.createFactory()
-        .addServer(host, PORT1)
-        .addServer(host, PORT2)
-        .setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(-1)
-        .setMinConnections(4)
-        .setSocketBufferSize(1000)
-        .setReadTimeout(2000)
-        // .setRetryInterval(250)
-        // .setRetryAttempts(2)
-        .create("UpdatePropagationDUnitTestPool");
+      int PORT1 = port1.intValue() ;
+      int PORT2 = port2.intValue();
+      Properties props = new Properties();
+      props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+      props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+      ClientCacheFactory cf = new ClientCacheFactory();
+      cf.addPoolServer(host, PORT1)
+      .addPoolServer(host, PORT2)
+      .setPoolSubscriptionEnabled(true)
+      .setPoolSubscriptionRedundancy(-1)
+      .setPoolMinConnections(4)
+      .setPoolSocketBufferSize(1000)
+      .setPoolReadTimeout(2000)
+      .setPoolPingInterval(300);
+       cache = getClientCache(cf);
     } finally {
       System.setProperty("gemfire.PoolImpl.DISABLE_RANDOM", "false");
       CacheServerTestUtil.enableShufflingOfEndpoints();
     }
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-    RegionAttributes attrs = factory.create();
-    cache.createRegion(REGION_NAME, attrs);
-
+    cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
+      .addCacheListener(new EventTrackingCacheListener())
+      .create(REGION_NAME);
   }
 
   public Integer createServerCache() throws Exception
   {
-    new UpdatePropagationDUnitTest("temp").createCache(new Properties());
-    RegionAttributes attrs = createCacheServerAttributes(); 
+    Cache cache = getCache();
+    RegionAttributes attrs = createCacheServerAttributes();
     cache.createRegion(REGION_NAME, attrs);
     CacheServer server = cache.addCacheServer();
     assertNotNull(server);
@@ -505,10 +292,10 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
     return factory.create();
   }
 
-  public static void registerKeysK1andK2()
+  public void registerKeysK1andK2()
   {
     try {
-      Region r = cache.getRegion(Region.SEPARATOR+ REGION_NAME);
+      Region r = getCache().getRegion(Region.SEPARATOR+ REGION_NAME);
       assertNotNull(r);
       List list = new ArrayList();
       list.add("key1");
@@ -521,25 +308,22 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
     }
   }
 
-  public static void verifyNoUpdates()
+  public void verifySenderUpdateCount()
   {
-    try {
-      Region r = cache.getRegion(Region.SEPARATOR+ REGION_NAME);
-      assertNotNull(r);
-      // verify no updates
-      assertEquals("key-1", r.getEntry("key1").getValue());
-      assertEquals("key-2", r.getEntry("key2").getValue());
-    }
-    catch (Exception ex) {
-      Assert.fail("failed while verifyNoUpdates()", ex);
-    }
+    Region r = getCache().getRegion(Region.SEPARATOR+ REGION_NAME);
+    EventTrackingCacheListener listener = (EventTrackingCacheListener) r.getAttributes().getCacheListeners()[0];
+
+    final List<EntryEvent> events = listener.receivedEvents;
+
+    //We only expect to see 1 create and 1 update from the original put
+    assertEquals("Expected only 2 events for key1", 2, events.stream().filter(event -> event.getKey().equals("key1")).count());
+    assertEquals("Expected only 2 events for key2", 2, events.stream().filter(event -> event.getKey().equals("key2")).count());
   }
 
-  public static void verifyUpdates()
+  public void verifyUpdates()
   {
-    try {
-      Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-      assertNotNull(r);
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      Region r = getCache().getRegion(Region.SEPARATOR + REGION_NAME);
       // verify updates
       if (r.getAttributes().getPartitionAttributes() == null) {
         assertEquals("server-value2", r.getEntry("key2").getValue());
@@ -549,28 +333,25 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
         assertEquals("server-value2", r.get("key2"));
         assertEquals("server-value1", r.get("key1"));
       }
+    });
+  }
+
+  private static class EventTrackingCacheListener extends CacheListenerAdapter {
+    List<EntryEvent> receivedEvents = new ArrayList<>();
+
+    @Override public void afterCreate(final EntryEvent event) {
+      receivedEvents.add(event);
     }
-    catch (Exception ex) {
-      Assert.fail("failed while region", ex);
+
+    @Override public void afterUpdate(final EntryEvent event) {
+      receivedEvents.add(event);
     }
-  }
 
-  public static void closeCache()
-  {
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
+    @Override public void afterDestroy(final EntryEvent event) {
+      receivedEvents.add(event);
     }
-  }
 
-  @Override
-  public final void preTearDown() throws Exception {
-    //close client
-    client1.invoke(() -> closeCache());
-    client2.invoke(() -> closeCache());
-    //close server
-    server1.invoke(() -> closeCache());
-    server2.invoke(() -> closeCache());
+
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a49978e3/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java
index 8edac45..589b455 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java
@@ -24,9 +24,7 @@ import com.gemstone.gemfire.cache.*;
  */
 public class UpdatePropagationPRDUnitTest extends UpdatePropagationDUnitTest {
 
-  public UpdatePropagationPRDUnitTest(String name) {
-    super(name);
-  }
+  @Override
   protected RegionAttributes createCacheServerAttributes()
   {
     AttributesFactory factory = new AttributesFactory();


[12/19] incubator-geode git commit: GEODE-17: make geode authorization case-sensitive since our region names are case sensitive

Posted by kl...@apache.org.
GEODE-17: make geode authorization case-sensitive since our region names are case sensitive

* Specify case sensitive when creating the permission context
* Specify case sensitive when resolving the permission from shiro-ini file
* rename shiro-init to security-shiro-init since it's security related in DistributionConfig
* For DATA operations, a "NULL" regionName is used when regionName couldn't be resolved yet. Since for permissions,
  DATA:READ is different from DATA:READ:NULL


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

Branch: refs/heads/feature/GEODE-1392
Commit: 1179c08eb4f9d1fe1c1ffea337a34a0f1c6c89c7
Parents: 758643c
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri May 13 14:04:10 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed May 18 10:04:25 2016 -0700

----------------------------------------------------------------------
 .../cache/operations/OperationContext.java      | 10 ++++-
 .../internal/AbstractDistributionConfig.java    |  2 +-
 .../internal/DistributionConfig.java            | 26 ++++++++-----
 .../gemfire/internal/AbstractConfig.java        |  2 +-
 .../internal/security/GeodeSecurityUtil.java    | 41 ++++++++++++++++++++
 .../security/shiro/GeodePermissionResolver.java | 28 +++++++++++++
 .../internal/SystemManagementService.java       | 29 ++------------
 .../security/ResourceOperationContext.java      | 10 ++++-
 .../CacheServerMBeanShiroJUnitTest.java         |  2 +-
 .../security/DataCommandsSecurityTest.java      |  4 +-
 .../GeodeSecurityUtilCustomRealmJUnitTest.java  | 18 ++-------
 .../GeodeSecurityUtilWithIniFileJUnitTest.java  | 15 ++++---
 .../security/GfshCommandsSecurityTest.java      |  2 +-
 .../ResourceOperationContextJUnitTest.java      | 11 ++++--
 .../internal/security/ShiroCacheStartRule.java  |  2 +-
 .../internal/security/TestCommand.java          | 16 ++++----
 16 files changed, 140 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
index dec716c..b81016d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
@@ -304,7 +304,7 @@ public abstract class OperationContext extends WildcardPermission{
   }
 
   public String getRegionName(){
-    return "NULL";
+    return null;
   }
 
   /**
@@ -358,4 +358,12 @@ public abstract class OperationContext extends WildcardPermission{
         || opCode.isRegionDestroy() || opCode.isRegionClear());
   }
 
+  @Override
+  public String toString(){
+    if(getRegionName()==null)
+      return getResource()+":"+getOperationCode();
+    else
+      return getResource()+":"+getOperationCode()+":"+getRegionName();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
index d38e1a9..17e7c2b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
@@ -1125,7 +1125,7 @@ public abstract class AbstractDistributionConfig
     m.put(LOCK_MEMORY_NAME, LocalizedStrings.AbstractDistributionConfig_LOCK_MEMORY.toLocalizedString(DEFAULT_LOCK_MEMORY));
     m.put(DISTRIBUTED_TRANSACTIONS_NAME, "Flag to indicate whether all transactions including JTA should be distributed transactions.  Default is false, meaning colocated transactions.");
 
-    m.put(SHIRO_INIT_NAME, "The name of the shiro configuration file in the classpath, e.g. shiro.ini");
+    m.put(SECURITY_SHIRO_INIT_NAME, "The name of the shiro configuration file in the classpath, e.g. shiro.ini");
 
     dcAttDescriptions = Collections.unmodifiableMap(m);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
index c0e560c..36ef671 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
@@ -17,6 +17,17 @@
 
 package com.gemstone.gemfire.distributed.internal;
 
+import java.io.File;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.Config;
 import com.gemstone.gemfire.internal.ConfigSource;
@@ -25,12 +36,6 @@ import com.gemstone.gemfire.internal.logging.LogConfig;
 import com.gemstone.gemfire.internal.tcp.Connection;
 import com.gemstone.gemfire.memcached.GemFireMemcachedServer;
 
-import java.io.File;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.net.InetAddress;
-import java.util.*;
-
 /**
  * Provides accessor (and in some cases mutator) methods for the
  * various GemFire distribution configuration properties.  The
@@ -47,7 +52,8 @@ import java.util.*;
  *
  * @since 2.1
  */
-public interface DistributionConfig extends Config, LogConfig {
+public interface
+DistributionConfig extends Config, LogConfig {
 
   ////////////////////  Instance Methods  ////////////////////
 
@@ -3739,11 +3745,11 @@ public interface DistributionConfig extends Config, LogConfig {
   public void setLockMemory(boolean value);
 
   @ConfigAttribute(type=String.class)
-  public String SHIRO_INIT_NAME="shiro-init";
+  public String SECURITY_SHIRO_INIT_NAME ="security-shiro-init";
 
-  @ConfigAttributeSetter(name=SHIRO_INIT_NAME)
+  @ConfigAttributeSetter(name= SECURITY_SHIRO_INIT_NAME)
   public void setShiroInit(String value);
-  @ConfigAttributeGetter(name=SHIRO_INIT_NAME)
+  @ConfigAttributeGetter(name= SECURITY_SHIRO_INIT_NAME)
   public String getShiroInit();
 
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
index a4c2f2f..93cb9b2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
@@ -172,7 +172,7 @@ public abstract class AbstractConfig implements Config {
         }
       }
       // hide the shiro-init configuration for now. Remove after we can allow customer to specify shiro.ini file
-      if(attName.equals("shiro-init")){
+      if(attName.equals(DistributionConfig.SECURITY_SHIRO_INIT_NAME)){
         continue;
       }
       pw.print(attName);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index 6e10f3f..236b00b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -18,13 +18,16 @@
 package com.gemstone.gemfire.internal.security;
 
 import java.security.AccessController;
+import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
 import com.gemstone.gemfire.cache.operations.OperationContext;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.cache.operations.OperationContext.Resource;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.security.shiro.CustomAuthRealm;
 import com.gemstone.gemfire.internal.security.shiro.ShiroPrincipal;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
 import com.gemstone.gemfire.management.internal.security.ResourceOperationContext;
@@ -37,6 +40,11 @@ import org.apache.shiro.SecurityUtils;
 import org.apache.shiro.ShiroException;
 import org.apache.shiro.UnavailableSecurityManagerException;
 import org.apache.shiro.authc.UsernamePasswordToken;
+import org.apache.shiro.config.Ini.Section;
+import org.apache.shiro.config.IniSecurityManagerFactory;
+import org.apache.shiro.mgt.DefaultSecurityManager;
+import org.apache.shiro.mgt.SecurityManager;
+import org.apache.shiro.realm.Realm;
 import org.apache.shiro.subject.Subject;
 import org.apache.shiro.util.ThreadContext;
 
@@ -211,4 +219,37 @@ public class GeodeSecurityUtil {
     return true;
   }
 
+  /**
+   * initialize Shiro's Security Manager and Security Utilities
+   * @param securityProps
+   */
+  public static void initSecurity(Properties securityProps){
+    if(securityProps==null)
+      return;
+
+    String shiroConfig = securityProps.getProperty(DistributionConfig.SECURITY_SHIRO_INIT_NAME);
+    String customAuthenticator =securityProps.getProperty(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME);
+    if (!com.gemstone.gemfire.internal.lang.StringUtils.isBlank(shiroConfig)) {
+      IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:"+shiroConfig);
+
+      // we will need to make sure that shiro uses a case sensitive permission resolver
+      Section main = factory.getIni().addSection("main");
+      main.put("geodePermissionResolver", "com.gemstone.gemfire.internal.security.shiro.GeodePermissionResolver");
+      if(!main.containsKey("iniRealm.permissionResolver")) {
+        main.put("iniRealm.permissionResolver", "$geodePermissionResolver");
+      }
+
+      SecurityManager securityManager = factory.getInstance();
+      SecurityUtils.setSecurityManager(securityManager);
+    }
+    else if (!com.gemstone.gemfire.internal.lang.StringUtils.isBlank(customAuthenticator)) {
+      Realm realm = new CustomAuthRealm(securityProps);
+      SecurityManager securityManager = new DefaultSecurityManager(realm);
+      SecurityUtils.setSecurityManager(securityManager);
+    }
+    else{
+      SecurityUtils.setSecurityManager(null);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
new file mode 100644
index 0000000..d170756
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.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 com.gemstone.gemfire.internal.security.shiro;
+
+import org.apache.shiro.authz.Permission;
+import org.apache.shiro.authz.permission.PermissionResolver;
+import org.apache.shiro.authz.permission.WildcardPermission;
+
+public class GeodePermissionResolver implements PermissionResolver {
+  @Override public Permission resolvePermission(final String permissionString) {
+    return new WildcardPermission(permissionString, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
index fd2a834..b773b94 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
@@ -28,14 +28,13 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ResourceEvent;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.AlreadyRunningException;
 import com.gemstone.gemfire.management.AsyncEventQueueMXBean;
 import com.gemstone.gemfire.management.CacheServerMXBean;
@@ -54,13 +53,8 @@ import com.gemstone.gemfire.management.RegionMXBean;
 import com.gemstone.gemfire.management.internal.beans.ManagementAdapter;
 import com.gemstone.gemfire.management.membership.MembershipEvent;
 import com.gemstone.gemfire.management.membership.MembershipListener;
-import com.gemstone.gemfire.internal.security.shiro.CustomAuthRealm;
+
 import org.apache.logging.log4j.Logger;
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.config.IniSecurityManagerFactory;
-import org.apache.shiro.mgt.DefaultSecurityManager;
-import org.apache.shiro.mgt.SecurityManager;
-import org.apache.shiro.realm.Realm;
 import org.apache.shiro.util.ThreadContext;
 
 /**
@@ -157,24 +151,7 @@ public final class SystemManagementService extends BaseManagementService {
     this.jmxAdapter = new MBeanJMXAdapter();      
     this.repo = new ManagementResourceRepo();
 
-    DistributionConfig config = system.getConfig();
-
-    // setup shiro for authentication and authorization if it's desired
-    String shiroConfig = config.getShiroInit();
-    String customAuthenticator = config.getSecurityClientAuthenticator();
-    if (!StringUtils.isBlank(shiroConfig)) {
-      IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:"+shiroConfig);
-      SecurityManager securityManager = factory.getInstance();
-      SecurityUtils.setSecurityManager(securityManager);
-    }
-    else if (!StringUtils.isBlank(customAuthenticator)) {
-      Realm realm = new CustomAuthRealm(config.getSecurityProps());
-      SecurityManager securityManager = new DefaultSecurityManager(realm);
-      SecurityUtils.setSecurityManager(securityManager);
-    }
-    else{
-      SecurityUtils.setSecurityManager(null);
-    }
+    GeodeSecurityUtil.initSecurity(system.getConfig().getSecurityProps());
 
     this.notificationHub = new NotificationHub(repo);
     if (system.getConfig().getJmxManager()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
index 2e46104..ab49270 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
@@ -43,7 +43,15 @@ public class ResourceOperationContext extends OperationContext {
     if (operation != null) this.operation = OperationCode.valueOf(operation);
     if (regionName !=null ) this.regionName = regionName;
 
-    setParts(this.resource.name()+":"+this.operation.name()+":"+this.regionName);
+    //for DATA resource, when we construct the lock to guard the operations, there should always be a 3rd part (regionName),
+    // if no regionName is specified, we need to add "NULL" to it.
+    // this means, for general data operations, or operations that we can't put a regionName on yet, like backup diskstore, query data, create regions
+    // it will require DATA:REAT/WRITE:NULL role
+    if(this.resource==Resource.DATA && this.regionName==null){
+      this.regionName = "NULL";
+    }
+
+    setParts(this.resource.name()+":"+this.operation.name()+":"+this.regionName, true);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
index 85a55a7..1c8586f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
@@ -83,7 +83,7 @@ public class CacheServerMBeanShiroJUnitTest {
 
   @Test
   @JMXConnectionConfiguration(user = "dataReader", password = "12345")
-  public void testDataRead() throws Exception{
+  public void ztestDataRead() throws Exception{
     assertThatThrownBy(() -> bean.removeIndex("foo")).hasMessageContaining(TestCommand.dataManage.toString());
     assertThatThrownBy(() -> bean.fetchLoadProbe()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.getActiveCQCount()).hasMessageContaining(TestCommand.clusterRead.toString());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
index 97260d8..9c9b4fc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
@@ -74,10 +74,10 @@ public class DataCommandsSecurityTest {
     assertThatThrownBy(() -> bean.processCommand("import data --region=region2 --file=foo.txt --member=value")).isInstanceOf(GemFireSecurityException.class);
 
     assertThatThrownBy(() -> bean.processCommand("put --key=key1 --value=value1 --region=region2")).isInstanceOf(GemFireSecurityException.class)
-        .hasMessageContaining("[data]:[write]:[region2]");
+        .hasMessageContaining("DATA:WRITE:region2");
 
     assertThatThrownBy(() -> bean.processCommand("get --key=key1 --region=region2")).isInstanceOf(GemFireSecurityException.class)
-        .hasMessageContaining("[data]:[read]:[region2]");
+        .hasMessageContaining("DATA:READ:region2");
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java
index 0bf3cab..52f37e6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java
@@ -17,16 +17,10 @@
 
 package com.gemstone.gemfire.management.internal.security;
 
-import java.util.Properties;
-
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.security.shiro.CustomAuthRealm;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.mgt.DefaultSecurityManager;
-import org.apache.shiro.mgt.SecurityManager;
-import org.apache.shiro.realm.Realm;
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
 
@@ -39,14 +33,10 @@ import org.junit.experimental.categories.Category;
 public class GeodeSecurityUtilCustomRealmJUnitTest extends GeodeSecurityUtilWithIniFileJUnitTest {
   @BeforeClass
   public static void beforeClass() throws Exception{
-    Properties properties = new Properties();
-    properties.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, JSONAuthorization.class.getName() + ".create");
-    properties.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
+    props.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, JSONAuthorization.class.getName() + ".create");
+    props.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
     JSONAuthorization.setUpWithJsonFile("shiro-ini.json");
-
-    Realm realm = new CustomAuthRealm(properties);
-    SecurityManager securityManager = new DefaultSecurityManager(realm);
-    SecurityUtils.setSecurityManager(securityManager);
+    GeodeSecurityUtil.initSecurity(props);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java
index fe80180..63bf447 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java
@@ -19,14 +19,14 @@ package com.gemstone.gemfire.management.internal.security;
 
 import static org.assertj.core.api.Assertions.*;
 
+import java.util.Properties;
+
 import com.gemstone.gemfire.cache.operations.OperationContext;
-import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
+import com.gemstone.gemfire.security.GemFireSecurityException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.config.IniSecurityManagerFactory;
-import org.apache.shiro.mgt.SecurityManager;
 import org.apache.shiro.util.ThreadContext;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -39,12 +39,11 @@ import org.junit.experimental.categories.Category;
  */
 @Category(UnitTest.class)
 public class GeodeSecurityUtilWithIniFileJUnitTest {
+  protected static Properties props = new Properties();
   @BeforeClass
   public static void beforeClass() throws Exception{
-    ThreadContext.remove();
-    IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:shiro.ini");
-    SecurityManager securityManager = factory.getInstance();
-    SecurityUtils.setSecurityManager(securityManager);
+    props.setProperty(DistributionConfig.SECURITY_SHIRO_INIT_NAME, "shiro.ini");
+    GeodeSecurityUtil.initSecurity(props);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
index 8eaaf6a..377ab77 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
@@ -112,7 +112,7 @@ public class GfshCommandsSecurityTest {
 
 
   private void runCommandsWithAndWithout(String permission) throws Exception{
-    List<TestCommand> permitted = TestCommand.getPermittedCommands(new WildcardPermission(permission));
+    List<TestCommand> permitted = TestCommand.getPermittedCommands(new WildcardPermission(permission, true));
     for(TestCommand clusterRead:permitted) {
       LogService.getLogger().info("Processing authorized command: "+clusterRead.getCommand());gfsh.executeCommand(clusterRead.getCommand());
       CommandResult result = (CommandResult) gfsh.getResult();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java
index 9e2e41a..ec89aaa 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java
@@ -61,6 +61,11 @@ public class ResourceOperationContextJUnitTest {
     context = new ResourceOperationContext("DATA", null, null);
     assertEquals(Resource.DATA, context.getResource());
     assertEquals(OperationCode.NULL, context.getOperationCode());
+    assertEquals("NULL", context.getRegionName());
+
+    context = new ResourceOperationContext("CLUSTER", null, null);
+    assertEquals(Resource.CLUSTER, context.getResource());
+    assertEquals(OperationCode.NULL, context.getOperationCode());
     assertEquals(null, context.getRegionName());
 
     context = new ResourceOperationContext(null, "MANAGE", "REGIONA");
@@ -77,12 +82,12 @@ public class ResourceOperationContextJUnitTest {
   @Test
   public void testToString(){
     context = new ResourceOperationContext();
-    assertEquals("[null]:[null]:[null]", context.toString());
+    assertEquals("NULL:NULL", context.toString());
 
     context = new ResourceOperationContext("DATA", "MANAGE");
-    assertEquals("[data]:[manage]:[null]", context.toString());
+    assertEquals("DATA:MANAGE:NULL", context.toString());
 
     context = new ResourceOperationContext("DATA", "MANAGE", "REGIONA");
-    assertEquals("[data]:[manage]:[regiona]", context.toString());
+    assertEquals("DATA:MANAGE:REGIONA", context.toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java
index 7d683f3..f4c2e06 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java
@@ -43,7 +43,7 @@ public class ShiroCacheStartRule extends ExternalResource {
     properties.put(DistributionConfig.JMX_MANAGER_START_NAME, "true");
     properties.put(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxManagerPort));
     properties.put(DistributionConfig.HTTP_SERVICE_PORT_NAME, "0");
-    properties.put(DistributionConfig.SHIRO_INIT_NAME, shiroFile);
+    properties.put(DistributionConfig.SECURITY_SHIRO_INIT_NAME, shiroFile);
 
     cache = new CacheFactory(properties).create();
     cache.addCacheServer().start();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1179c08e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
index 56eeeec..667330c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
@@ -96,14 +96,14 @@ public class TestCommand {
     createTestCommand("destroy region --name=value", dataManage);
 
     //Data Commands
-    createTestCommand("rebalance --include-region=regionA", dataManage);
-    createTestCommand("export data --region=regionA --file=export.txt --member=exportMember", regionARead);
-    createTestCommand("import data --region=regionA --file=import.txt --member=importMember", regionAWrite);
-    createTestCommand("put --key=key1 --value=value1 --region=regionA", regionAWrite);
-    createTestCommand("get --key=key1 --region=regionA", regionARead);
-    createTestCommand("remove --region=regionA", dataManage);
-    createTestCommand("query --query='SELECT * FROM /region1'", dataRead);
-    createTestCommand("locate entry --key=k1 --region=regionA", regionARead);
+    createTestCommand("rebalance --include-region=RegionA", dataManage);
+    createTestCommand("export data --region=RegionA --file=export.txt --member=exportMember", regionARead);
+    createTestCommand("import data --region=RegionA --file=import.txt --member=importMember", regionAWrite);
+    createTestCommand("put --key=key1 --value=value1 --region=RegionA", regionAWrite);
+    createTestCommand("get --key=key1 --region=RegionA", regionARead);
+    createTestCommand("remove --region=RegionA", dataManage);
+    createTestCommand("query --query='SELECT * FROM /RegionA'", dataRead);
+    createTestCommand("locate entry --key=k1 --region=RegionA", regionARead);
 
     // Deploy commands
     //createTestCommand("deploy --jar=group1_functions.jar --group=Group1", dataManage); // TODO: this command will fail in GfshCommandsSecurityTest at interceptor for jar file checking