You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2019/04/13 00:18:13 UTC

[asterixdb] branch master updated (393215e -> 019f6fe)

This is an automated email from the ASF dual-hosted git repository.

dlych pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git.


    from 393215e  [ASTERIXDB-2539][COMP] Exclude LET variables from "Single Variable Resolution" rule
     new c3ea16c  [NO ISSUE][*DB][STO] indicate null/missing value fix in index checkpoint
     new f5ccf50  [NO ISSUE][*DB][CLUS] Add failed nodes API to ClusterStateManager
     new d8229a8  [NO ISSUE][HYR] Use System.nanotime for StopWatch
     new d06b007  [NO ISSUE][*DB][CLUS] Add API to return all known nodes atomically
     new fab68d2  [NO ISSUE][*DB] Warn (not fail) on running *DB processes on sample start
     new 019f6fe  Merge commit 'fab68d291e' from 'stabilization-f69489'

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../dataflow/CheckpointInSecondaryIndexTest.java   | 460 ++++++++++-----------
 .../common/cluster/IClusterStateManager.java       |  10 +
 .../asterix/common/storage/IndexCheckpoint.java    |  11 +-
 .../asterix/runtime/utils/ClusterStateManager.java |  12 +
 .../main/opt/local/bin/start-sample-cluster.bat    |  21 +-
 .../src/main/opt/local/bin/start-sample-cluster.sh |  24 +-
 .../src/main/opt/local/bin/stop-sample-cluster.sh  |   2 +-
 .../org/apache/hyracks/api/util/StopWatch.java     |  19 +-
 8 files changed, 274 insertions(+), 285 deletions(-)


[asterixdb] 04/06: [NO ISSUE][*DB][CLUS] Add API to return all known nodes atomically

Posted by dl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dlych pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit d06b0070d5a29443a6688339370bb364f5e75a4a
Author: Michael Blow <mi...@couchbase.com>
AuthorDate: Fri Apr 12 12:24:06 2019 -0400

    [NO ISSUE][*DB][CLUS] Add API to return all known nodes atomically
    
    Change-Id: Ie6bb64e331666ef44b07034d430ba2ee563bba82
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3340
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
---
 .../org/apache/asterix/common/cluster/IClusterStateManager.java    | 5 +++++
 .../java/org/apache/asterix/runtime/utils/ClusterStateManager.java | 7 +++++++
 2 files changed, 12 insertions(+)

diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
index d0d039d..0e62851 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
@@ -144,6 +144,11 @@ public interface IClusterStateManager {
     Set<String> getFailedNodes();
 
     /**
+     * @return the set of cluster nodes
+     */
+    Set<String> getNodes();
+
+    /**
      * Returns the IO devices configured for a Node Controller
      *
      * @param nodeId
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index 720b089..16a479e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -256,6 +256,13 @@ public class ClusterStateManager implements IClusterStateManager {
     }
 
     @Override
+    public synchronized Set<String> getNodes() {
+        Set<String> nodes = new HashSet<>(participantNodes);
+        nodes.addAll(failedNodes);
+        return nodes;
+    }
+
+    @Override
     public synchronized Set<String> getParticipantNodes(boolean excludePendingRemoval) {
         final Set<String> participantNodesCopy = getParticipantNodes();
         if (excludePendingRemoval) {


[asterixdb] 06/06: Merge commit 'fab68d291e' from 'stabilization-f69489'

Posted by dl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dlych pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit 019f6fea1e81915375e3f076ae82729c11505407
Merge: 393215e fab68d2
Author: Michael Blow <mb...@apache.org>
AuthorDate: Fri Apr 12 18:05:32 2019 -0400

    Merge commit 'fab68d291e' from 'stabilization-f69489'
    
    Change-Id: Idcabca9b2b126f891a12e60e4b427fdc4828f07f

 .../dataflow/CheckpointInSecondaryIndexTest.java   | 460 ++++++++++-----------
 .../common/cluster/IClusterStateManager.java       |  10 +
 .../asterix/common/storage/IndexCheckpoint.java    |  11 +-
 .../asterix/runtime/utils/ClusterStateManager.java |  12 +
 .../main/opt/local/bin/start-sample-cluster.bat    |  21 +-
 .../src/main/opt/local/bin/start-sample-cluster.sh |  24 +-
 .../src/main/opt/local/bin/stop-sample-cluster.sh  |   2 +-
 .../org/apache/hyracks/api/util/StopWatch.java     |  19 +-
 8 files changed, 274 insertions(+), 285 deletions(-)

diff --cc asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
index 8f40b3a,bb9c10e..3da845d
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
@@@ -79,7 -80,11 +80,9 @@@ import org.junit.AfterClass
  import org.junit.Assert;
  import org.junit.Before;
  import org.junit.BeforeClass;
+ import org.junit.Rule;
  import org.junit.Test;
+ import org.junit.rules.TestRule;
 -import org.junit.rules.TestWatcher;
 -import org.junit.runner.Description;
  import org.junit.runners.Parameterized;
  
  public class CheckpointInSecondaryIndexTest {


[asterixdb] 05/06: [NO ISSUE][*DB] Warn (not fail) on running *DB processes on sample start

Posted by dl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dlych pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit fab68d291e12973a4c28f9150067f8fc00d56475
Author: Michael Blow <mi...@couchbase.com>
AuthorDate: Fri Apr 12 12:48:53 2019 -0400

    [NO ISSUE][*DB] Warn (not fail) on running *DB processes on sample start
    
    also, fix printing of running processes post sample stop
    
    Change-Id: I84037ad5f64536bafcb7c9500b9a7668e6849156
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3341
    Reviewed-by: Ian Maxon <im...@uci.edu>
    Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
---
 .../main/opt/local/bin/start-sample-cluster.bat    | 21 +++++--------------
 .../src/main/opt/local/bin/start-sample-cluster.sh | 24 +++++-----------------
 .../src/main/opt/local/bin/stop-sample-cluster.sh  |  2 +-
 3 files changed, 11 insertions(+), 36 deletions(-)

diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.bat b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.bat
index 60f5cb5..652abff 100644
--- a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.bat
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.bat
@@ -32,9 +32,9 @@ exit /B 0
 if "%1" == "" goto postopts
 
 if "%1" == "-f" (
-  set force=1
+  rem ignored, this is always the case
 ) else if "%1" == "-force" (
-  set force=1
+  rem ignored, this is always the case
 ) else if "%1" == "-usage" (
   goto :usage
 ) else if "%1" == "-help" (
@@ -80,23 +80,12 @@ wmic process where ^
   "name='java.exe' and CommandLine like '%%org.codehaus.mojo.appassembler.booter.AppassemblerBooter%%' and (CommandLine like '%%app.name=\"%%[cn]c\"%%' or CommandLine like '%%app.name=\"%%ncservice\"%%')" ^
   GET processid > %tempfile% 2> nul
 
-set severity=ERROR
-if "%force%" == "1" set severity=WARNING
-
 for /F "skip=1" %%P in ('type %tempfile%') DO set found=1
 
 if "%found%" == "1" (
-  if "%force%" == "1" (
-    echo %severity%: ${PRODUCT} processes are already running; -f[orce] specified, ignoring
-    del %tempfile%
- ) else (
-    echo %severity%: ${PRODUCT} processes are already running; aborting"
-    echo.
-    echo Re-run with -f to ignore, or run stop-sample-cluster.bat -f to forcibly terminate all running ${PRODUCT} processes:
-    for /F "skip=1" %%P in ('type %tempfile%') DO @echo     - %%P
-    del %tempfile%
-    exit /B 1
-  )
+  echo WARNING: ${PRODUCT} processes are already running:
+  for /F "skip=1" %%P in ('type %tempfile%') DO @echo     - %%P
+  del %tempfile%
 )
 
 goto :post_timestamp
diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
index 34d156c..9268bed 100755
--- a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
@@ -20,14 +20,13 @@
 
 function usage() {
   echo
-  echo Usage: $(basename $0) [-f[orce]]
+  echo Usage: $(basename $0)
   echo
-  echo "  -f[orce]  : Forces a start attempt when ${PRODUCT} processes are found to be running"
 }
 
 while [ -n "$1" ]; do
   case $1 in
-    -f|-force) force=1;;
+    -f|-force);; # ignored, this is always the case
     -help|--help|-usage|--usage) usage; exit 0;;
     *) echo "ERROR: unknown argument '$1'"; usage; exit 1;;
   esac
@@ -115,22 +114,9 @@ mkdir -p "$LOGSDIR"
     && echo "ERROR: sample cluster address (localhost:${LISTEN_PORT}) already in use" && exit 1
 
 if ps -ef | grep 'java.*org\.apache\.hyracks\.control\.[cn]c\.\([CN]CDriver\|service\.NCService\)' > /tmp/$$_pids; then
-  if [ $force ]; then
-    severity=WARNING
-  else
-    severity=ERROR
-  fi
-  echo -n "${severity}: ${PRODUCT} processes are already running; "
-  if [ $force ]; then
-    echo "-f[orce] specified, ignoring"
-  else
-    echo "aborting"
-    echo
-    echo "Re-run with -f to ignore, or run stop-sample-cluster.sh -f to forcibly terminate all running ${PRODUCT} processes:"
-    cat /tmp/$$_pids |  sed 's/^ *[0-9]* \([0-9]*\).*org\.apache\.hyracks\.control\.[cn]c[^ ]*\.\([^ ]*\) .*/\1 - \2/'
-    rm /tmp/$$_pids
-    exit 1
-  fi
+  echo "WARNING: ${PRODUCT} processes are already running:"
+  cat /tmp/$$_pids |  sed 's/^ *[0-9]* \([0-9]*\).*org\.apache\.hyracks\.control\.[cn]c[^ ]*\.\([^ ]*\) .*/\1 - \2/'
+  rm /tmp/$$_pids
 fi
 
 rm /tmp/$$_pids
diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh b/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
index 80647d4..97473ac 100755
--- a/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
@@ -146,7 +146,7 @@ if ps -ef | grep 'java.*org\.apache\.hyracks\.control\.[cn]c\.\([CN]CDriver\|ser
     done
   else
     echo "re-run with -f|-force to forcibly terminate all ${PRODUCT} processes:"
-    cat /tmp/pids |  sed 's/^ *[0-9]* \([0-9]*\).*org\.apache\.hyracks\.control\.[cn]c[^ ]*\.\([^ ]*\) .*/\1 - \2/'
+    cat /tmp/$$_pids |  sed 's/^ *[0-9]* \([0-9]*\).*org\.apache\.hyracks\.control\.[cn]c[^ ]*\.\([^ ]*\) .*/\1 - \2/'
   fi
 fi
 rm /tmp/$$_pids


[asterixdb] 02/06: [NO ISSUE][*DB][CLUS] Add failed nodes API to ClusterStateManager

Posted by dl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dlych pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit f5ccf50f4695a8557983abcfa8b1ad989ed8822a
Author: Michael Blow <mi...@couchbase.com>
AuthorDate: Thu Apr 11 10:53:13 2019 -0400

    [NO ISSUE][*DB][CLUS] Add failed nodes API to ClusterStateManager
    
    Change-Id: I1b56b29c38f0bbfe54b30ee5781770e90404c528
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3335
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Till Westmann <ti...@apache.org>
---
 .../java/org/apache/asterix/common/cluster/IClusterStateManager.java | 5 +++++
 .../java/org/apache/asterix/runtime/utils/ClusterStateManager.java   | 5 +++++
 2 files changed, 10 insertions(+)

diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
index 5e99eec..d0d039d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
@@ -139,6 +139,11 @@ public interface IClusterStateManager {
     Set<String> getParticipantNodes();
 
     /**
+     * @return the set of currently failed nodes
+     */
+    Set<String> getFailedNodes();
+
+    /**
      * Returns the IO devices configured for a Node Controller
      *
      * @param nodeId
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index 8d3187b..720b089 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -251,6 +251,11 @@ public class ClusterStateManager implements IClusterStateManager {
     }
 
     @Override
+    public synchronized Set<String> getFailedNodes() {
+        return new HashSet<>(failedNodes);
+    }
+
+    @Override
     public synchronized Set<String> getParticipantNodes(boolean excludePendingRemoval) {
         final Set<String> participantNodesCopy = getParticipantNodes();
         if (excludePendingRemoval) {


[asterixdb] 01/06: [NO ISSUE][*DB][STO] indicate null/missing value fix in index checkpoint

Posted by dl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dlych pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit c3ea16c8278111d10dee4d0a01d0a2cd08a81af0
Author: Michael Blow <mi...@couchbase.com>
AuthorDate: Thu Apr 11 11:18:58 2019 -0400

    [NO ISSUE][*DB][STO] indicate null/missing value fix in index checkpoint
    
    - user model changes: no
    - storage format changes: yes (backwards-compatible)
    - interface changes: no
    
    Details:
    
    - add marker on inintial IndexCheckpoint, propagate on subsequent checkpoints
    - ignore extra JSON fields when deserializing IndexCheckpoint
    
    Change-Id: Id93ab9d16887b37cf6c0d011950e7c57f1a1d646
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3324
    Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
---
 .../dataflow/CheckpointInSecondaryIndexTest.java   | 462 ++++++++++-----------
 .../asterix/common/storage/IndexCheckpoint.java    |  11 +-
 2 files changed, 236 insertions(+), 237 deletions(-)

diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
index a0ed26e..bb9c10e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
@@ -54,6 +54,7 @@ import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
+import org.apache.asterix.test.base.TestMethodTracer;
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -79,7 +80,11 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.rules.TestWatcher;
+import org.junit.runner.Description;
 import org.junit.runners.Parameterized;
 
 public class CheckpointInSecondaryIndexTest {
@@ -101,7 +106,7 @@ public class CheckpointInSecondaryIndexTest {
     private static final boolean[] UNIQUE_META_FIELDS = null;
     private static final int[] KEY_INDEXES = { 0 };
     private static final int[] KEY_INDICATORS = { Index.RECORD_INDICATOR };
-    private static final List<Integer> KEY_INDICATORS_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
+    private static final List<Integer> KEY_INDICATORS_LIST = Collections.singletonList(Index.RECORD_INDICATOR);
     private static final int RECORDS_PER_COMPONENT = 500;
     private static final int DATASET_ID = 101;
     private static final String DATAVERSE_NAME = "TestDV";
@@ -113,9 +118,9 @@ public class CheckpointInSecondaryIndexTest {
     private static final IFieldValueGenerator[] SECONDARY_INDEX_VALUE_GENERATOR =
             { new AInt64ValueGenerator(), new AInt32ValueGenerator() };
     private static final List<List<String>> INDEX_FIELD_NAMES =
-            Arrays.asList(Arrays.asList(RECORD_TYPE.getFieldNames()[1]));
-    private static final List<Integer> INDEX_FIELD_INDICATORS = Arrays.asList(Index.RECORD_INDICATOR);
-    private static final List<IAType> INDEX_FIELD_TYPES = Arrays.asList(BuiltinType.AINT64);
+            Collections.singletonList(Collections.singletonList(RECORD_TYPE.getFieldNames()[1]));
+    private static final List<Integer> INDEX_FIELD_INDICATORS = Collections.singletonList(Index.RECORD_INDICATOR);
+    private static final List<IAType> INDEX_FIELD_TYPES = Collections.singletonList(BuiltinType.AINT64);
     private static final StorageComponentProvider storageManager = new StorageComponentProvider();
     private static TestNodeController nc;
     private static NCAppRuntimeContext ncAppCtx;
@@ -154,6 +159,9 @@ public class CheckpointInSecondaryIndexTest {
         TestHelper.deleteExistingInstanceFiles();
     }
 
+    @Rule
+    public TestRule tracer = new TestMethodTracer();
+
     @Before
     public void createIndex() throws Exception {
         List<List<String>> partitioningKeys = new ArrayList<>();
@@ -206,50 +214,47 @@ public class CheckpointInSecondaryIndexTest {
 
     @Test
     public void testCheckpointUpdatedWhenSecondaryIsEmpty() throws Exception {
-        try {
-            // create secondary
-            createSecondaryIndex();
-            actor.add(new Request(Request.Action.INSERT_PATCH));
-            ensureDone(actor);
-            // search now and ensure partition 0 has all the records
-            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
-            // and that secondary index is empty
-            Assert.assertTrue(secondaryLsmBtree.isCurrentMutableComponentEmpty());
-            // flush
-            actor.add(new Request(Request.Action.FLUSH_DATASET));
-            ensureDone(actor);
-            // ensure primary has a component
-            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+        // create secondary
+        createSecondaryIndex();
+        actor.add(new Request(Request.Action.INSERT_PATCH));
+        ensureDone(actor);
+        // search now and ensure partition 0 has all the records
+        StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+        // and that secondary index is empty
+        Assert.assertTrue(secondaryLsmBtree.isCurrentMutableComponentEmpty());
+        // flush
+        actor.add(new Request(Request.Action.FLUSH_DATASET));
+        ensureDone(actor);
+        // ensure primary has a component
+        Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
 
-            // ensure secondary doesn't have a component
-            Assert.assertEquals(0, secondaryLsmBtree.getDiskComponents().size());
-            // ensure that current memory component index match
-            Assert.assertEquals(secondaryLsmBtree.getCurrentMemoryComponentIndex(),
-                    primaryLsmBtree.getCurrentMemoryComponentIndex());
-            // ensure both checkpoint files has the same component id as the last flushed component id
-            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
-            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
-            long min = id.getMinId();
-            // primary ref
-            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
-            fileManagerField.setAccessible(true); //Make it accessible so you can access it
-            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
-            final ResourceReference primaryRef = ResourceReference
-                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
-            // secondary ref
-            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
-            final ResourceReference secondaryRef = ResourceReference.of(
-                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
-            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
-            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
+        // ensure secondary doesn't have a component
+        Assert.assertEquals(0, secondaryLsmBtree.getDiskComponents().size());
+        // ensure that current memory component index match
+        Assert.assertEquals(secondaryLsmBtree.getCurrentMemoryComponentIndex(),
+                primaryLsmBtree.getCurrentMemoryComponentIndex());
+        // ensure both checkpoint files has the same component id as the last flushed component id
+        ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+        LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+        long min = id.getMinId();
+        // primary ref
+        Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+        fileManagerField.setAccessible(true); //Make it accessible so you can access it
+        ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+        final ResourceReference primaryRef = ResourceReference
+                .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+        // secondary ref
+        ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+        final ResourceReference secondaryRef = ResourceReference
+                .of(secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+        IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+        Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        Assert.assertTrue(latestPrimaryCheckpoint.hasNullMissingValuesFix());
+        Assert.assertTrue(latestSecondaryCheckpoint.hasNullMissingValuesFix());
     }
 
     private void createSecondaryIndex()
@@ -266,204 +271,192 @@ public class CheckpointInSecondaryIndexTest {
 
     @Test
     public void testCheckpointWhenBulkloadingSecondaryAndPrimaryIsSingleComponent() throws Exception {
-        try {
-            // create secondary
-            actor.add(new Request(Request.Action.INSERT_PATCH));
-            ensureDone(actor);
-            // search now and ensure partition 0 has all the records
-            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
-            // flush
-            actor.add(new Request(Request.Action.FLUSH_DATASET));
-            ensureDone(actor);
-            // ensure primary has a component
-            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
-            // ensure both checkpoint files has the same component id as the last flushed component id
-            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
-            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
-            long min = id.getMinId();
-            // primary ref
-            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
-            fileManagerField.setAccessible(true); //Make it accessible so you can access it
-            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
-            final ResourceReference primaryRef = ResourceReference
-                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
-            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
-            createSecondaryIndex();
-            JobId jobId = nc.newJobId();
-            loadTaskCtx = nc.createTestContext(jobId, 0, false);
-            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
-                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
-                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
-            indexLoadOp = infoAndOp.getRight();
-            secondaryIndexInfo = infoAndOp.getLeft();
-            actor.add(new Request(Request.Action.LOAD_OPEN));
-            actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
-            actor.add(new Request(Request.Action.LOAD_CLOSE));
-            ensureDone(actor);
-            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
-            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
-            final ResourceReference secondaryRef = ResourceReference.of(
-                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
-            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
+        // create secondary
+        actor.add(new Request(Request.Action.INSERT_PATCH));
+        ensureDone(actor);
+        // search now and ensure partition 0 has all the records
+        StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+        // flush
+        actor.add(new Request(Request.Action.FLUSH_DATASET));
+        ensureDone(actor);
+        // ensure primary has a component
+        Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+        // ensure both checkpoint files has the same component id as the last flushed component id
+        ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+        LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+        long min = id.getMinId();
+        // primary ref
+        Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+        fileManagerField.setAccessible(true); //Make it accessible so you can access it
+        ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+        final ResourceReference primaryRef = ResourceReference
+                .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+        IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+        createSecondaryIndex();
+        JobId jobId = nc.newJobId();
+        loadTaskCtx = nc.createTestContext(jobId, 0, false);
+        Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                        KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+        indexLoadOp = infoAndOp.getRight();
+        secondaryIndexInfo = infoAndOp.getLeft();
+        actor.add(new Request(Request.Action.LOAD_OPEN));
+        actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
+        actor.add(new Request(Request.Action.LOAD_CLOSE));
+        ensureDone(actor);
+        latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+        ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+        final ResourceReference secondaryRef = ResourceReference
+                .of(secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+        IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        Assert.assertTrue(latestPrimaryCheckpoint.hasNullMissingValuesFix());
+        Assert.assertTrue(latestSecondaryCheckpoint.hasNullMissingValuesFix());
     }
 
     @Test
     public void testCheckpointWhenBulkloadingSecondaryAndPrimaryIsTwoComponents() throws Exception {
-        try {
-            // create secondary
-            actor.add(new Request(Request.Action.INSERT_PATCH));
-            ensureDone(actor);
-            // search now and ensure partition 0 has all the records
-            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
-            // flush
-            actor.add(new Request(Request.Action.FLUSH_DATASET));
-            ensureDone(actor);
-            // ensure primary has a component
-            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
-            // ensure both checkpoint files has the same component id as the last flushed component id
-            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
-            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
-            long min = id.getMinId();
-            // primary ref
-            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
-            fileManagerField.setAccessible(true); //Make it accessible so you can access it
-            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
-            final ResourceReference primaryRef = ResourceReference
-                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
-            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
-            actor.add(new Request(Request.Action.INSERT_PATCH));
-            ensureDone(actor);
-            actor.add(new Request(Request.Action.FLUSH_DATASET));
-            ensureDone(actor);
-            Assert.assertEquals(2, primaryLsmBtree.getDiskComponents().size());
-            // ensure both checkpoint files has the same component id as the last flushed component id
-            primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
-            id = (LSMComponentId) primaryDiskComponent.getId();
-            min = id.getMaxId();
-            createSecondaryIndex();
-            JobId jobId = nc.newJobId();
-            loadTaskCtx = nc.createTestContext(jobId, 0, false);
-            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
-                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
-                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
-            indexLoadOp = infoAndOp.getRight();
-            secondaryIndexInfo = infoAndOp.getLeft();
-            actor.add(new Request(Request.Action.LOAD_OPEN));
-            actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
-            actor.add(new Request(Request.Action.LOAD_CLOSE));
-            ensureDone(actor);
-            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
-            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
-            final ResourceReference secondaryRef = ResourceReference.of(
-                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
-            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
+        // create secondary
+        actor.add(new Request(Request.Action.INSERT_PATCH));
+        ensureDone(actor);
+        // search now and ensure partition 0 has all the records
+        StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+        // flush
+        actor.add(new Request(Request.Action.FLUSH_DATASET));
+        ensureDone(actor);
+        // ensure primary has a component
+        Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+        // ensure both checkpoint files has the same component id as the last flushed component id
+        ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+        LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+        long min = id.getMinId();
+        // primary ref
+        Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+        fileManagerField.setAccessible(true); //Make it accessible so you can access it
+        ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+        final ResourceReference primaryRef = ResourceReference
+                .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+        IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+        actor.add(new Request(Request.Action.INSERT_PATCH));
+        ensureDone(actor);
+        actor.add(new Request(Request.Action.FLUSH_DATASET));
+        ensureDone(actor);
+        Assert.assertEquals(2, primaryLsmBtree.getDiskComponents().size());
+        // ensure both checkpoint files has the same component id as the last flushed component id
+        primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+        id = (LSMComponentId) primaryDiskComponent.getId();
+        min = id.getMaxId();
+        createSecondaryIndex();
+        JobId jobId = nc.newJobId();
+        loadTaskCtx = nc.createTestContext(jobId, 0, false);
+        Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                        KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+        indexLoadOp = infoAndOp.getRight();
+        secondaryIndexInfo = infoAndOp.getLeft();
+        actor.add(new Request(Request.Action.LOAD_OPEN));
+        actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
+        actor.add(new Request(Request.Action.LOAD_CLOSE));
+        ensureDone(actor);
+        latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+        ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+        final ResourceReference secondaryRef = ResourceReference
+                .of(secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+        IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        Assert.assertTrue(latestPrimaryCheckpoint.hasNullMissingValuesFix());
+        Assert.assertTrue(latestSecondaryCheckpoint.hasNullMissingValuesFix());
     }
 
     @Test
     public void testCheckpointWhenBulkloadedSecondaryIsEmptyAndPrimaryIsEmpty() throws Exception {
-        try {
-            // ensure primary has no component
-            Assert.assertEquals(0, primaryLsmBtree.getDiskComponents().size());
-            // primary ref
-            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
-            fileManagerField.setAccessible(true); //Make it accessible so you can access it
-            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
-            final ResourceReference primaryRef = ResourceReference
-                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
-            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            createSecondaryIndex();
-            JobId jobId = nc.newJobId();
-            loadTaskCtx = nc.createTestContext(jobId, 0, false);
-            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
-                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
-                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
-            indexLoadOp = infoAndOp.getRight();
-            secondaryIndexInfo = infoAndOp.getLeft();
-            actor.add(new Request(Request.Action.LOAD_OPEN));
-            actor.add(new Request(Request.Action.LOAD_CLOSE));
-            ensureDone(actor);
-            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
-            final ResourceReference secondaryRef = ResourceReference.of(
-                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
-            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(),
-                    latestPrimaryCheckpoint.getLastComponentId());
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
+        // ensure primary has no component
+        Assert.assertEquals(0, primaryLsmBtree.getDiskComponents().size());
+        // primary ref
+        Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+        fileManagerField.setAccessible(true); //Make it accessible so you can access it
+        ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+        final ResourceReference primaryRef = ResourceReference
+                .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+        IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        createSecondaryIndex();
+        JobId jobId = nc.newJobId();
+        loadTaskCtx = nc.createTestContext(jobId, 0, false);
+        Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                        KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+        indexLoadOp = infoAndOp.getRight();
+        secondaryIndexInfo = infoAndOp.getLeft();
+        actor.add(new Request(Request.Action.LOAD_OPEN));
+        actor.add(new Request(Request.Action.LOAD_CLOSE));
+        ensureDone(actor);
+        latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+        final ResourceReference secondaryRef = ResourceReference
+                .of(secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+        IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(),
+                latestPrimaryCheckpoint.getLastComponentId());
+        Assert.assertTrue(latestPrimaryCheckpoint.hasNullMissingValuesFix());
+        Assert.assertTrue(latestSecondaryCheckpoint.hasNullMissingValuesFix());
     }
 
     @Test
     public void testCheckpointWhenBulkloadedSecondaryIsEmptyAndPrimaryIsNotEmpty() throws Exception {
-        try {
-            // create secondary
-            actor.add(new Request(Request.Action.INSERT_PATCH));
-            ensureDone(actor);
-            // search now and ensure partition 0 has all the records
-            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
-            // flush
-            actor.add(new Request(Request.Action.FLUSH_DATASET));
-            ensureDone(actor);
-            // ensure primary has a component
-            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
-            // ensure both checkpoint files has the same component id as the last flushed component id
-            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
-            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
-            long min = id.getMinId();
-            // primary ref
-            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
-            fileManagerField.setAccessible(true); //Make it accessible so you can access it
-            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
-            final ResourceReference primaryRef = ResourceReference
-                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
-            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
-            createSecondaryIndex();
-            JobId jobId = nc.newJobId();
-            loadTaskCtx = nc.createTestContext(jobId, 0, false);
-            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
-                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
-                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
-            indexLoadOp = infoAndOp.getRight();
-            secondaryIndexInfo = infoAndOp.getLeft();
-            actor.add(new Request(Request.Action.LOAD_OPEN));
-            actor.add(new Request(Request.Action.LOAD_CLOSE));
-            ensureDone(actor);
-            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
-            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
-            final ResourceReference secondaryRef = ResourceReference.of(
-                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
-            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
-            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
-            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
+        // create secondary
+        actor.add(new Request(Request.Action.INSERT_PATCH));
+        ensureDone(actor);
+        // search now and ensure partition 0 has all the records
+        StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+        // flush
+        actor.add(new Request(Request.Action.FLUSH_DATASET));
+        ensureDone(actor);
+        // ensure primary has a component
+        Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+        // ensure both checkpoint files has the same component id as the last flushed component id
+        ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+        LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+        long min = id.getMinId();
+        // primary ref
+        Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+        fileManagerField.setAccessible(true); //Make it accessible so you can access it
+        ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+        final ResourceReference primaryRef = ResourceReference
+                .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+        IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+        createSecondaryIndex();
+        JobId jobId = nc.newJobId();
+        loadTaskCtx = nc.createTestContext(jobId, 0, false);
+        Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                        KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+        indexLoadOp = infoAndOp.getRight();
+        secondaryIndexInfo = infoAndOp.getLeft();
+        actor.add(new Request(Request.Action.LOAD_OPEN));
+        actor.add(new Request(Request.Action.LOAD_CLOSE));
+        ensureDone(actor);
+        latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+        ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+        final ResourceReference secondaryRef = ResourceReference
+                .of(secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+        IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+        IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+        Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        Assert.assertTrue(latestPrimaryCheckpoint.hasNullMissingValuesFix());
+        Assert.assertTrue(latestSecondaryCheckpoint.hasNullMissingValuesFix());
     }
 
     protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
@@ -566,9 +559,6 @@ public class CheckpointInSecondaryIndexTest {
                     default:
                         break;
                 }
-            } catch (Throwable th) {
-                th.printStackTrace();
-                throw th;
             } finally {
                 req.complete();
             }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
index cb34600..3d0b9cb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
@@ -27,14 +27,18 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class IndexCheckpoint {
 
     private static final Logger LOGGER = LogManager.getLogger();
     private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
     private static final long INITIAL_CHECKPOINT_ID = 0;
+    // TODO(mblow): remove this marker & related logic once we no longer are able to read indexes prior to the fix
+    private static final long HAS_NULL_MISSING_VALUES_FIX = -1;
     private long id;
     private long validComponentSequence;
     private long lowWatermark;
@@ -48,6 +52,7 @@ public class IndexCheckpoint {
         firstCheckpoint.validComponentSequence = lastComponentSequence;
         firstCheckpoint.lastComponentId = validComponentId;
         firstCheckpoint.masterNodeFlushMap = new HashMap<>();
+        firstCheckpoint.masterNodeFlushMap.put(HAS_NULL_MISSING_VALUES_FIX, HAS_NULL_MISSING_VALUES_FIX);
         return firstCheckpoint;
     }
 
@@ -66,7 +71,7 @@ public class IndexCheckpoint {
         next.validComponentSequence = validComponentSequence;
         next.masterNodeFlushMap = latest.getMasterNodeFlushMap();
         // remove any lsn from the map that wont be used anymore
-        next.masterNodeFlushMap.values().removeIf(lsn -> lsn <= lowWatermark);
+        next.masterNodeFlushMap.values().removeIf(lsn -> lsn <= lowWatermark && lsn != HAS_NULL_MISSING_VALUES_FIX);
         return next;
     }
 
@@ -94,6 +99,10 @@ public class IndexCheckpoint {
         return id;
     }
 
+    public boolean hasNullMissingValuesFix() {
+        return masterNodeFlushMap.containsKey(HAS_NULL_MISSING_VALUES_FIX);
+    }
+
     public String asJson() throws HyracksDataException {
         try {
             return OBJECT_MAPPER.writeValueAsString(this);


[asterixdb] 03/06: [NO ISSUE][HYR] Use System.nanotime for StopWatch

Posted by dl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dlych pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit d8229a82a492eac70034444344c1521cee8cf3bc
Author: Michael Blow <mi...@couchbase.com>
AuthorDate: Fri Apr 12 09:44:06 2019 -0400

    [NO ISSUE][HYR] Use System.nanotime for StopWatch
    
    Change-Id: Ic64f1b938c26161f5fb482af5ee1da6121f49643
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3339
    Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Ian Maxon <im...@uci.edu>
---
 .../java/org/apache/hyracks/api/util/StopWatch.java   | 19 +++++++------------
 1 file changed, 7 insertions(+), 12 deletions(-)

diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/StopWatch.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/StopWatch.java
index e752cb3..a1a39cb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/StopWatch.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/StopWatch.java
@@ -19,32 +19,27 @@
 
 package org.apache.hyracks.api.util;
 
+import java.util.concurrent.TimeUnit;
+
 public class StopWatch {
     private long startTime = 0;
-    private long stopTime = 0;
     private long elapsedTime = 0;
 
     public void start() {
         elapsedTime = 0;
-        startTime = System.currentTimeMillis();
+        startTime = System.nanoTime();
     }
 
     public void stop() {
-        stopTime = System.currentTimeMillis();
-        elapsedTime += stopTime - startTime;
+        elapsedTime += System.nanoTime() - startTime;
     }
 
     public void resume() {
-        startTime = System.currentTimeMillis();
+        startTime = System.nanoTime();
     }
 
-    //elaspsed time in milliseconds
-    public long getElapsedTime() {
-        return elapsedTime;
+    public long elapsed(TimeUnit unit) {
+        return unit.convert(elapsedTime, TimeUnit.NANOSECONDS);
     }
 
-    //elaspsed time in seconds
-    public long getElapsedTimeSecs() {
-        return elapsedTime / 1000;
-    }
 }
\ No newline at end of file