You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org> on 2015/09/16 22:43:51 UTC

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

Yingyi Bu has uploaded a new change for review.

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread
......................................................................

Set the priority of the following threads to be Thread.MAX_PRIORITY:
1. heartbeat thread at NC
2. IPC network thread

Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
---
A hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java
M hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
M hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
M hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
M hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
M hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
7 files changed, 97 insertions(+), 22 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/89/389/1

diff --git a/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java b/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java
new file mode 100644
index 0000000..6749eab
--- /dev/null
+++ b/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.hyracks.client.heartbeat;
+
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.hyracks.api.client.HyracksConnection;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.client.NodeControllerInfo;
+import org.apache.hyracks.client.stats.HyracksUtils;
+import org.junit.Test;
+
+public class HeartBeatTest {
+
+    @Test
+    public void test() throws Exception {
+        HyracksUtils.init();
+        int cores = Runtime.getRuntime().availableProcessors();
+        Thread[] cpuConsumers = new Thread[cores * 4];
+        for (int i = 0; i < cpuConsumers.length; i++) {
+            cpuConsumers[i] = new Thread(new Runnable() {
+                private long counter = 0;
+                private long round = 0;
+
+                @Override
+                public void run() {
+                    while (true) {
+                        counter = (counter + 1) % Long.MAX_VALUE - 1;
+                        if (counter == 0) {
+                            round++;
+                            if (round == 8L * Integer.MAX_VALUE) {
+                                return;
+                            }
+                        }
+                    }
+                }
+
+            });
+            cpuConsumers[i].setPriority(Thread.MIN_PRIORITY);
+            cpuConsumers[i].start();
+        }
+        for (int i = 0; i < cpuConsumers.length; i++) {
+            cpuConsumers[i].join();
+        }
+        IHyracksClientConnection hcc = new HyracksConnection("localhost", 2099);
+        Map<String, NodeControllerInfo> ncMap = hcc.getNodeControllerInfos();
+        Assert.assertEquals(ncMap.size(), 2);
+        HyracksUtils.deinit();
+    }
+}
diff --git a/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java b/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
index a41ddd9..f3c12dc 100644
--- a/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
+++ b/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
@@ -57,7 +57,8 @@
         ccConfig.defaultMaxJobAttempts = 0;
         ccConfig.jobHistorySize = 0;
         ccConfig.profileDumpPeriod = -1;
-        ccConfig.heartbeatPeriod = 50;
+        ccConfig.heartbeatPeriod = 60;
+        ccConfig.maxHeartbeatLapsePeriods = 5;
 
         // cluster controller
         cc = new ClusterControllerService(ccConfig);
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
index e0d0eb0..94b8d97 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
@@ -96,7 +96,7 @@
     private class WorkerThread extends Thread {
         WorkerThread() {
             setDaemon(true);
-            setPriority(MAX_PRIORITY);
+            setPriority(Thread.MAX_PRIORITY);
         }
 
         @Override
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index e7689d4..92eb0ae 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -26,6 +26,7 @@
 import java.lang.management.OperatingSystemMXBean;
 import java.lang.management.RuntimeMXBean;
 import java.lang.management.ThreadMXBean;
+import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Hashtable;
@@ -42,7 +43,6 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.api.application.INCApplicationEntryPoint;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.comm.NetworkAddress;
@@ -170,8 +170,8 @@
             throw new Exception("id not set");
         }
         partitionManager = new PartitionManager(this);
-        netManager = new NetworkManager(ncConfig.dataIPAddress, ncConfig.dataPort, partitionManager, ncConfig.nNetThreads,
-                                        ncConfig.nNetBuffers, ncConfig.dataPublicIPAddress, ncConfig.dataPublicPort);
+        netManager = new NetworkManager(ncConfig.dataIPAddress, ncConfig.dataPort, partitionManager,
+                ncConfig.nNetThreads, ncConfig.nNetBuffers, ncConfig.dataPublicIPAddress, ncConfig.dataPublicPort);
 
         lccm = new LifeCycleComponentManager();
         queue = new WorkQueue();
@@ -243,11 +243,11 @@
 
     private void init() throws Exception {
         ctx.getIOManager().setExecutor(executor);
-        datasetPartitionManager = new DatasetPartitionManager
-            (this, executor, ncConfig.resultManagerMemory, ncConfig.resultTTL, ncConfig.resultSweepThreshold);
-        datasetNetworkManager = new DatasetNetworkManager
-            (ncConfig.resultIPAddress, ncConfig.resultPort, datasetPartitionManager,
-             ncConfig.nNetThreads, ncConfig.nNetBuffers, ncConfig.resultPublicIPAddress, ncConfig.resultPublicPort);
+        datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory,
+                ncConfig.resultTTL, ncConfig.resultSweepThreshold);
+        datasetNetworkManager = new DatasetNetworkManager(ncConfig.resultIPAddress, ncConfig.resultPort,
+                datasetPartitionManager, ncConfig.nNetThreads, ncConfig.nNetBuffers, ncConfig.resultPublicIPAddress,
+                ncConfig.resultPublicPort);
     }
 
     @Override
@@ -273,12 +273,11 @@
         if (ncConfig.dataPublicIPAddress != null) {
             netAddress = new NetworkAddress(ncConfig.dataPublicIPAddress, ncConfig.dataPublicPort);
         }
-        ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netAddress,
-                datasetAddress, osMXBean.getName(), osMXBean.getArch(), osMXBean
-                        .getVersion(), osMXBean.getAvailableProcessors(), runtimeMXBean.getVmName(), runtimeMXBean
-                        .getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean.getClassPath(), runtimeMXBean
-                        .getLibraryPath(), runtimeMXBean.getBootClassPath(), runtimeMXBean.getInputArguments(),
-                runtimeMXBean.getSystemProperties(), hbSchema));
+        ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netAddress, datasetAddress,
+                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
+                runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean
+                        .getClassPath(), runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(),
+                runtimeMXBean.getInputArguments(), runtimeMXBean.getSystemProperties(), hbSchema));
 
         synchronized (this) {
             while (registrationPending) {
@@ -294,6 +293,11 @@
 
         heartbeatTask = new HeartbeatTask(ccs);
 
+        // Use reflection to set the priority of the timer thread.
+        Field threadField = timer.getClass().getDeclaredField("thread");
+        threadField.setAccessible(true);
+        Thread timerThread = (Thread) threadField.get(timer); // The internal timer thread of the Timer object.
+        timerThread.setPriority(Thread.MAX_PRIORITY);
         // Schedule heartbeat generator.
         timer.schedule(heartbeatTask, 0, nodeParameters.getHeartbeatPeriod());
 
@@ -571,6 +575,7 @@
             this.nodeControllerService = ncAppEntryPoint;
         }
 
+        @Override
         public void run() {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Shutdown hook in progress");
diff --git a/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java b/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
index dd3d2f9..e02e4f4 100644
--- a/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
+++ b/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
@@ -64,6 +64,7 @@
     IPCConnectionManager(IPCSystem system, InetSocketAddress socketAddress) throws IOException {
         this.system = system;
         this.networkThread = new NetworkThread();
+        this.networkThread.setPriority(Thread.MAX_PRIORITY);
         this.serverSocketChannel = ServerSocketChannel.open();
         serverSocketChannel.socket().setReuseAddress(true);
         serverSocketChannel.configureBlocking(false);
@@ -114,8 +115,8 @@
                 }
             } else if (attempt < retries) {
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Connection to " + remoteAddress +
-                            " failed (Attempt " + attempt + " of " + retries + ")");
+                    LOGGER.info("Connection to " + remoteAddress + " failed (Attempt " + attempt + " of " + retries
+                            + ")");
                     attempt++;
                     Thread.sleep(5000);
                 }
@@ -308,8 +309,7 @@
                                     if (!channel.finishConnect()) {
                                         throw new Exception("Connection did not finish");
                                     }
-                                }
-                                catch (Exception e) {
+                                } catch (Exception e) {
                                     e.printStackTrace();
                                     handle.setState(HandleState.CONNECT_FAILED);
                                     continue;
diff --git a/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java b/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
index 42e19f7..4b6c22f 100644
--- a/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
+++ b/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
@@ -106,7 +106,7 @@
         public IOThread() throws IOException {
             super("TCPEndpoint IO Thread");
             setDaemon(true);
-            setPriority(MAX_PRIORITY);
+            setPriority(Thread.NORM_PRIORITY);
             this.pendingConnections = new ArrayList<InetSocketAddress>();
             this.workingPendingConnections = new ArrayList<InetSocketAddress>();
             this.incomingConnections = new ArrayList<SocketChannel>();
diff --git a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 4f9d9ce..facd8c1 100644
--- a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -475,7 +475,7 @@
         private int cleanedCount = 0;
 
         public CleanerThread() {
-            setPriority(MAX_PRIORITY);
+            setPriority(Thread.NORM_PRIORITY);
             setDaemon(true);
         }
 

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 6: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/461/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

Posted by "Yingyi Bu (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/389

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. the work queue thread in the cluster controller
......................................................................

Set the priority of the following threads to be Thread.MAX_PRIORITY:
1. heartbeat thread at NC
2. IPC network thread
3. the work queue thread in the cluster controller

This change cannot guarantee no false failures, but should reduce the possibility of false failures.

Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
---
A hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java
M hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
M hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
M hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
M hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
M hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
8 files changed, 115 insertions(+), 30 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/89/389/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/389
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/460/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread
......................................................................


Patch Set 1: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/456/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

Posted by "Yingyi Bu (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/389

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................

Set the priority of the following threads to be Thread.MAX_PRIORITY:
1. heartbeat thread at NC
2. IPC network thread
3. work queue thread in CC

This change cannot guarantee no false failures, but should reduce the possibility of false failures.

Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
---
A hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java
M hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
M hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
M hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
M hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
M hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
8 files changed, 115 insertions(+), 30 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/89/389/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/389
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 6: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Set the priority of the following threads to be Thread.MAX_PRIORITY:
1. heartbeat thread at NC
2. IPC network thread
3. work queue thread in CC

Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Reviewed-on: https://asterix-gerrit.ics.uci.edu/389
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
---
M hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
M hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
M hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
M hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
6 files changed, 41 insertions(+), 29 deletions(-)

Approvals:
  Till Westmann: Looks good to me, approved
  Jenkins: Verified



diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
index 9ae7f77..f2f450d 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
@@ -35,8 +35,6 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.xml.sax.InputSource;
-
 import org.apache.hyracks.api.application.ICCApplicationEntryPoint;
 import org.apache.hyracks.api.client.ClusterControllerInfo;
 import org.apache.hyracks.api.client.HyracksClientInterfaceFunctions;
@@ -104,6 +102,7 @@
 import org.apache.hyracks.ipc.exceptions.IPCException;
 import org.apache.hyracks.ipc.impl.IPCSystem;
 import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+import org.xml.sax.InputSource;
 
 public class ClusterControllerService extends AbstractRemoteService {
     private static Logger LOGGER = Logger.getLogger(ClusterControllerService.class.getName());
@@ -172,6 +171,7 @@
         runMapArchive = new LinkedHashMap<JobId, JobRun>() {
             private static final long serialVersionUID = 1L;
 
+            @Override
             protected boolean removeEldestEntry(Map.Entry<JobId, JobRun> eldest) {
                 return size() > ccConfig.jobHistorySize;
             }
@@ -181,11 +181,12 @@
             /** history size + 1 is for the case when history size = 0 */
             private int allowedSize = 100 * (ccConfig.jobHistorySize + 1);
 
+            @Override
             protected boolean removeEldestEntry(Map.Entry<JobId, List<Exception>> eldest) {
                 return size() > allowedSize;
             }
         };
-        workQueue = new WorkQueue();
+        workQueue = new WorkQueue(Thread.MAX_PRIORITY); // WorkQueue is in charge of heartbeat as well as other events.
         this.timer = new Timer(true);
         final ClusterTopology topology = computeClusterTopology(ccConfig);
         ccContext = new ICCContext() {
@@ -604,7 +605,7 @@
 
     /**
      * Add a deployment run
-     * 
+     *
      * @param deploymentKey
      * @param nodeControllerIds
      */
@@ -614,7 +615,7 @@
 
     /**
      * Get a deployment run
-     * 
+     *
      * @param deploymentKey
      */
     public synchronized DeploymentRun getDeploymentRun(DeploymentId deploymentKey) {
@@ -623,7 +624,7 @@
 
     /**
      * Remove a deployment run
-     * 
+     *
      * @param deploymentKey
      */
     public synchronized void removeDeploymentRun(DeploymentId deploymentKey) {
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
index e0d0eb0..da341a7 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
@@ -36,8 +36,14 @@
     private boolean stopped;
     private AtomicInteger enqueueCount;
     private AtomicInteger dequeueCount;
+    private int threadPriority = Thread.MAX_PRIORITY;
 
-    public WorkQueue() {
+    public WorkQueue(int threadPriority) {
+        if (threadPriority != Thread.MAX_PRIORITY && threadPriority != Thread.NORM_PRIORITY
+                && threadPriority != Thread.MIN_PRIORITY) {
+            throw new IllegalArgumentException("Illegal thread priority number.");
+        }
+        this.threadPriority = threadPriority;
         queue = new LinkedBlockingQueue<AbstractWork>();
         thread = new WorkerThread();
         stopSemaphore = new Semaphore(1);
@@ -96,7 +102,7 @@
     private class WorkerThread extends Thread {
         WorkerThread() {
             setDaemon(true);
-            setPriority(MAX_PRIORITY);
+            setPriority(threadPriority);
         }
 
         @Override
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index e7689d4..ab0f16b 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -26,6 +26,7 @@
 import java.lang.management.OperatingSystemMXBean;
 import java.lang.management.RuntimeMXBean;
 import java.lang.management.ThreadMXBean;
+import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Hashtable;
@@ -42,7 +43,6 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.api.application.INCApplicationEntryPoint;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.comm.NetworkAddress;
@@ -170,11 +170,11 @@
             throw new Exception("id not set");
         }
         partitionManager = new PartitionManager(this);
-        netManager = new NetworkManager(ncConfig.dataIPAddress, ncConfig.dataPort, partitionManager, ncConfig.nNetThreads,
-                                        ncConfig.nNetBuffers, ncConfig.dataPublicIPAddress, ncConfig.dataPublicPort);
+        netManager = new NetworkManager(ncConfig.dataIPAddress, ncConfig.dataPort, partitionManager,
+                ncConfig.nNetThreads, ncConfig.nNetBuffers, ncConfig.dataPublicIPAddress, ncConfig.dataPublicPort);
 
         lccm = new LifeCycleComponentManager();
-        queue = new WorkQueue();
+        queue = new WorkQueue(Thread.NORM_PRIORITY); // Reserves MAX_PRIORITY of the heartbeat thread.
         jobletMap = new Hashtable<JobId, Joblet>();
         timer = new Timer(true);
         serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
@@ -243,11 +243,11 @@
 
     private void init() throws Exception {
         ctx.getIOManager().setExecutor(executor);
-        datasetPartitionManager = new DatasetPartitionManager
-            (this, executor, ncConfig.resultManagerMemory, ncConfig.resultTTL, ncConfig.resultSweepThreshold);
-        datasetNetworkManager = new DatasetNetworkManager
-            (ncConfig.resultIPAddress, ncConfig.resultPort, datasetPartitionManager,
-             ncConfig.nNetThreads, ncConfig.nNetBuffers, ncConfig.resultPublicIPAddress, ncConfig.resultPublicPort);
+        datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory,
+                ncConfig.resultTTL, ncConfig.resultSweepThreshold);
+        datasetNetworkManager = new DatasetNetworkManager(ncConfig.resultIPAddress, ncConfig.resultPort,
+                datasetPartitionManager, ncConfig.nNetThreads, ncConfig.nNetBuffers, ncConfig.resultPublicIPAddress,
+                ncConfig.resultPublicPort);
     }
 
     @Override
@@ -273,12 +273,11 @@
         if (ncConfig.dataPublicIPAddress != null) {
             netAddress = new NetworkAddress(ncConfig.dataPublicIPAddress, ncConfig.dataPublicPort);
         }
-        ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netAddress,
-                datasetAddress, osMXBean.getName(), osMXBean.getArch(), osMXBean
-                        .getVersion(), osMXBean.getAvailableProcessors(), runtimeMXBean.getVmName(), runtimeMXBean
-                        .getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean.getClassPath(), runtimeMXBean
-                        .getLibraryPath(), runtimeMXBean.getBootClassPath(), runtimeMXBean.getInputArguments(),
-                runtimeMXBean.getSystemProperties(), hbSchema));
+        ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netAddress, datasetAddress,
+                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
+                runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean
+                        .getClassPath(), runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(),
+                runtimeMXBean.getInputArguments(), runtimeMXBean.getSystemProperties(), hbSchema));
 
         synchronized (this) {
             while (registrationPending) {
@@ -294,6 +293,11 @@
 
         heartbeatTask = new HeartbeatTask(ccs);
 
+        // Use reflection to set the priority of the timer thread.
+        Field threadField = timer.getClass().getDeclaredField("thread");
+        threadField.setAccessible(true);
+        Thread timerThread = (Thread) threadField.get(timer); // The internal timer thread of the Timer object.
+        timerThread.setPriority(Thread.MAX_PRIORITY);
         // Schedule heartbeat generator.
         timer.schedule(heartbeatTask, 0, nodeParameters.getHeartbeatPeriod());
 
@@ -571,6 +575,7 @@
             this.nodeControllerService = ncAppEntryPoint;
         }
 
+        @Override
         public void run() {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Shutdown hook in progress");
diff --git a/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java b/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
index dd3d2f9..e02e4f4 100644
--- a/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
+++ b/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
@@ -64,6 +64,7 @@
     IPCConnectionManager(IPCSystem system, InetSocketAddress socketAddress) throws IOException {
         this.system = system;
         this.networkThread = new NetworkThread();
+        this.networkThread.setPriority(Thread.MAX_PRIORITY);
         this.serverSocketChannel = ServerSocketChannel.open();
         serverSocketChannel.socket().setReuseAddress(true);
         serverSocketChannel.configureBlocking(false);
@@ -114,8 +115,8 @@
                 }
             } else if (attempt < retries) {
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Connection to " + remoteAddress +
-                            " failed (Attempt " + attempt + " of " + retries + ")");
+                    LOGGER.info("Connection to " + remoteAddress + " failed (Attempt " + attempt + " of " + retries
+                            + ")");
                     attempt++;
                     Thread.sleep(5000);
                 }
@@ -308,8 +309,7 @@
                                     if (!channel.finishConnect()) {
                                         throw new Exception("Connection did not finish");
                                     }
-                                }
-                                catch (Exception e) {
+                                } catch (Exception e) {
                                     e.printStackTrace();
                                     handle.setState(HandleState.CONNECT_FAILED);
                                     continue;
diff --git a/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java b/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
index 42e19f7..4b6c22f 100644
--- a/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
+++ b/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
@@ -106,7 +106,7 @@
         public IOThread() throws IOException {
             super("TCPEndpoint IO Thread");
             setDaemon(true);
-            setPriority(MAX_PRIORITY);
+            setPriority(Thread.NORM_PRIORITY);
             this.pendingConnections = new ArrayList<InetSocketAddress>();
             this.workingPendingConnections = new ArrayList<InetSocketAddress>();
             this.incomingConnections = new ArrayList<SocketChannel>();
diff --git a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 4f9d9ce..facd8c1 100644
--- a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -475,7 +475,7 @@
         private int cleanedCount = 0;
 
         public CleanerThread() {
-            setPriority(MAX_PRIORITY);
+            setPriority(Thread.NORM_PRIORITY);
             setDaemon(true);
         }
 

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 7
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/458/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 4:

(3 comments)

https://asterix-gerrit.ics.uci.edu/#/c/389/4/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java
File hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java:

Line 59:             cpuConsumers[i].setPriority(Thread.MIN_PRIORITY);
Should this be NORM_PRIORITY as the other worker threads?

Also, did you see an improvement in this test with the changes to the priorities (without changing the timeouts)?


https://asterix-gerrit.ics.uci.edu/#/c/389/4/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
File hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java:

Line 61:         ccConfig.maxHeartbeatLapsePeriods = 5;
What was the default before?


https://asterix-gerrit.ics.uci.edu/#/c/389/4/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
File hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java:

Line 300:         timerThread.setPriority(Thread.MAX_PRIORITY);
This looks slightly hackish :)


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 4: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/459/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 3: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/458/ : ABORTED

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/461/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 5: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/460/ : ABORTED

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/456/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/459/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................


Patch Set 4:

(3 comments)

I tried the test case with various configurations, the result depends on the hardware and OS.  Therefore, I have to drop the test. 

Also, the thread priority change doesn't seem to make much difference, at least in my test environment. 

Do we still want that change?   I guess this change at least should not be worse than before:-)

https://asterix-gerrit.ics.uci.edu/#/c/389/4/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java
File hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java:

Line 59:             cpuConsumers[i].setPriority(Thread.MIN_PRIORITY);
> Should this be NORM_PRIORITY as the other worker threads?
The test result really depends on the hardware and OS.  Therefore I dropped this test.


https://asterix-gerrit.ics.uci.edu/#/c/389/4/hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
File hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java:

Line 61:         ccConfig.maxHeartbeatLapsePeriods = 5;
> What was the default before?
The default is defined in CCConfig.java:
    @Option(name = "-heartbeat-period", usage = "Sets the time duration between two heartbeats from each node controller in milliseconds (default: 10000)")
    public int heartbeatPeriod = 10000;

    @Option(name = "-max-heartbeat-lapse-periods", usage = "Sets the maximum number of missed heartbeats before a node is marked as dead (default: 5)")
    public int maxHeartbeatLapsePeriods = 5;


https://asterix-gerrit.ics.uci.edu/#/c/389/4/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
File hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java:

Line 300:         timerThread.setPriority(Thread.MAX_PRIORITY);
> This looks slightly hackish :)
I tried to implement our own Timer, but it turned out tricky because in line 307 we use the same timer to schedule another Task which is the ProfileDumpTask.
So, this hacky way might be the smallest change to achieve the functionality:-)


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

Posted by "Yingyi Bu (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/389

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................

Set the priority of the following threads to be Thread.MAX_PRIORITY:
1. heartbeat thread at NC
2. IPC network thread
3. work queue thread in CC

Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
---
M hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
M hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
M hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
M hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
6 files changed, 41 insertions(+), 29 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/89/389/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/389
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

Posted by "Yingyi Bu (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/389

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................

Set the priority of the following threads to be Thread.MAX_PRIORITY:
1. heartbeat thread at NC
2. IPC network thread
3. work queue thread in CC

Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
---
M hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
M hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
M hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
M hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
M hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
7 files changed, 41 insertions(+), 30 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/89/389/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/389
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Set the priority of the following threads to be Thread.MAX_P...

Posted by "Yingyi Bu (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/389

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

Change subject: Set the priority of the following threads to be Thread.MAX_PRIORITY: 1. heartbeat thread at NC 2. IPC network thread 3. work queue thread in CC
......................................................................

Set the priority of the following threads to be Thread.MAX_PRIORITY:
1. heartbeat thread at NC
2. IPC network thread
3. work queue thread in CC

This change cannot guarantee no false failures, but should reduce the possibility of false failures.

Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
---
A hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/heartbeat/HeartBeatTest.java
M hyracks/hyracks-client/src/test/java/org/apache/hyracks/client/stats/HyracksUtils.java
M hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
M hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
M hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/tcp/TCPEndpoint.java
M hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
8 files changed, 115 insertions(+), 30 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/89/389/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/389
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4e53a85e21a6bdee48a3ca8d004569700f911fbd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>