// *Desired* distribution function on keys & replication factor. Replica #0 // is the master, replica #1, 2, 3, etc represent additional desired // replication nodes. Note that this function is just the distribution // function - it does not DO any replication, nor does it dictate any policy // on how fast replication occurs. Returns -1 if the desired replica // is nonsense, e.g. asking for replica #3 in a 2-Node system. int D(int repl) { int hsz = H2O.CLOUD.size(); // See if this is a specifically homed Key if (!user_allowed() && repl < _kb[1]) { // Asking for a replica# from the homed list? assert _kb[0] != Key.DVEC; H2ONode h2o = H2ONode.intern(_kb, 2 + repl * (4 + 2 /*serialized bytesize of H2OKey*/)); // Reverse the home to the index int idx = h2o.index(); if (idx >= 0) return idx; // Else homed to a node which is no longer in the cloud! // Fall back to the normal home mode } // Distribution of Fluid Vectors is a special case. // Fluid Vectors are grouped into vector groups, each of which must have // the same distribution of chunks so that MRTask2 run over group of // vectors will keep data-locality. The fluid vecs from the same group // share the same key pattern + each has 4 bytes identifying particular // vector in the group. Since we need the same chunks end up on the same // node in the group, we need to skip the 4 bytes containing vec# from the // hash. Apart from that, we keep the previous mode of operation, so that // ByteVec would have first 64MB distributed around cloud randomly and then // go round-robin in 64MB chunks. if (_kb[0] == DVEC) { // Homed Chunk? if (_kb[1] != -1) throw H2O.unimpl(); // For round-robin on Chunks in the following pattern: // 1 Chunk-per-node, until all nodes have 1 chunk (max parallelism). // Then 2 chunks-per-node, once around, then 4, then 8, then 16. // Getting several chunks-in-a-row on a single Node means that stencil // calculations that step off the end of one chunk into the next won't // force a chunk local - replicating the data. If all chunks round robin // exactly, then any stencil calc will double the cached volume of data // (every node will have it's own chunk, plus a cached next-chunk). // Above 16-chunks-in-a-row we hit diminishing returns. int cidx = UnsafeUtils.get4(_kb, 1 + 1 + 4); // Chunk index int x = cidx / hsz; // Multiples of cluster size // 0 -> 1st trip around the cluster; nidx= (cidx- 0*hsz)>>0 // 1,2 -> 2nd & 3rd trip; allocate in pairs: nidx= (cidx- 1*hsz)>>1 // 3,4,5,6 -> next 4 rounds; allocate in quads: nidx= (cidx- 3*hsz)>>2 // 7-14 -> next 8 rounds in octets: nidx= (cidx- 7*hsz)>>3 // 15+ -> remaining rounds in groups of 16: nidx= (cidx-15*hsz)>>4 int z = x == 0 ? 0 : (x <= 2 ? 1 : (x <= 6 ? 2 : (x <= 14 ? 3 : 4))); int nidx = (cidx - ((1 << z) - 1) * hsz) >> z; return ((nidx + repl) & 0x7FFFFFFF) % hsz; } // Easy Cheesy Stupid: return ((_hash + repl) & 0x7FFFFFFF) % hsz; }
public Key localModelInfoKey(H2ONode node) { return Key.make( get_params()._model_id + ".node" + node.index(), (byte) 1 /*replica factor*/, (byte) 31 /*hidden user-key*/, true, node); }
// Make a Key which is homed to specific nodes. public static Key make(byte[] kb, byte rf, byte systemType, H2ONode... replicas) { // no more than 3 replicas allowed to be stored in the key assert 0 <= replicas.length && replicas.length <= 3; assert systemType < 32; // only system keys allowed // Key byte layout is: // 0 - systemType, from 0-31 // 1 - replica-count, plus up to 3 bits for ip4 vs ip6 // 2-n - zero, one, two or 3 IP4 (4+2 bytes) or IP6 (16+2 bytes) addresses // 2-5- 4 bytes of chunk#, or -1 for masters // n+ - repeat of the original kb AutoBuffer ab = new AutoBuffer(); ab.put1(systemType).put1(replicas.length); for (H2ONode h2o : replicas) h2o.write(ab); ab.put4(-1); ab.putA1(kb, kb.length); return make(Arrays.copyOf(ab.buf(), ab.position()), rf); }
private void sendAck() { // Send results back DTask dt, origDt = _dt; // _dt can go null the instant it is send over wire assert origDt != null; // Freed after completion while ((dt = _dt) != null) { // Retry loop for broken TCP sends AutoBuffer ab = null; try { // Start the ACK with results back to client. If the client is // asking for a class/id mapping (or any job running at FETCH_ACK // priority) then return a udp.fetchack byte instead of a udp.ack. // The receiver thread then knows to handle the mapping at the higher // priority. UDP.udp udp = dt.priority() == H2O.FETCH_ACK_PRIORITY ? UDP.udp.fetchack : UDP.udp.ack; ab = new AutoBuffer(_client, udp._prior).putTask(udp, _tsknum).put1(SERVER_UDP_SEND); assert ab.position() == 1 + 2 + 4 + 1; dt.write(ab); // Write the DTask - could be very large write dt._repliedTcp = ab.hasTCP(); // Resends do not need to repeat TCP result ab.close(); // Then close; send final byte _computedAndReplied = true; // After the final handshake, set computed+replied bit break; // Break out of retry loop } catch (AutoBuffer.AutoBufferException e) { if (!_client._heartbeat._client) // Report on servers only; clients allowed to be flaky Log.info( "IOException during ACK, " + e._ioe.getMessage() + ", t#" + _tsknum + " AB=" + ab + ", waiting and retrying..."); ab.drainClose(); if (_client._heartbeat._client) // Dead client will not accept a TCP ACK response? this.CAS_DT(dt, null); // cancel the ACK try { Thread.sleep(100); } catch (InterruptedException ignore) { } } catch (Exception e) { // Custom serializer just barfed? Log.err(e); // Log custom serializer exception ab.drainClose(); } } // end of while(true) if (dt == null) Log.info( "Cancelled remote task#" + _tsknum + " " + origDt.getClass() + " to " + _client + " has been cancelled by remote"); else { if (dt instanceof MRTask && dt.logVerbose()) Log.debug("Done remote task#" + _tsknum + " " + dt.getClass() + " to " + _client); _client.record_task_answer(this); // Setup for retrying Ack & AckAck, if not canceled } }
// Do the remote execution in a F/J thread & send a reply packet. // Caller must call 'tryComplete'. private static AutoBuffer remexec(DTask dt, H2ONode client, int task, AutoBuffer abold) { abold.close(); // Closing the old guy, returning a new guy // Now compute on it! dt.invoke(client); // Send results back AutoBuffer ab = new AutoBuffer(client).putTask(UDP.udp.ack, task).put1(SERVER_UDP_SEND); dt.write(ab); // Write the DTask dt._repliedTcp = ab.hasTCP(); // Resends do not need to repeat TCP result // Install answer so retries get this very answer client.record_task_answer(task, dt); return ab; }
// Attempt to cancel job @Override public final boolean cancel(boolean mayInterruptIfRunning) { boolean did = false; synchronized (this) { // Install the answer under lock if (!isCancelled()) { did = true; // Did cancel (was not cancelled already) _target.taskRemove(_tasknum); _target = null; // Flag as canceled // UDPTimeOutThread.PENDING.remove(this); } notifyAll(); // notify in any case } return did; }
// The Run Method. // Started by main() on a single thread, this code publishes Cloud membership // to the Cloud once a second (across all members). If anybody disagrees // with the membership Heartbeat, they will start a round of Paxos group // discovery. public void run() { MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); ObjectName os; try { os = new ObjectName("java.lang:type=OperatingSystem"); } catch (MalformedObjectNameException e) { throw Log.errRTExcept(e); } Thread.currentThread().setPriority(Thread.MAX_PRIORITY); int counter = 0; while (true) { // Once per second, for the entire cloud a Node will multi-cast publish // itself, so other unrelated Clouds discover each other and form up. try { Thread.sleep(SLEEP); } // Only once-sec per entire Cloud catch (InterruptedException e) { } // Update the interesting health self-info for publication also H2O cloud = H2O.CLOUD; HeartBeat hb = H2O.SELF._heartbeat; hb._hb_version = HB_VERSION++; hb._jvm_boot_msec = TimeLine.JVM_BOOT_MSEC; final Runtime run = Runtime.getRuntime(); hb.set_free_mem(run.freeMemory()); hb.set_max_mem(run.maxMemory()); hb.set_tot_mem(run.totalMemory()); hb._keys = (H2O.STORE.size()); hb.set_valsz(myHisto.histo(false)._cached); hb._num_cpus = (char) run.availableProcessors(); if (counter % 300 == 2) { // run mini-benchmark every 5 mins hb._gflops = Linpack.run(); hb._membw = MemoryBandwidth.run(); } Object load = null; try { load = mbs.getAttribute(os, "SystemLoadAverage"); } catch (Exception e) { // Ignore, data probably not available on this VM } hb._system_load_average = load instanceof Double ? ((Double) load).floatValue() : 0; int rpcs = 0; for (H2ONode h2o : cloud._memary) rpcs += h2o.taskSize(); hb._rpcs = (char) rpcs; // Scrape F/J pool counts hb._fjthrds = new short[H2O.MAX_PRIORITY + 1]; hb._fjqueue = new short[H2O.MAX_PRIORITY + 1]; for (int i = 0; i < hb._fjthrds.length; i++) { hb._fjthrds[i] = (short) H2O.getWrkThrPoolSize(i); hb._fjqueue[i] = (short) H2O.getWrkQueueSize(i); } hb._tcps_active = (char) H2ONode.TCPS.get(); // get the usable and total disk storage for the partition where the // persistent KV pairs are stored hb.set_free_disk(Persist.getIce().getUsableSpace()); hb.set_max_disk(Persist.getIce().getTotalSpace()); // get cpu utilization for the system and for this process. (linux only.) LinuxProcFileReader lpfr = new LinuxProcFileReader(); lpfr.read(); if (lpfr.valid()) { hb._system_idle_ticks = lpfr.getSystemIdleTicks(); hb._system_total_ticks = lpfr.getSystemTotalTicks(); hb._process_total_ticks = lpfr.getProcessTotalTicks(); hb._process_num_open_fds = lpfr.getProcessNumOpenFds(); } else { hb._system_idle_ticks = -1; hb._system_total_ticks = -1; hb._process_total_ticks = -1; hb._process_num_open_fds = -1; } hb._pid = lpfr.getProcessID(); // Announce what Cloud we think we are in. // Publish our health as well. UDPHeartbeat.build_and_multicast(cloud, hb); // If we have no internet connection, then the multicast goes // nowhere and we never receive a heartbeat from ourselves! // Fake it now. long now = System.currentTimeMillis(); H2O.SELF._last_heard_from = now; // Look for napping Nodes & propose removing from Cloud for (H2ONode h2o : cloud._memary) { long delta = now - h2o._last_heard_from; if (delta > SUSPECT) { // We suspect this Node has taken a dirt nap if (!h2o._announcedLostContact) { Paxos.print("hart: announce suspect node", cloud._memary, h2o.toString()); h2o._announcedLostContact = true; } } else if (h2o._announcedLostContact) { Paxos.print("hart: regained contact with node", cloud._memary, h2o.toString()); h2o._announcedLostContact = false; } } counter++; } }
public synchronized RPC<V> call() { // Any Completer will not be carried over to remote; add it to the RPC call // so completion is signaled after the remote comes back. CountedCompleter cc = _dt.getCompleter(); if (cc != null) handleCompleter(cc); // If running on self, just submit to queues & do locally if (_target == H2O.SELF) return handleLocal(); // Keep a global record, for awhile if (_target != null) _target.taskPut(_tasknum, this); try { if (_nack) return this; // Racing Nack rechecked under lock; no need to send retry // We could be racing timeouts-vs-replies. Blow off timeout if we have an answer. if (isDone()) { if (_target != null) _target.taskRemove(_tasknum); return this; } // Default strategy: (re)fire the packet and (re)start the timeout. We // "count" exactly 1 failure: just whether or not we shipped via TCP ever // once. After that we fearlessly (re)send UDP-sized packets until the // server replies. // Pack classloader/class & the instance data into the outgoing // AutoBuffer. If it fits in a single UDP packet, ship it. If not, // finish off the current AutoBuffer (which is now going TCP style), and // make a new UDP-sized packet. On a re-send of a TCP-sized hunk, just // send the basic UDP control packet. if (!_sentTcp) { while (true) { // Retry loop for broken TCP sends AutoBuffer ab = new AutoBuffer(_target, _dt.priority()); try { final boolean t; int offset = ab.position(); ab.putTask(UDP.udp.exec, _tasknum).put1(CLIENT_UDP_SEND); ab.put(_dt); t = ab.hasTCP(); assert sz_check(ab) : "Resend of " + _dt.getClass() + " changes size from " + _size + " to " + ab.size() + " for task#" + _tasknum; ab.close(); // Then close; send final byte _sentTcp = t; // Set after close (and any other possible fail) break; // Break out of retry loop } catch (AutoBuffer.AutoBufferException e) { Log.info( "IOException during RPC call: " + e._ioe.getMessage() + ", AB=" + ab + ", for task#" + _tasknum + ", waiting and retrying..."); ab.drainClose(); try { Thread.sleep(500); } catch (InterruptedException ignore) { } } } // end of while(true) } else { // Else it was sent via TCP in a prior attempt, and we've timed out. // This means the caller's ACK/answer probably got dropped and we need // him to resend it (or else the caller is still processing our // request). Send a UDP reminder - but with the CLIENT_TCP_SEND flag // instead of the UDP send, and no DTask (since it previously went via // TCP, no need to resend it). AutoBuffer ab = new AutoBuffer(_target, _dt.priority()).putTask(UDP.udp.exec, _tasknum); ab.put1(CLIENT_TCP_SEND).close(); } // Double retry until we exceed existing age. This is the time to delay // until we try again. Note that we come here immediately on creation, // so the first doubling happens before anybody does any waiting. Also // note the generous 5sec cap: ping at least every 5 sec. _retry += (_retry < MAX_TIMEOUT) ? _retry : MAX_TIMEOUT; // Put self on the "TBD" list of tasks awaiting Timeout. // So: dont really 'forget' but remember me in a little bit. // UDPTimeOutThread.PENDING.put(_tasknum, this); return this; } catch (Throwable t) { t.printStackTrace(); throw Log.throwErr(t); } }
RPC<V> setTaskNum() { assert _tasknum == 0; _tasknum = _target.nextTaskNum(); return this; }