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 2017/03/16 20:35:54 UTC

[1/2] geode git commit: GEODE-2674: Changing the lucene listener to fetch the value from the region [Forced Update!]

Repository: geode
Updated Branches:
  refs/heads/feature/GEODE-2645 18bee99cc -> 41e14b528 (forced update)


GEODE-2674: Changing the lucene listener to fetch the value from the region

Rather than use the value that is in the queue, use the latest value
from the region to update the lucene index.

This ensures that even if the queue contains spurious events due to
retries or other issues, we put the correct value in the index.

This also potentially saves memory and disk space for the queue, because
the queue does not need to hold the value for the entry.


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

Branch: refs/heads/feature/GEODE-2645
Commit: 1602a2683408ef608554d4c22a8e7c8c61f3e946
Parents: f329f4a
Author: Dan Smith <up...@apache.org>
Authored: Wed Mar 15 13:23:20 2017 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu Mar 16 11:29:18 2017 -0700

----------------------------------------------------------------------
 .../lucene/internal/LuceneEventListener.java    | 23 ++++++------
 .../internal/LuceneEventSubstitutionFilter.java | 37 ++++++++++++++++++++
 .../cache/lucene/internal/LuceneIndexImpl.java  |  1 +
 .../internal/LuceneEventListenerJUnitTest.java  | 24 ++++++-------
 4 files changed, 63 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/1602a268/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventListener.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventListener.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventListener.java
index 2943ce8..0f55533 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventListener.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventListener.java
@@ -20,6 +20,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.geode.cache.EntryDestroyedException;
+import org.apache.geode.cache.Region.Entry;
 import org.apache.geode.internal.cache.wan.parallel.ParallelGatewaySenderQueue;
 import org.apache.logging.log4j.Logger;
 import org.apache.geode.cache.CacheClosedException;
@@ -87,19 +89,20 @@ public class LuceneEventListener implements AsyncEventListener {
 
         IndexRepository repository = repositoryManager.getRepository(region, key, callbackArgument);
 
-        Operation op = event.getOperation();
+        final Entry entry = region.getEntry(key);
+        Object value;
+        try {
+          value = entry == null ? null : entry.getValue();
+        } catch (EntryDestroyedException e) {
+          value = null;
+        }
 
-        if (op.isCreate()) {
-          repository.update(key, event.getDeserializedValue());
-        } else if (op.isUpdate()) {
-          repository.update(key, event.getDeserializedValue());
-        } else if (op.isDestroy()) {
-          repository.delete(key);
-        } else if (op.isInvalidate()) {
-          repository.delete(key);
+        if (value != null) {
+          repository.update(key, value);
         } else {
-          throw new InternalGemFireError("Unhandled operation " + op + " on " + event.getRegion());
+          repository.delete(key);
         }
+
         affectedRepos.add(repository);
       }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/1602a268/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventSubstitutionFilter.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventSubstitutionFilter.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventSubstitutionFilter.java
new file mode 100644
index 0000000..04a465a
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneEventSubstitutionFilter.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.lucene.internal;
+
+import org.apache.geode.cache.EntryEvent;
+import org.apache.geode.cache.wan.GatewayEventSubstitutionFilter;
+import org.apache.geode.internal.cache.Token;
+
+/**
+ * A substitution filter which throws away the value of the entry and replaces it with an empty
+ * string. For the lucene index, we will just fetch the new value from the region, so we don't need
+ * the value.
+ */
+public class LuceneEventSubstitutionFilter implements GatewayEventSubstitutionFilter {
+  @Override
+  public Object getSubstituteValue(final EntryEvent event) {
+    return "";
+  }
+
+  @Override
+  public void close() {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/1602a268/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
index 0c9d220..66dc1f9 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
@@ -180,6 +180,7 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
       return null;
     }
     LuceneEventListener listener = new LuceneEventListener(repositoryManager);
+    factory.setGatewayEventSubstitutionListener(new LuceneEventSubstitutionFilter());
     String aeqId = LuceneServiceImpl.getUniqueIndexName(getName(), regionPath);
     AsyncEventQueue indexQueue = factory.create(aeqId, listener);
     return indexQueue;

http://git-wip-us.apache.org/repos/asf/geode/blob/1602a268/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneEventListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneEventListenerJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneEventListenerJUnitTest.java
index 0320068..79de29a 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneEventListenerJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneEventListenerJUnitTest.java
@@ -27,6 +27,8 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.geode.InternalGemFireError;
+import org.apache.geode.cache.Region.Entry;
+import org.apache.geode.internal.cache.RegionEntry;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -80,18 +82,16 @@ public class LuceneEventListenerJUnitTest {
       Mockito.when(event.getKey()).thenReturn(i);
       Mockito.when(event.getCallbackArgument()).thenReturn(callback);
 
-      switch (i % 3) {
+      switch (i % 4) {
         case 0:
-          Mockito.when(event.getOperation()).thenReturn(Operation.CREATE);
-          Mockito.when(event.getDeserializedValue()).thenReturn(i);
-          break;
         case 1:
-          Mockito.when(event.getOperation()).thenReturn(Operation.UPDATE);
-          Mockito.when(event.getDeserializedValue()).thenReturn(i);
+          final Entry entry = mock(Entry.class);
+          when(entry.getValue()).thenReturn(i);
+          when(region.getEntry(eq(i))).thenReturn(entry);
           break;
         case 2:
-          Mockito.when(event.getOperation()).thenReturn(Operation.DESTROY);
-          Mockito.when(event.getDeserializedValue()).thenThrow(new AssertionError());
+        case 3:
+          // Do nothing, get value will return a destroy
           break;
       }
 
@@ -100,10 +100,10 @@ public class LuceneEventListenerJUnitTest {
 
     listener.processEvents(events);
 
-    verify(repo1, atLeast(numEntries / 6)).delete(any());
-    verify(repo1, atLeast(numEntries / 3)).update(any(), any());
-    verify(repo2, atLeast(numEntries / 6)).delete(any());
-    verify(repo2, atLeast(numEntries / 3)).update(any(), any());
+    verify(repo1, atLeast(numEntries / 4)).delete(any());
+    verify(repo1, atLeast(numEntries / 4)).update(any(), any());
+    verify(repo2, atLeast(numEntries / 4)).delete(any());
+    verify(repo2, atLeast(numEntries / 4)).update(any(), any());
     verify(repo1, times(1)).commit();
     verify(repo2, times(1)).commit();
   }


[2/2] geode git commit: GEODE-2645: rewrite test to fix flakiness and improve readability

Posted by kl...@apache.org.
GEODE-2645: rewrite test to fix flakiness and improve readability


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

Branch: refs/heads/feature/GEODE-2645
Commit: 41e14b5286e9d086673dc806ae126da0f16ffe8e
Parents: 1602a26
Author: Kirk Lund <kl...@apache.org>
Authored: Mon Mar 13 14:51:36 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Thu Mar 16 13:35:39 2017 -0700

----------------------------------------------------------------------
 .../geode/cache30/CacheLogRollDUnitTest.java    | 500 -------------------
 .../logging/CacheLogRollingIntegrationTest.java | 356 +++++++++++++
 2 files changed, 356 insertions(+), 500 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/41e14b52/geode-core/src/test/java/org/apache/geode/cache30/CacheLogRollDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache30/CacheLogRollDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache30/CacheLogRollDUnitTest.java
deleted file mode 100644
index 3a0becf..0000000
--- a/geode-core/src/test/java/org/apache/geode/cache30/CacheLogRollDUnitTest.java
+++ /dev/null
@@ -1,500 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.cache30;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.Properties;
-import java.util.regex.Pattern;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.logging.InternalLogWriter;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.FlakyTest;
-
-/**
- * Test to make sure cache close is working.
- *
- * @since GemFire 6.5
- */
-@Category(DistributedTest.class)
-public class CacheLogRollDUnitTest extends JUnit4CacheTestCase {
-
-  private void logAndRollAndVerify(String baseLogName, DistributedSystem ds, String mainId)
-      throws FileNotFoundException, IOException {
-    String logfile = baseLogName + ".log";
-    String metaLogFile = "meta-" + baseLogName + "-" + mainId + ".log";
-    String rolledLogFile1 = baseLogName + "-" + mainId + "-01.log";
-    String rolledLogFile2 = baseLogName + "-" + mainId + "-02.log";
-    String META_MARKER_1 = "Switching to log " + baseLogName + ".log";
-    String META_MARKER_2 = "Rolling current log to " + baseLogName + "-" + mainId + "-01.log";
-    String META_MARKER_3 = "Rolling current log to " + baseLogName + "-" + mainId + "-02.log";
-
-    String FIRST_CHILD_MARKER = "hey guys im the first child whatsup";
-    String LOG_NONSENSE =
-        "what is your story what are you doing hey whatsup i can't believe it wow";
-    System.out.println("LOGNAME:" + logfile);
-    /*
-     * 1. Lets assert that the logfile exists and that it is a proper normal logfile 2. Asser that
-     * the meta logfile exists and has good stuff in it 3. Let's log a bunch and show that we
-     * rolled, 4. Show that old file has right old stuff in it 5. Show that new file has right new
-     * stuff in it 6. Show that meta has right stuff in it
-     */
-
-    ds.getLogWriter().info(FIRST_CHILD_MARKER);
-
-    File f = new File(logfile);
-    assertTrue("log-file :" + logfile + " should exist", f.exists());
-
-    File fm = new File(metaLogFile);
-    assertTrue("meta-log-file :" + metaLogFile + " should exist", fm.exists());
-
-    File f1 = new File(rolledLogFile1);
-    assertTrue("child-log-file :" + rolledLogFile1 + " should'nt exist", !f1.exists());
-
-    File f2 = new File(rolledLogFile2);
-    assertTrue("child-log-file2 :" + rolledLogFile2 + " should'nt exist yet", !f2.exists());
-
-
-    String metalog = getLogContents(metaLogFile);
-    assertTrue("metalog(" + metaLogFile + ") should have " + META_MARKER_1 + " in it:\n" + metalog,
-        metalog.indexOf(META_MARKER_1) != -1);
-
-    String mainlog = getLogContents(logfile);
-    assertTrue("log(" + logfile + ") should have " + FIRST_CHILD_MARKER + " in it:\n" + mainlog,
-        mainlog.indexOf(FIRST_CHILD_MARKER) != -1);
-
-    int i = 0;
-    while (i < 100000 && !f2.exists()) {
-      i++;
-      ds.getLogWriter().info(LOG_NONSENSE);
-    }
-
-    assertTrue("child-log-file1 :" + rolledLogFile1 + " should exist now", f1.exists());
-    assertTrue("child-log-file2 :" + rolledLogFile2 + " should exist now", f2.exists());
-
-    metalog = getLogContents(metaLogFile);
-
-    assertTrue("log(" + metaLogFile + ") should have " + META_MARKER_2 + " in it:\n" + metalog,
-        metalog.indexOf(META_MARKER_2) != -1);
-    assertTrue("log(" + metaLogFile + ") should have " + META_MARKER_3 + " in it:\n" + metalog,
-        metalog.indexOf(META_MARKER_3) != -1);
-    assertTrue("log(" + metaLogFile + ") should'nt have " + LOG_NONSENSE + " in it:\n" + metalog,
-        metalog.indexOf(LOG_NONSENSE) == -1);
-
-    {
-      String logChild2 = getLogContents(logfile);
-      assertTrue("log(" + logfile + ") should have " + LOG_NONSENSE + " in it:\n" + logChild2,
-          logChild2.indexOf(LOG_NONSENSE) != -1);
-    }
-
-    {
-      String logChild2 = getLogContents(rolledLogFile1);
-      assertTrue(
-          "log(" + rolledLogFile1 + ") should have " + LOG_NONSENSE + " in it:\n" + logChild2,
-          logChild2.indexOf(LOG_NONSENSE) != -1);
-    }
-
-    {
-      String logChild2 = getLogContents(rolledLogFile2);
-      assertTrue(
-          "log(" + rolledLogFile2 + ") should have " + LOG_NONSENSE + " in it:\n" + logChild2,
-          logChild2.indexOf(LOG_NONSENSE) != -1);
-    }
-  }
-
-  private void SecurityLogAndRollAndVerify(String baseLogName, DistributedSystem ds, String mainId)
-      throws FileNotFoundException, IOException {
-    String logfile = baseLogName + ".log";
-    String sec_logfile = "sec_" + logfile;
-    String metaLogFile = "meta-" + baseLogName + "-" + mainId + ".log";
-    String rolledLogFile1 = baseLogName + "-" + mainId + "-01.log";
-    String rolledLogFile2 = baseLogName + "-" + mainId + "-02.log";
-    String rolledSecLogFile1 = "sec_" + baseLogName + "-" + mainId + "-01.log";
-    String rolledSecLogFile2 = "sec_" + baseLogName + "-" + mainId + "-02.log";
-    String META_MARKER_1 = "Switching to log " + baseLogName + ".log";
-    String META_MARKER_2 = "Rolling current log to " + baseLogName + "-" + mainId + "-01.log";
-    String META_MARKER_3 = "Rolling current log to " + baseLogName + "-" + mainId + "-02.log";
-
-    String FIRST_CHILD_MARKER = "hey guys im the first child whatsup";
-    String LOG_NONSENSE =
-        "what is your story what are you doing hey whatsup i can't believe it wow";
-    System.out.println("LOGNAME:" + logfile + ", SECLOGNAME:" + sec_logfile);
-    /*
-     * 1. Lets assert that the logfile exists and that it is a proper normal logfile 2. Asser that
-     * the meta logfile exists and has good stuff in it 3. Let's log a bunch and show that we
-     * rolled, 4. Show that old file has right old stuff in it 5. Show that new file has right new
-     * stuff in it 6. Show that meta has right stuff in it
-     */
-
-    ds.getLogWriter().info(FIRST_CHILD_MARKER);
-    ds.getSecurityLogWriter().info(FIRST_CHILD_MARKER);
-
-    File f = new File(logfile);
-    File sec_f = new File(sec_logfile);
-    assertTrue("log-file :" + logfile + " should exist", f.exists());
-    assertTrue("security-log-file :" + sec_logfile + " should exist", sec_f.exists());
-
-    File fm = new File(metaLogFile);
-    assertTrue("meta-log-file :" + metaLogFile + " should exist", fm.exists());
-
-    File f1 = new File(rolledLogFile1);
-    File sec_f1 = new File(rolledSecLogFile1);
-    assertTrue("child-log-file :" + rolledLogFile1 + " should'nt exist", !f1.exists());
-    assertTrue("security-child-log-file :" + rolledLogFile1 + " should'nt exist", !sec_f1.exists());
-
-    File f2 = new File(rolledLogFile2);
-    File sec_f2 = new File(rolledSecLogFile2);
-    assertTrue("child-log-file2 :" + rolledLogFile2 + " should'nt exist yet", !f2.exists());
-    assertTrue("security-child-log-file2 :" + rolledSecLogFile2 + " should'nt exist yet",
-        !sec_f2.exists());
-
-
-    String metalog = getLogContents(metaLogFile);
-    assertTrue("metalog(" + metaLogFile + ") should have " + META_MARKER_1 + " in it:\n" + metalog,
-        metalog.indexOf(META_MARKER_1) != -1);
-
-    String mainlog = getLogContents(logfile);
-    assertTrue("log(" + logfile + ") should have " + FIRST_CHILD_MARKER + " in it:\n" + mainlog,
-        mainlog.indexOf(FIRST_CHILD_MARKER) != -1);
-    String sec_mainlog = getLogContents(sec_logfile);
-    assertTrue(
-        "log(" + sec_logfile + ") should have " + FIRST_CHILD_MARKER + " in it:\n" + sec_mainlog,
-        sec_mainlog.indexOf(FIRST_CHILD_MARKER) != -1);
-
-    int i = 0;
-    while (i < 100000 && !f2.exists()) {
-      i++;
-      ds.getLogWriter().info(LOG_NONSENSE);
-    }
-
-    int j = 0;
-    while (j < 100000 && !sec_f2.exists()) {
-      j++;
-      ds.getSecurityLogWriter().info(LOG_NONSENSE);
-    }
-
-    assertTrue("child-log-file1 :" + rolledLogFile1 + " should exist now", f1.exists());
-    assertTrue("child-log-file2 :" + rolledLogFile2 + " should exist now", f2.exists());
-    assertTrue("security-child-log-file1 :" + rolledSecLogFile1 + " should exist now",
-        sec_f1.exists());
-    assertTrue("security-child-log-file2 :" + rolledSecLogFile2 + " should exist now",
-        sec_f2.exists());
-
-    metalog = getLogContents(metaLogFile);
-
-    assertTrue("log(" + metaLogFile + ") should have " + META_MARKER_2 + " in it:\n" + metalog,
-        metalog.indexOf(META_MARKER_2) != -1);
-    assertTrue("log(" + metaLogFile + ") should have " + META_MARKER_3 + " in it:\n" + metalog,
-        metalog.indexOf(META_MARKER_3) != -1);
-    assertTrue("log(" + metaLogFile + ") should'nt have " + LOG_NONSENSE + " in it:\n" + metalog,
-        metalog.indexOf(LOG_NONSENSE) == -1);
-
-    {
-      String logChild2 = getLogContents(logfile);
-      assertTrue("log(" + logfile + ") should have " + LOG_NONSENSE + " in it:\n" + logChild2,
-          logChild2.indexOf(LOG_NONSENSE) != -1);
-      String sec_logChild2 = getLogContents(sec_logfile);
-      assertTrue(
-          "log(" + sec_logfile + ") should have " + LOG_NONSENSE + " in it:\n" + sec_logChild2,
-          sec_logChild2.indexOf(LOG_NONSENSE) != -1);
-    }
-
-    {
-      String logChild2 = getLogContents(rolledLogFile1);
-      assertTrue(
-          "log(" + rolledLogFile1 + ") should have " + LOG_NONSENSE + " in it:\n" + logChild2,
-          logChild2.indexOf(LOG_NONSENSE) != -1);
-      String sec_logChild2 = getLogContents(rolledSecLogFile1);
-      assertTrue("log(" + rolledSecLogFile1 + ") should have " + LOG_NONSENSE + " in it:\n"
-          + sec_logChild2, sec_logChild2.indexOf(LOG_NONSENSE) != -1);
-    }
-
-    {
-      String logChild2 = getLogContents(rolledLogFile2);
-      assertTrue(
-          "log(" + rolledLogFile2 + ") should have " + LOG_NONSENSE + " in it:\n" + logChild2,
-          logChild2.indexOf(LOG_NONSENSE) != -1);
-      String sec_logChild2 = getLogContents(rolledSecLogFile2);
-      assertTrue("log(" + rolledSecLogFile2 + ") should have " + LOG_NONSENSE + " in it:\n"
-          + sec_logChild2, sec_logChild2.indexOf(LOG_NONSENSE) != -1);
-    }
-  }
-
-  @Test
-  public void testDiskSpace() throws Exception {
-    Properties props = new Properties();
-    String baseLogName = "diskarito";
-    String logfile = baseLogName + ".log";
-    props.put(LOG_FILE, logfile);
-    props.put(LOG_FILE_SIZE_LIMIT, "1");
-    DistributedSystem ds = this.getSystem(props);
-    props.put(LOG_DISK_SPACE_LIMIT, "200");
-    for (int i = 0; i < 10; i++) {
-      ds = this.getSystem(props);
-      ds.disconnect();
-    }
-
-    /*
-     * This was throwing NPEs until my fix...
-     */
-  }
-
-  @Category(FlakyTest.class) // GEODE-674: possible disk pollution, file size sensitive
-  @Test
-  public void testSimpleStartRestartWithRolling() throws Exception {
-    Properties props = new Properties();
-    String baseLogName = "restarto";
-    String logfile = baseLogName + ".log";
-    props.put(LOG_FILE, logfile);
-    props.put(LOG_FILE_SIZE_LIMIT, "1");
-    props.put(LOG_DISK_SPACE_LIMIT, "200");
-    props.put(LOG_LEVEL, "config");
-    InternalDistributedSystem ids = getSystem(props);
-    assertEquals(InternalLogWriter.INFO_LEVEL,
-        ((InternalLogWriter) ids.getLogWriter()).getLogWriterLevel());
-    ids.disconnect();
-    String mainId;
-    {
-      final Pattern mainIdPattern = Pattern.compile("meta-" + baseLogName + "-\\d\\d.log");
-      File[] metaLogs = new File(".").listFiles(new FilenameFilter() {
-        public boolean accept(File d, String name) {
-          return mainIdPattern.matcher(name).matches();
-        }
-      });
-      assertEquals(1, metaLogs.length);
-      String f = metaLogs[0].getName();
-      int idx = f.lastIndexOf("-");
-      int idx2 = f.lastIndexOf(".");
-      mainId = f.substring(idx + 1, idx2);
-    }
-    String metaName = "meta-" + baseLogName + "-" + mainId + ".log";
-    File fm = new File(metaName);
-    assertTrue("Ok so metalog:" + metaName + " better exist:", fm.exists());
-    for (int i = 1; i < 10; i++) {
-      int mainInt = Integer.parseInt(mainId) + (i);
-      String myid;
-      if (mainInt < 10) {
-        myid = "0" + mainInt;
-      } else {
-        myid = "" + mainInt;
-      }
-      String oldMain;
-      if (mainInt < 11) {
-        oldMain = "0" + (mainInt - 1);
-      } else {
-        oldMain = "" + (mainInt - 1);
-      }
-      String lfold = "meta-" + baseLogName + "-" + (oldMain) + ".log";
-      File fold = new File(lfold);
-      assertTrue(
-          "before we even get going here[" + i + "] mainInt:" + mainInt + " myid:" + myid + " "
-              + lfold + " should exist the metaname was :" + metaName + " and it should match that",
-          fold.exists());
-      String lf = "meta-" + baseLogName + "-" + myid + ".log";
-      String lfl = baseLogName + "-" + (oldMain) + "-01.log";
-      File f1m = new File(lf);
-      File f1l = new File(lfl);
-      assertTrue(!f1m.exists());
-      assertTrue(!f1l.exists());
-      DistributedSystem ds = this.getSystem(props);
-      assertTrue("We are hoping that:" + lf + " exists", f1m.exists());
-      assertTrue("We are hoping that:" + lfl + " exists", f1l.exists());
-      ds.disconnect();
-    }
-  }
-
-  @Category(FlakyTest.class) // GEODE-677: possible disk pollution, file size sensitive
-  @Test
-  public void testStartWithRollingThenRestartWithRolling() throws Exception {
-    Properties props = new Properties();
-    String baseLogName = "biscuits";
-    String logfile = baseLogName + ".log";
-    props.put(LOG_FILE, logfile);
-    props.put(LOG_FILE_SIZE_LIMIT, "1");
-    props.put(LOG_LEVEL, "config");
-    DistributedSystem ds = getSystem(props);
-    InternalDistributedSystem ids = (InternalDistributedSystem) ds;
-    assertEquals(InternalLogWriter.INFO_LEVEL,
-        ((InternalLogWriter) ids.getLogWriter()).getLogWriterLevel());
-
-    // Lets figure out the mainId we start with
-    String mainId;
-    {
-      final Pattern mainIdPattern = Pattern.compile("meta-" + baseLogName + "-\\d\\d\\d*.log");
-      File[] metaLogs = new File(".").listFiles(new FilenameFilter() {
-        public boolean accept(File d, String name) {
-          return mainIdPattern.matcher(name).matches();
-        }
-      });
-      assertEquals(1, metaLogs.length);
-      String f = metaLogs[0].getName();
-      int idx = f.lastIndexOf("-");
-      int idx2 = f.lastIndexOf(".");
-      mainId = f.substring(idx + 1, idx2);
-    }
-    logAndRollAndVerify(baseLogName, ds, mainId);
-    /*
-     * Ok now we have rolled and yada yada. Let's disconnect and reconnect with same name!
-     */
-    int dsId = System.identityHashCode(ds);
-    props.put(LOG_DISK_SPACE_LIMIT, "200");
-
-    File f1m = new File(logfile);
-    assertTrue(f1m.exists());
-    File f1c1 = new File(baseLogName + "-" + mainId + "-01.log");
-    assertTrue(f1c1.exists());
-    File f1c2 = new File(baseLogName + "-" + mainId + "-02.log");
-    assertTrue(f1c2.exists());
-
-    File f1c3 = new File(baseLogName + "-" + mainId + "-03.log");
-    assertTrue(!f1c3.exists());
-
-    String nextMainId;
-    {
-      int mId = Integer.parseInt(mainId);
-      mId++;
-      StringBuffer sb = new StringBuffer();
-      if (mId < 10) {
-        sb.append('0');
-      }
-      sb.append(mId);
-      nextMainId = sb.toString();
-    }
-    File f2c1 = new File(baseLogName + "-" + nextMainId + "-01.log");
-    assertTrue(!f2c1.exists());
-
-
-    /*
-     * Lets just make sure all the proper files exist
-     */
-    ds = this.getSystem(props);
-    int dsId2 = System.identityHashCode(ds);
-    assertTrue("This should be a new ds!", dsId != dsId2);
-    /*
-     * creating the new system should have rolled the old rolling log
-     * (biscuits.log->biscuits-02-01.log)
-     * 
-     */
-    // The following assert does not work on Windows because
-    // we can't rename the last biscuits.log because it is still open
-    // The DistributedSystem disconnect is not closing the logger enough
-    // so that it can be renamed.
-    // Reenable this assertion once this issue (bug 42176) is fixed.
-    assertTrue(f1c3.exists());
-  }
-
-  @Category(FlakyTest.class) // GEODE-676: possible disk pollution, file size sensitive
-  @Test
-  public void testLogFileLayoutAndRolling() throws Exception {
-    String baseLogName = "tacos";
-    Properties props = new Properties();
-
-    String logfile = baseLogName + ".log";
-    props.put(LOG_FILE, logfile);
-    props.put(LOG_FILE_SIZE_LIMIT, "1");
-    props.put(LOG_LEVEL, "config");
-
-    DistributedSystem ds = getSystem(props);
-    InternalDistributedSystem ids = (InternalDistributedSystem) ds;
-    assertEquals(InternalLogWriter.INFO_LEVEL,
-        ((InternalLogWriter) ids.getLogWriter()).getLogWriterLevel());
-
-    // Lets figure out the mainId we start with
-    String mainId;
-    {
-      final Pattern mainIdPattern = Pattern.compile("meta-" + baseLogName + "-\\d+.log");
-      File[] metaLogs = new File(".").listFiles(new FilenameFilter() {
-        public boolean accept(File d, String name) {
-          return mainIdPattern.matcher(name).matches();
-        }
-      });
-      assertEquals(1, metaLogs.length);
-      String f = metaLogs[0].getName();
-      int idx = f.lastIndexOf("-");
-      int idx2 = f.lastIndexOf(".");
-      mainId = f.substring(idx + 1, idx2);
-    }
-    ds.getProperties();
-    logAndRollAndVerify(baseLogName, ds, mainId);
-  }
-
-  @Category(FlakyTest.class) // GEODE-675: possible disk pollution, file size sensitive
-  @Test
-  public void testSecurityLogFileLayoutAndRolling() throws Exception {
-    String baseLogName = "securitytacos";
-    Properties props = new Properties();
-
-    String logfile = baseLogName + ".log";
-    String sec_logfile = "sec_" + baseLogName + ".log";
-    props.put(LOG_FILE, logfile);
-    props.put(LOG_FILE_SIZE_LIMIT, "1");
-    props.put(LOG_LEVEL, "config");
-    props.put(SECURITY_LOG_FILE, sec_logfile);
-    props.put(SECURITY_LOG_LEVEL, "config");
-
-    DistributedSystem ds = getSystem(props);
-    InternalDistributedSystem ids = (InternalDistributedSystem) ds;
-    assertEquals(InternalLogWriter.INFO_LEVEL,
-        ((InternalLogWriter) ids.getLogWriter()).getLogWriterLevel());
-    assertEquals(InternalLogWriter.INFO_LEVEL,
-        ((InternalLogWriter) ids.getSecurityLogWriter()).getLogWriterLevel());
-
-    // Lets figure out the mainId we start with
-    String mainId;
-    {
-      final Pattern mainIdPattern = Pattern.compile("meta-" + baseLogName + "-\\d+.log");
-      File[] metaLogs = new File(".").listFiles(new FilenameFilter() {
-        public boolean accept(File d, String name) {
-          return mainIdPattern.matcher(name).matches();
-        }
-      });
-      assertEquals(1, metaLogs.length);
-      String f = metaLogs[0].getName();
-      int idx = f.lastIndexOf("-");
-      int idx2 = f.lastIndexOf(".");
-      mainId = f.substring(idx + 1, idx2);
-    }
-    ds.getProperties();
-    SecurityLogAndRollAndVerify(baseLogName, ds, mainId);
-  }
-
-  String getLogContents(String logfile) throws FileNotFoundException, IOException {
-    File f = new File(logfile);
-    BufferedReader reader = new BufferedReader(new FileReader(f));
-    StringBuffer fileData = new StringBuffer();
-    int numRead = 0;
-    char[] buf = new char[1024];
-    while ((numRead = reader.read(buf)) != -1) {
-      String readData = String.valueOf(buf, 0, numRead);
-      fileData.append(readData);
-      buf = new char[1024];
-    }
-    return fileData.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/41e14b52/geode-core/src/test/java/org/apache/geode/internal/logging/CacheLogRollingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/logging/CacheLogRollingIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/internal/logging/CacheLogRollingIntegrationTest.java
new file mode 100644
index 0000000..2444f98
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/logging/CacheLogRollingIntegrationTest.java
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.logging;
+
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+import java.util.regex.Pattern;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.LogWriter;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for log rolling with cache lifecycle.
+ *
+ * @since GemFire 6.5
+ */
+@Category(IntegrationTest.class)
+public class CacheLogRollingIntegrationTest {
+
+  private static final int MAX_LOG_STATEMENTS = 100000;
+  private static final String SECURITY_PREFIX = "security_";
+
+  private String baseName;
+  private File dir;
+  private File logFile;
+  private File securityLogFile;
+  private Pattern mainIdPattern;
+  private DistributedSystem system;
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void before() throws Exception {
+    this.baseName = this.testName.getMethodName();
+    this.dir = this.temporaryFolder.getRoot();
+    this.logFile = new File(this.dir, logFileName());
+    this.securityLogFile = new File(this.dir, securityLogFileName());
+    this.mainIdPattern = Pattern.compile("meta-" + this.baseName + "-\\d\\d.log");
+  }
+
+  @After
+  public void after() throws Exception {
+    if (this.system != null) {
+      this.system.disconnect();
+    }
+  }
+
+  @Test
+  public void testSimpleStartRestartWithRolling() throws Exception {
+    Properties config = createConfig();
+    config.put(LOG_FILE, this.logFile.getAbsolutePath());
+    config.put(LOG_FILE_SIZE_LIMIT, "1");
+    config.put(LOG_DISK_SPACE_LIMIT, "200");
+
+    this.system = DistributedSystem.connect(config);
+    this.system.disconnect();
+
+    for (int mainInt = 2; mainInt <= 4; mainInt++) {
+      assertThat(metaFile(mainInt - 1)).exists();
+
+      File newMetaFile = metaFile(mainInt);
+      File newRolledLogFile = childFile(mainInt - 1, 1);
+
+      assertThat(newMetaFile).doesNotExist();
+      assertThat(newRolledLogFile).doesNotExist();
+
+      this.system = DistributedSystem.connect(config);
+
+      assertThat(newMetaFile).exists();
+      assertThat(newRolledLogFile).exists();
+
+      this.system.disconnect();
+    }
+  }
+
+  @Test
+  public void testStartWithRollingThenRestartWithRolling() throws Exception {
+    Properties config = createConfig();
+    config.put(LOG_FILE, this.logFile.getAbsolutePath());
+    config.put(LOG_FILE_SIZE_LIMIT, "1");
+
+    this.system = DistributedSystem.connect(config);
+
+    logAndRollAndVerify(1);
+
+    DistributedSystem firstSystem = this.system;
+
+    assertThat(this.logFile).exists();
+    assertThat(childFile(1, 1)).exists();
+    assertThat(childFile(1, 2)).exists();
+    assertThat(childFile(1, 3)).doesNotExist();
+    assertThat(childFile(2, 1)).doesNotExist();
+
+    this.system.disconnect();
+
+    config.put(LOG_DISK_SPACE_LIMIT, "200");
+    this.system = DistributedSystem.connect(config);
+
+    assertThat(this.system).isNotSameAs(firstSystem);
+    assertThat(childFile(1, 3)).exists();
+  }
+
+  @Test
+  public void testLogFileLayoutAndRolling() throws Exception {
+    Properties config = createConfig();
+    config.put(LOG_FILE, this.logFile.getAbsolutePath());
+    config.put(LOG_FILE_SIZE_LIMIT, "1");
+
+    this.system = DistributedSystem.connect(config);
+
+    logAndRollAndVerify(1);
+  }
+
+  @Test
+  public void testSecurityLogFileLayoutAndRolling() throws Exception {
+    Properties config = createConfig();
+    config.put(LOG_FILE, this.logFile.getAbsolutePath());
+    config.put(LOG_FILE_SIZE_LIMIT, "1");
+    config.put(SECURITY_LOG_FILE, this.securityLogFile.getAbsolutePath());
+
+    this.system = DistributedSystem.connect(config);
+
+    securityLogAndRollAndVerify(1);
+  }
+
+  @Test
+  public void with_logFileSizeLimit_should_createMetaLogFile() throws Exception {
+    Properties config = createConfig();
+    config.put(LOG_FILE, this.logFile.getAbsolutePath());
+    config.put(LOG_FILE_SIZE_LIMIT, "1");
+
+    this.system = DistributedSystem.connect(config);
+
+    File[] metaLogsMatched =
+        this.dir.listFiles((dir, name) -> mainIdPattern.matcher(name).matches());
+    assertThat(metaLogsMatched).hasSize(1);
+
+    File metaLogFile = metaFile(1);
+    assertThat(metaLogFile).exists();
+  }
+
+  @Test
+  public void without_logFileSizeLimit_shouldNot_createMetaLogFile() throws Exception {
+    Properties config = createConfig();
+    config.put(LOG_FILE, this.logFile.getAbsolutePath());
+
+    this.system = DistributedSystem.connect(config);
+
+    File[] metaLogsMatched =
+        this.dir.listFiles((dir, name) -> mainIdPattern.matcher(name).matches());
+    assertThat(metaLogsMatched).hasSize(0);
+
+    File metaLogFile = metaFile(12);
+    assertThat(metaLogFile).doesNotExist();
+  }
+
+  private Properties createConfig() {
+    Properties config = new Properties();
+    config.setProperty(LOCATORS, "");
+    config.setProperty(MCAST_PORT, "0");
+    return config;
+  }
+
+  private String readContents(final File file) throws IOException {
+    assertThat(file).exists();
+
+    BufferedReader reader = new BufferedReader(new FileReader(file));
+    StringBuffer buffer = new StringBuffer();
+    int numRead;
+    char[] chars = new char[1024];
+
+    while ((numRead = reader.read(chars)) != -1) {
+      String readData = String.valueOf(chars, 0, numRead);
+      buffer.append(readData);
+      chars = new char[1024];
+    }
+
+    return buffer.toString();
+  }
+
+  /**
+   * 1. Lets assert that the logfile exists and that it is a proper normal logfile<br>
+   * 2. Assert that the meta logfile exists and has good stuff in it<br>
+   * 3. Let's log a bunch and show that we rolled<br>
+   * 4. Show that old file has right old stuff in it<br>
+   * 5. Show that new file has right new stuff in it<br>
+   * 6. Show that meta has right stuff in it<br>
+   */
+  private void logAndRollAndVerify(final int mainId) throws IOException {
+    File metaLogFile = metaFile(mainId);
+    File childLogFile01 = childFile(mainId, 1);
+    File childLogFile02 = childFile(mainId, 2);
+
+    String switchingToLog = "Switching to log " + this.logFile;
+    String rollingCurrentLogTo01 = "Rolling current log to " + childLogFile01;
+    String rollingCurrentLogTo02 = "Rolling current log to " + childLogFile02;
+
+    String messageInChild = "hey guys im the first child";
+    String messagePrefix = "hey whatsup i can't believe it wow ";
+
+    this.system.getLogWriter().info(messageInChild);
+
+    assertThat(this.logFile).exists();
+    assertThat(metaLogFile).exists();
+    assertThat(childLogFile01).doesNotExist();
+    assertThat(childLogFile02).doesNotExist();
+    assertThat(readContents(metaLogFile)).contains(switchingToLog);
+    assertThat(readContents(this.logFile)).contains(messageInChild);
+
+    logUntilFileExists(this.system.getLogWriter(), messagePrefix, childLogFile02);
+
+    assertThat(childLogFile01).exists();
+    assertThat(childLogFile02).exists();
+
+    String metaLogContents = readContents(metaLogFile);
+    assertThat(metaLogContents).contains(rollingCurrentLogTo01);
+    assertThat(metaLogContents).contains(rollingCurrentLogTo02);
+    assertThat(metaLogContents).doesNotContain(messagePrefix);
+
+    assertThat(readContents(this.logFile)).contains(messagePrefix);
+    assertThat(readContents(childLogFile01)).contains(messagePrefix);
+    assertThat(readContents(childLogFile02)).contains(messagePrefix);
+  }
+
+  /**
+   * 1. Lets assert that the logfile exists and that it is a proper normal logfile<br>
+   * 2. Assert that the meta logfile exists and has good stuff in it<br>
+   * 3. Let's log a bunch and show that we rolled<br>
+   * 4. Show that old file has right old stuff in it<br>
+   * 5. Show that new file has right new stuff in it<br>
+   * 6. Show that meta has right stuff in it<br>
+   */
+  private void securityLogAndRollAndVerify(final int mainId) throws IOException {
+    File metaLogFile = metaFile(mainId);
+    File childLogFile01 = childFile(mainId, 1);
+    File childLogFile02 = childFile(mainId, 2);
+    File childSecurityLogFile01 = childSecurityFile(mainId, 1);
+    File childSecurityLogFile02 = childSecurityFile(mainId, 2);
+
+    String switchingToLog = "Switching to log " + this.logFile;
+    String rollingCurrentLogTo01 = "Rolling current log to " + childLogFile01;
+    String rollingCurrentLogTo02 = "Rolling current log to " + childLogFile02;
+
+    String messageInChild = "hey guys im the first child";
+    String messageInSecurityChild = "hey guys im the first security child";
+    String messagePrefix = "hey whatsup i can't believe it wow ";
+
+    this.system.getLogWriter().info(messageInChild);
+    this.system.getSecurityLogWriter().info(messageInSecurityChild);
+
+    assertThat(readContents(this.logFile)).contains(messageInChild)
+        .doesNotContain(messageInSecurityChild);
+    assertThat(readContents(this.securityLogFile)).contains(messageInSecurityChild)
+        .doesNotContain(messageInChild);
+
+    assertThat(readContents(metaLogFile)).contains(switchingToLog);
+
+    assertThat(childLogFile01).doesNotExist();
+    assertThat(childSecurityLogFile01).doesNotExist();
+    assertThat(childLogFile02).doesNotExist();
+    assertThat(childSecurityLogFile02).doesNotExist();
+
+    logUntilFileExists(this.system.getLogWriter(), messagePrefix, childLogFile02);
+    logUntilFileExists(this.system.getSecurityLogWriter(), messagePrefix, childSecurityLogFile02);
+
+    assertThat(readContents(this.logFile)).contains(messagePrefix);
+    assertThat(readContents(this.securityLogFile)).contains(messagePrefix);
+
+    String metaLogContents = readContents(metaLogFile);
+    assertThat(metaLogContents).contains(rollingCurrentLogTo01);
+    assertThat(metaLogContents).contains(rollingCurrentLogTo02);
+    assertThat(metaLogContents).doesNotContain(messagePrefix);
+
+    assertThat(readContents(childLogFile01)).contains(messagePrefix);
+    assertThat(readContents(childSecurityLogFile01)).contains(messagePrefix);
+    assertThat(readContents(childLogFile02)).contains(messagePrefix);
+    assertThat(readContents(childSecurityLogFile02)).contains(messagePrefix);
+  }
+
+  private void logUntilFileExists(final LogWriter logWriter, final String message,
+      final File logFile) {
+    for (int i = 0; i < MAX_LOG_STATEMENTS && !logFile.exists(); i++) {
+      logWriter.info(message + "line-" + i);
+    }
+    assertThat(logFile).exists();
+  }
+
+  private String formatId(final int id) {
+    return String.format("%02d", id);
+  }
+
+  private String logFileName() {
+    return this.baseName + ".log";
+  }
+
+  private String securityLogFileName() {
+    return SECURITY_PREFIX + this.baseName + ".log";
+  }
+
+  private String metaFileName(int mainId) {
+    return "meta-" + this.baseName + "-" + formatId(mainId) + ".log";
+  }
+
+  private File metaFile(int mainId) {
+    return new File(this.dir, metaFileName(mainId));
+  }
+
+  private String childFileName(int mainId, int childId) {
+    return this.baseName + "-" + formatId(mainId) + "-" + formatId(childId) + ".log";
+  }
+
+  private File childFile(int mainId, int childId) {
+    return new File(this.dir, childFileName(mainId, childId));
+  }
+
+  private String childSecurityFileName(int mainId, int childId) {
+    return SECURITY_PREFIX + this.baseName + "-" + formatId(mainId) + "-" + formatId(childId)
+        + ".log";
+  }
+
+  private File childSecurityFile(int mainId, int childId) {
+    return new File(this.dir, childSecurityFileName(mainId, childId));
+  }
+
+}