You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Chris Hillery (Code Review)" <do...@asterixdb.incubator.apache.org> on 2016/05/20 07:32:53 UTC

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Chris Hillery has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/874

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................

Fixes and tweaks for running Asterix atop new config-management framework.

Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
---
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
3 files changed, 19 insertions(+), 11 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/74/874/1

diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
index 62bdbf5..45e3b06 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
@@ -136,12 +136,13 @@
         this.cfg = cfg;
         instanceName = cfg.getString("asterix", "instance", "DEFAULT_INSTANCE");
         String mdNode = null;
+        nodePartitionsMap = new HashMap<>();
+        int uniquePartitionId = 0;
         for (String section : cfg.getSections()) {
             if (!section.startsWith("nc/")) {
                 continue;
             }
             String ncId = section.substring(3);
-            nodeNames.add(ncId);
 
             if (mdNode == null) {
                 // Default is first node == metadata node
@@ -156,15 +157,23 @@
             // be a default.ini? They can't be inserted by TriggerNCWork except
             // possibly for hyracks-specified values. Certainly wherever they are,
             // they should be platform-dependent.
-            stores.put(ncId, cfg.getString(section, "iodevices", "/var/lib/asterixdb/data").split(","));
             coredumpConfig.put(ncId, cfg.getString(section, "coredumpdir", "/var/lib/asterixdb/coredump"));
             transactionLogDirs.put(ncId, cfg.getString(section, "txnlogdir", "/var/lib/asterixdb/txn-log"));
+            String[] storeDirs = cfg.getString(section, "storagedir", "storage").trim().split(",");
+            ClusterPartition[] nodePartitions = new ClusterPartition[storeDirs.length];
+            for (int i = 0; i < nodePartitions.length; i++) {
+                ClusterPartition partition = new ClusterPartition(uniquePartitionId++, ncId, i);
+                clusterPartitions.put(partition.getPartitionId(), partition);
+                nodePartitions[i] = partition;
+            }
+            stores.put(ncId, storeDirs);
+            nodePartitionsMap.put(ncId, nodePartitions);
+            nodeNames.add(ncId);
         }
 
         metadataNodeName = mdNode;
         asterixConfigurationParams = null;
         asterixBuildProperties = null;
-        nodePartitionsMap = null;
     }
 
     public String getMetadataNodeName() {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 561b144..399be3d 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -86,9 +86,9 @@
                 throw new HyracksDataException(mountPointDir.getAbsolutePath() + " doesn't exist.");
             }
             if (!mountPoint.endsWith(System.getProperty("file.separator"))) {
-                mountPoints[i] = new String(mountPoint + System.getProperty("file.separator"));
+                mountPoints[i] = mountPoint + System.getProperty("file.separator");
             } else {
-                mountPoints[i] = new String(mountPoint);
+                mountPoints[i] = mountPoint;
             }
         }
         resourceCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHED_RESOURCES).build();
@@ -136,10 +136,9 @@
 
             String storageRootDirPath;
             if (storageRootDirName.startsWith(System.getProperty("file.separator"))) {
-                storageRootDirPath = new String(
-                        mountPoints[i] + storageRootDirName.substring(System.getProperty("file.separator").length()));
+                storageRootDirPath = mountPoints[i] + storageRootDirName.substring(System.getProperty("file.separator").length());
             } else {
-                storageRootDirPath = new String(mountPoints[i] + storageRootDirName);
+                storageRootDirPath = mountPoints[i] + storageRootDirName;
             }
 
             LocalResource rootLocalResource = new LocalResource(STORAGE_LOCAL_RESOURCE_ID,
@@ -301,7 +300,7 @@
             if (!baseDir.endsWith(System.getProperty("file.separator"))) {
                 baseDir += System.getProperty("file.separator");
             }
-            return new String(baseDir + METADATA_FILE_NAME);
+            return baseDir + METADATA_FILE_NAME;
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
index af80b33..f44a291 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
@@ -38,8 +38,8 @@
     @Option(name = "-config-file", usage = "Local NC configuration file (default: none)", required = false)
     public String configFile = null;
 
-    @Option(name = "-address", usage = "Address to listen on for connections from CC (default: localhost)", required = false)
-    public String address = InetAddress.getLoopbackAddress().getHostAddress();
+    @Option(name = "-address", usage = "Address to listen on for connections from CC (default: all addresses)", required = false)
+    public String address = null;
 
     @Option(name = "-port", usage = "Port to listen on for connections from CC (default: 9090)", required = false)
     public int port = 9090;

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/1450/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Chris Hillery (Code Review)" <do...@asterixdb.incubator.apache.org>.
Chris Hillery has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 1:

Important change is in AsterixPropertiesAccessor, and it is only on the code path that is executed when using the new deployment mechanism, so it should be safe.

NCService change is to have NCService listen by default on all interfaces rather than just localhost, which is I think a better default and allows for a quicker out-of-the-box experience.

PersistentLocalResourceRepository changes are just cleanups that IDEA recommended - removing redundant String constructors.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 3:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/874/3/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java:

Line 162:             String[] storeDirs = cfg.getString(section, "storagedir", "storage").trim().split(",");
Did we give up on "iodevices"?


Line 169:             stores.put(ncId, storeDirs);
Just move this up to the declaration of storeDirs?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Michael Blow (Code Review)" <do...@asterixdb.incubator.apache.org>.
Michael Blow has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 3: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Michael Blow (Code Review)" <do...@asterixdb.incubator.apache.org>.
Michael Blow has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 2:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/874/2/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java:

Line 41:     @Option(name = "-address", usage = "Address to listen on for connections from CC (default: all addresses)", required = false)
line-width.  this wasn't reported by sonarqube since it was long in the original version as well.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/1443/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Chris Hillery (Code Review)" <do...@asterixdb.incubator.apache.org>.
Chris Hillery has submitted this change and it was merged.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Fixes and tweaks for running Asterix atop new config-management framework.

Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Reviewed-on: https://asterix-gerrit.ics.uci.edu/874
Reviewed-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
Reviewed-by: Michael Blow <mi...@couchbase.com>
---
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
3 files changed, 27 insertions(+), 14 deletions(-)

Approvals:
  Michael Blow: Looks good to me, approved
  Till Westmann: Looks good to me, approved
  Jenkins: Looks good to me, but someone else must approve; Verified



diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
index 62bdbf5..45e3b06 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
@@ -136,12 +136,13 @@
         this.cfg = cfg;
         instanceName = cfg.getString("asterix", "instance", "DEFAULT_INSTANCE");
         String mdNode = null;
+        nodePartitionsMap = new HashMap<>();
+        int uniquePartitionId = 0;
         for (String section : cfg.getSections()) {
             if (!section.startsWith("nc/")) {
                 continue;
             }
             String ncId = section.substring(3);
-            nodeNames.add(ncId);
 
             if (mdNode == null) {
                 // Default is first node == metadata node
@@ -156,15 +157,23 @@
             // be a default.ini? They can't be inserted by TriggerNCWork except
             // possibly for hyracks-specified values. Certainly wherever they are,
             // they should be platform-dependent.
-            stores.put(ncId, cfg.getString(section, "iodevices", "/var/lib/asterixdb/data").split(","));
             coredumpConfig.put(ncId, cfg.getString(section, "coredumpdir", "/var/lib/asterixdb/coredump"));
             transactionLogDirs.put(ncId, cfg.getString(section, "txnlogdir", "/var/lib/asterixdb/txn-log"));
+            String[] storeDirs = cfg.getString(section, "storagedir", "storage").trim().split(",");
+            ClusterPartition[] nodePartitions = new ClusterPartition[storeDirs.length];
+            for (int i = 0; i < nodePartitions.length; i++) {
+                ClusterPartition partition = new ClusterPartition(uniquePartitionId++, ncId, i);
+                clusterPartitions.put(partition.getPartitionId(), partition);
+                nodePartitions[i] = partition;
+            }
+            stores.put(ncId, storeDirs);
+            nodePartitionsMap.put(ncId, nodePartitions);
+            nodeNames.add(ncId);
         }
 
         metadataNodeName = mdNode;
         asterixConfigurationParams = null;
         asterixBuildProperties = null;
-        nodePartitionsMap = null;
     }
 
     public String getMetadataNodeName() {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 561b144..fc79ae0 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -86,9 +86,9 @@
                 throw new HyracksDataException(mountPointDir.getAbsolutePath() + " doesn't exist.");
             }
             if (!mountPoint.endsWith(System.getProperty("file.separator"))) {
-                mountPoints[i] = new String(mountPoint + System.getProperty("file.separator"));
+                mountPoints[i] = mountPoint + System.getProperty("file.separator");
             } else {
-                mountPoints[i] = new String(mountPoint);
+                mountPoints[i] = mountPoint;
             }
         }
         resourceCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHED_RESOURCES).build();
@@ -136,10 +136,10 @@
 
             String storageRootDirPath;
             if (storageRootDirName.startsWith(System.getProperty("file.separator"))) {
-                storageRootDirPath = new String(
-                        mountPoints[i] + storageRootDirName.substring(System.getProperty("file.separator").length()));
+                storageRootDirPath = mountPoints[i] +
+                        storageRootDirName.substring(System.getProperty("file.separator").length());
             } else {
-                storageRootDirPath = new String(mountPoints[i] + storageRootDirName);
+                storageRootDirPath = mountPoints[i] + storageRootDirName;
             }
 
             LocalResource rootLocalResource = new LocalResource(STORAGE_LOCAL_RESOURCE_ID,
@@ -301,7 +301,7 @@
             if (!baseDir.endsWith(System.getProperty("file.separator"))) {
                 baseDir += System.getProperty("file.separator");
             }
-            return new String(baseDir + METADATA_FILE_NAME);
+            return baseDir + METADATA_FILE_NAME;
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
index af80b33..bc982f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
@@ -35,16 +35,20 @@
      * If an option is specified both in the config file and on the command line, the config file
      * version will take precedence.
      */
-    @Option(name = "-config-file", usage = "Local NC configuration file (default: none)", required = false)
+    @Option(name = "-config-file", required = false,
+            usage = "Local NC configuration file (default: none)")
     public String configFile = null;
 
-    @Option(name = "-address", usage = "Address to listen on for connections from CC (default: localhost)", required = false)
-    public String address = InetAddress.getLoopbackAddress().getHostAddress();
+    @Option(name = "-address", required = false,
+            usage = "Address to listen on for connections from CC (default: all addresses)")
+    public String address = null;
 
-    @Option(name = "-port", usage = "Port to listen on for connections from CC (default: 9090)", required = false)
+    @Option(name = "-port", required = false,
+            usage = "Port to listen on for connections from CC (default: 9090)")
     public int port = 9090;
 
-    @Option(name = "-command", usage = "NC command to run (default: 'hyracksnc' on PATH)", required = false)
+    @Option(name = "-command", required = false,
+            usage = "NC command to run (default: 'hyracksnc' on PATH)")
     public String command = "hyracksnc";
 
     private Ini ini = null;

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Chris Hillery (Code Review)" <do...@asterixdb.incubator.apache.org>.
Chris Hillery has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 3:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/874/3/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java:

Line 162:             String[] storeDirs = cfg.getString(section, "storagedir", "storage").trim().split(",");
> Did we give up on "iodevices"?
No, "iodevices" is a Hyracks-level configuration parameter (my initial code reading it here was bogus).


Line 169:             stores.put(ncId, storeDirs);
> Just move this up to the declaration of storeDirs?
I copied this code from the other AsterixPropertiesAccessor, which did it in this order. I can move them both if you wish.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Chris Hillery (Code Review)" <do...@asterixdb.incubator.apache.org>.
Chris Hillery has uploaded a new patch set (#2).

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................

Fixes and tweaks for running Asterix atop new config-management framework.

Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
---
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
3 files changed, 20 insertions(+), 11 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/74/874/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Chris Hillery (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/874

to look at the new patch set (#3).

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................

Fixes and tweaks for running Asterix atop new config-management framework.

Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
---
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/src/main/java/org/apache/hyracks/control/nc/service/NCServiceConfig.java
3 files changed, 27 insertions(+), 14 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/74/874/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/1426/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 3: Code-Review+2

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/874/3/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java:

Line 162:             String[] storeDirs = cfg.getString(section, "storagedir", "storage").trim().split(",");
> No, "iodevices" is a Hyracks-level configuration parameter (my initial code
Done


Line 169:             stores.put(ncId, storeDirs);
> I copied this code from the other AsterixPropertiesAccessor, which did it i
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Fixes and tweaks for running Asterix atop new config-managem...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Fixes and tweaks for running Asterix atop new config-management framework.
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/1427/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/874
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie3027c8c839f25ea858790bd3340187f4b11f213
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Chris Hillery <ce...@lambda.nu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mi...@couchbase.com>
Gerrit-HasComments: No