Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E2D3510658 for ; Thu, 5 Mar 2015 09:06:34 +0000 (UTC) Received: (qmail 43605 invoked by uid 500); 5 Mar 2015 09:05:36 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 43572 invoked by uid 500); 5 Mar 2015 09:05:36 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 43563 invoked by uid 99); 5 Mar 2015 09:05:36 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 05 Mar 2015 09:05:36 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Thu, 05 Mar 2015 09:05:22 +0000 Received: (qmail 42449 invoked by uid 99); 5 Mar 2015 09:05:02 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 05 Mar 2015 09:05:02 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 87A7DE1099; Thu, 5 Mar 2015 09:05:01 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: yzhdanov@apache.org To: commits@ignite.incubator.apache.org Date: Thu, 05 Mar 2015 09:05:25 -0000 Message-Id: <3bd7dca1865a449a8bfeb6a40c389570@git.apache.org> In-Reply-To: <56d156eb01174f0b88f954783fd4b143@git.apache.org> References: <56d156eb01174f0b88f954783fd4b143@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [25/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes. X-Virus-Checked: Checked by ClamAV on apache.org http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java new file mode 100644 index 0000000..422d941 --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java @@ -0,0 +1,256 @@ +/* + * 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.ignite.internal.processors.hadoop.shuffle; + +import org.apache.ignite.*; +import org.apache.ignite.cluster.*; +import org.apache.ignite.internal.*; +import org.apache.ignite.internal.processors.hadoop.*; +import org.apache.ignite.internal.processors.hadoop.message.*; +import org.apache.ignite.internal.util.future.*; +import org.apache.ignite.internal.util.lang.*; +import org.apache.ignite.internal.util.offheap.unsafe.*; +import org.apache.ignite.internal.util.typedef.*; +import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.lang.*; + +import java.util.*; +import java.util.concurrent.*; + +/** + * Shuffle. + */ +public class HadoopShuffle extends HadoopComponent { + /** */ + private final ConcurrentMap> jobs = new ConcurrentHashMap<>(); + + /** */ + protected final GridUnsafeMemory mem = new GridUnsafeMemory(0); + + /** {@inheritDoc} */ + @Override public void start(HadoopContext ctx) throws IgniteCheckedException { + super.start(ctx); + + ctx.kernalContext().io().addUserMessageListener(GridTopic.TOPIC_HADOOP, + new IgniteBiPredicate() { + @Override public boolean apply(UUID nodeId, Object msg) { + return onMessageReceived(nodeId, (HadoopMessage)msg); + } + }); + } + + /** + * Stops shuffle. + * + * @param cancel If should cancel all ongoing activities. + */ + @Override public void stop(boolean cancel) { + for (HadoopShuffleJob job : jobs.values()) { + try { + job.close(); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to close job.", e); + } + } + + jobs.clear(); + } + + /** + * Creates new shuffle job. + * + * @param jobId Job ID. + * @return Created shuffle job. + * @throws IgniteCheckedException If job creation failed. + */ + private HadoopShuffleJob newJob(HadoopJobId jobId) throws IgniteCheckedException { + HadoopMapReducePlan plan = ctx.jobTracker().plan(jobId); + + HadoopShuffleJob job = new HadoopShuffleJob<>(ctx.localNodeId(), log, + ctx.jobTracker().job(jobId, null), mem, plan.reducers(), plan.reducers(ctx.localNodeId())); + + UUID[] rdcAddrs = new UUID[plan.reducers()]; + + for (int i = 0; i < rdcAddrs.length; i++) { + UUID nodeId = plan.nodeForReducer(i); + + assert nodeId != null : "Plan is missing node for reducer [plan=" + plan + ", rdc=" + i + ']'; + + rdcAddrs[i] = nodeId; + } + + boolean init = job.initializeReduceAddresses(rdcAddrs); + + assert init; + + return job; + } + + /** + * @param nodeId Node ID to send message to. + * @param msg Message to send. + * @throws IgniteCheckedException If send failed. + */ + private void send0(UUID nodeId, Object msg) throws IgniteCheckedException { + ClusterNode node = ctx.kernalContext().discovery().node(nodeId); + + ctx.kernalContext().io().sendUserMessage(F.asList(node), msg, GridTopic.TOPIC_HADOOP, false, 0); + } + + /** + * @param jobId Task info. + * @return Shuffle job. + */ + private HadoopShuffleJob job(HadoopJobId jobId) throws IgniteCheckedException { + HadoopShuffleJob res = jobs.get(jobId); + + if (res == null) { + res = newJob(jobId); + + HadoopShuffleJob old = jobs.putIfAbsent(jobId, res); + + if (old != null) { + res.close(); + + res = old; + } + else if (res.reducersInitialized()) + startSending(res); + } + + return res; + } + + /** + * Starts message sending thread. + * + * @param shuffleJob Job to start sending for. + */ + private void startSending(HadoopShuffleJob shuffleJob) { + shuffleJob.startSending(ctx.kernalContext().gridName(), + new IgniteInClosure2X() { + @Override public void applyx(UUID dest, HadoopShuffleMessage msg) throws IgniteCheckedException { + send0(dest, msg); + } + } + ); + } + + /** + * Message received callback. + * + * @param src Sender node ID. + * @param msg Received message. + * @return {@code True}. + */ + public boolean onMessageReceived(UUID src, HadoopMessage msg) { + if (msg instanceof HadoopShuffleMessage) { + HadoopShuffleMessage m = (HadoopShuffleMessage)msg; + + try { + job(m.jobId()).onShuffleMessage(m); + } + catch (IgniteCheckedException e) { + U.error(log, "Message handling failed.", e); + } + + try { + // Reply with ack. + send0(src, new HadoopShuffleAck(m.id(), m.jobId())); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to reply back to shuffle message sender [snd=" + src + ", msg=" + msg + ']', e); + } + } + else if (msg instanceof HadoopShuffleAck) { + HadoopShuffleAck m = (HadoopShuffleAck)msg; + + try { + job(m.jobId()).onShuffleAck(m); + } + catch (IgniteCheckedException e) { + U.error(log, "Message handling failed.", e); + } + } + else + throw new IllegalStateException("Unknown message type received to Hadoop shuffle [src=" + src + + ", msg=" + msg + ']'); + + return true; + } + + /** + * @param taskCtx Task info. + * @return Output. + */ + public HadoopTaskOutput output(HadoopTaskContext taskCtx) throws IgniteCheckedException { + return job(taskCtx.taskInfo().jobId()).output(taskCtx); + } + + /** + * @param taskCtx Task info. + * @return Input. + */ + public HadoopTaskInput input(HadoopTaskContext taskCtx) throws IgniteCheckedException { + return job(taskCtx.taskInfo().jobId()).input(taskCtx); + } + + /** + * @param jobId Job id. + */ + public void jobFinished(HadoopJobId jobId) { + HadoopShuffleJob job = jobs.remove(jobId); + + if (job != null) { + try { + job.close(); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to close job: " + jobId, e); + } + } + } + + /** + * Flushes all the outputs for the given job to remote nodes. + * + * @param jobId Job ID. + * @return Future. + */ + public IgniteInternalFuture flush(HadoopJobId jobId) { + HadoopShuffleJob job = jobs.get(jobId); + + if (job == null) + return new GridFinishedFutureEx<>(); + + try { + return job.flush(); + } + catch (IgniteCheckedException e) { + return new GridFinishedFutureEx<>(e); + } + } + + /** + * @return Memory. + */ + public GridUnsafeMemory memory() { + return mem; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleAck.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleAck.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleAck.java new file mode 100644 index 0000000..49cbd65 --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleAck.java @@ -0,0 +1,91 @@ +/* + * 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.ignite.internal.processors.hadoop.shuffle; + +import org.apache.ignite.internal.processors.hadoop.*; +import org.apache.ignite.internal.processors.hadoop.message.*; +import org.apache.ignite.internal.util.tostring.*; +import org.apache.ignite.internal.util.typedef.internal.*; + +import java.io.*; + +/** + * Acknowledgement message. + */ +public class HadoopShuffleAck implements HadoopMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + @GridToStringInclude + private long msgId; + + /** */ + @GridToStringInclude + private HadoopJobId jobId; + + /** + * + */ + public HadoopShuffleAck() { + // No-op. + } + + /** + * @param msgId Message ID. + */ + public HadoopShuffleAck(long msgId, HadoopJobId jobId) { + assert jobId != null; + + this.msgId = msgId; + this.jobId = jobId; + } + + /** + * @return Message ID. + */ + public long id() { + return msgId; + } + + /** + * @return Job ID. + */ + public HadoopJobId jobId() { + return jobId; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + jobId.writeExternal(out); + out.writeLong(msgId); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + jobId = new HadoopJobId(); + + jobId.readExternal(in); + msgId = in.readLong(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(HadoopShuffleAck.class, this); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java new file mode 100644 index 0000000..7ae52df --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java @@ -0,0 +1,593 @@ +/* + * 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.ignite.internal.processors.hadoop.shuffle; + +import org.apache.ignite.*; +import org.apache.ignite.internal.*; +import org.apache.ignite.internal.processors.hadoop.*; +import org.apache.ignite.internal.processors.hadoop.counter.*; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.*; +import org.apache.ignite.internal.util.future.*; +import org.apache.ignite.internal.util.io.*; +import org.apache.ignite.internal.util.lang.*; +import org.apache.ignite.internal.util.offheap.unsafe.*; +import org.apache.ignite.internal.util.typedef.*; +import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.internal.util.worker.*; +import org.apache.ignite.lang.*; +import org.apache.ignite.thread.*; + +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; + +import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.*; +import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*; + +/** + * Shuffle job. + */ +public class HadoopShuffleJob implements AutoCloseable { + /** */ + private static final int MSG_BUF_SIZE = 128 * 1024; + + /** */ + private final HadoopJob job; + + /** */ + private final GridUnsafeMemory mem; + + /** */ + private final boolean needPartitioner; + + /** Collection of task contexts for each reduce task. */ + private final Map reducersCtx = new HashMap<>(); + + /** Reducers addresses. */ + private T[] reduceAddrs; + + /** Local reducers address. */ + private final T locReduceAddr; + + /** */ + private final HadoopShuffleMessage[] msgs; + + /** */ + private final AtomicReferenceArray maps; + + /** */ + private volatile IgniteInClosure2X io; + + /** */ + protected ConcurrentMap>> sentMsgs = + new ConcurrentHashMap<>(); + + /** */ + private volatile GridWorker snd; + + /** Latch for remote addresses waiting. */ + private final CountDownLatch ioInitLatch = new CountDownLatch(1); + + /** Finished flag. Set on flush or close. */ + private volatile boolean flushed; + + /** */ + private final IgniteLogger log; + + /** + * @param locReduceAddr Local reducer address. + * @param log Logger. + * @param job Job. + * @param mem Memory. + * @param totalReducerCnt Amount of reducers in the Job. + * @param locReducers Reducers will work on current node. + * @throws IgniteCheckedException If error. + */ + public HadoopShuffleJob(T locReduceAddr, IgniteLogger log, HadoopJob job, GridUnsafeMemory mem, + int totalReducerCnt, int[] locReducers) throws IgniteCheckedException { + this.locReduceAddr = locReduceAddr; + this.job = job; + this.mem = mem; + this.log = log.getLogger(HadoopShuffleJob.class); + + if (!F.isEmpty(locReducers)) { + for (int rdc : locReducers) { + HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.REDUCE, job.id(), rdc, 0, null); + + reducersCtx.put(rdc, job.getTaskContext(taskInfo)); + } + } + + needPartitioner = totalReducerCnt > 1; + + maps = new AtomicReferenceArray<>(totalReducerCnt); + msgs = new HadoopShuffleMessage[totalReducerCnt]; + } + + /** + * @param reduceAddrs Addresses of reducers. + * @return {@code True} if addresses were initialized by this call. + */ + public boolean initializeReduceAddresses(T[] reduceAddrs) { + if (this.reduceAddrs == null) { + this.reduceAddrs = reduceAddrs; + + return true; + } + + return false; + } + + /** + * @return {@code True} if reducers addresses were initialized. + */ + public boolean reducersInitialized() { + return reduceAddrs != null; + } + + /** + * @param gridName Grid name. + * @param io IO Closure for sending messages. + */ + @SuppressWarnings("BusyWait") + public void startSending(String gridName, IgniteInClosure2X io) { + assert snd == null; + assert io != null; + + this.io = io; + + if (!flushed) { + snd = new GridWorker(gridName, "hadoop-shuffle-" + job.id(), log) { + @Override protected void body() throws InterruptedException { + try { + while (!isCancelled()) { + Thread.sleep(5); + + collectUpdatesAndSend(false); + } + } + catch (IgniteCheckedException e) { + throw new IllegalStateException(e); + } + } + }; + + new IgniteThread(snd).start(); + } + + ioInitLatch.countDown(); + } + + /** + * @param maps Maps. + * @param idx Index. + * @return Map. + */ + private HadoopMultimap getOrCreateMap(AtomicReferenceArray maps, int idx) { + HadoopMultimap map = maps.get(idx); + + if (map == null) { // Create new map. + map = get(job.info(), SHUFFLE_REDUCER_NO_SORTING, false) ? + new HadoopConcurrentHashMultimap(job.info(), mem, get(job.info(), PARTITION_HASHMAP_SIZE, 8 * 1024)): + new HadoopSkipList(job.info(), mem); + + if (!maps.compareAndSet(idx, null, map)) { + map.close(); + + return maps.get(idx); + } + } + + return map; + } + + /** + * @param msg Message. + * @throws IgniteCheckedException Exception. + */ + public void onShuffleMessage(HadoopShuffleMessage msg) throws IgniteCheckedException { + assert msg.buffer() != null; + assert msg.offset() > 0; + + HadoopTaskContext taskCtx = reducersCtx.get(msg.reducer()); + + HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(taskCtx.counters(), null); + + perfCntr.onShuffleMessage(msg.reducer(), U.currentTimeMillis()); + + HadoopMultimap map = getOrCreateMap(maps, msg.reducer()); + + // Add data from message to the map. + try (HadoopMultimap.Adder adder = map.startAdding(taskCtx)) { + final GridUnsafeDataInput dataInput = new GridUnsafeDataInput(); + final UnsafeValue val = new UnsafeValue(msg.buffer()); + + msg.visit(new HadoopShuffleMessage.Visitor() { + /** */ + private HadoopMultimap.Key key; + + @Override public void onKey(byte[] buf, int off, int len) throws IgniteCheckedException { + dataInput.bytes(buf, off, off + len); + + key = adder.addKey(dataInput, key); + } + + @Override public void onValue(byte[] buf, int off, int len) { + val.off = off; + val.size = len; + + key.add(val); + } + }); + } + } + + /** + * @param ack Shuffle ack. + */ + @SuppressWarnings("ConstantConditions") + public void onShuffleAck(HadoopShuffleAck ack) { + IgniteBiTuple> tup = sentMsgs.get(ack.id()); + + if (tup != null) + tup.get2().onDone(); + else + log.warning("Received shuffle ack for not registered shuffle id: " + ack); + } + + /** + * Unsafe value. + */ + private static class UnsafeValue implements HadoopMultimap.Value { + /** */ + private final byte[] buf; + + /** */ + private int off; + + /** */ + private int size; + + /** + * @param buf Buffer. + */ + private UnsafeValue(byte[] buf) { + assert buf != null; + + this.buf = buf; + } + + /** */ + @Override public int size() { + return size; + } + + /** */ + @Override public void copyTo(long ptr) { + UNSAFE.copyMemory(buf, BYTE_ARR_OFF + off, null, ptr, size); + } + } + + /** + * Sends map updates to remote reducers. + */ + private void collectUpdatesAndSend(boolean flush) throws IgniteCheckedException { + for (int i = 0; i < maps.length(); i++) { + HadoopMultimap map = maps.get(i); + + if (map == null || locReduceAddr.equals(reduceAddrs[i])) + continue; // Skip empty map and local node. + + if (msgs[i] == null) + msgs[i] = new HadoopShuffleMessage(job.id(), i, MSG_BUF_SIZE); + + final int idx = i; + + map.visit(false, new HadoopMultimap.Visitor() { + /** */ + private long keyPtr; + + /** */ + private int keySize; + + /** */ + private boolean keyAdded; + + /** {@inheritDoc} */ + @Override public void onKey(long keyPtr, int keySize) { + this.keyPtr = keyPtr; + this.keySize = keySize; + + keyAdded = false; + } + + private boolean tryAdd(long valPtr, int valSize) { + HadoopShuffleMessage msg = msgs[idx]; + + if (!keyAdded) { // Add key and value. + int size = keySize + valSize; + + if (!msg.available(size, false)) + return false; + + msg.addKey(keyPtr, keySize); + msg.addValue(valPtr, valSize); + + keyAdded = true; + + return true; + } + + if (!msg.available(valSize, true)) + return false; + + msg.addValue(valPtr, valSize); + + return true; + } + + /** {@inheritDoc} */ + @Override public void onValue(long valPtr, int valSize) { + if (tryAdd(valPtr, valSize)) + return; + + send(idx, keySize + valSize); + + keyAdded = false; + + if (!tryAdd(valPtr, valSize)) + throw new IllegalStateException(); + } + }); + + if (flush && msgs[i].offset() != 0) + send(i, 0); + } + } + + /** + * @param idx Index of message. + * @param newBufMinSize Min new buffer size. + */ + private void send(final int idx, int newBufMinSize) { + final GridFutureAdapterEx fut = new GridFutureAdapterEx<>(); + + HadoopShuffleMessage msg = msgs[idx]; + + final long msgId = msg.id(); + + IgniteBiTuple> old = sentMsgs.putIfAbsent(msgId, + new IgniteBiTuple>(msg, fut)); + + assert old == null; + + try { + io.apply(reduceAddrs[idx], msg); + } + catch (GridClosureException e) { + fut.onDone(U.unwrap(e)); + } + + fut.listenAsync(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture f) { + try { + f.get(); + + // Clean up the future from map only if there was no exception. + // Otherwise flush() should fail. + sentMsgs.remove(msgId); + } + catch (IgniteCheckedException e) { + log.error("Failed to send message.", e); + } + } + }); + + msgs[idx] = newBufMinSize == 0 ? null : new HadoopShuffleMessage(job.id(), idx, + Math.max(MSG_BUF_SIZE, newBufMinSize)); + } + + /** {@inheritDoc} */ + @Override public void close() throws IgniteCheckedException { + if (snd != null) { + snd.cancel(); + + try { + snd.join(); + } + catch (InterruptedException e) { + throw new IgniteInterruptedCheckedException(e); + } + } + + close(maps); + } + + /** + * @param maps Maps. + */ + private void close(AtomicReferenceArray maps) { + for (int i = 0; i < maps.length(); i++) { + HadoopMultimap map = maps.get(i); + + if (map != null) + map.close(); + } + } + + /** + * @return Future. + */ + @SuppressWarnings("unchecked") + public IgniteInternalFuture flush() throws IgniteCheckedException { + if (log.isDebugEnabled()) + log.debug("Flushing job " + job.id() + " on address " + locReduceAddr); + + flushed = true; + + if (maps.length() == 0) + return new GridFinishedFutureEx<>(); + + U.await(ioInitLatch); + + GridWorker snd0 = snd; + + if (snd0 != null) { + if (log.isDebugEnabled()) + log.debug("Cancelling sender thread."); + + snd0.cancel(); + + try { + snd0.join(); + + if (log.isDebugEnabled()) + log.debug("Finished waiting for sending thread to complete on shuffle job flush: " + job.id()); + } + catch (InterruptedException e) { + throw new IgniteInterruptedCheckedException(e); + } + } + + collectUpdatesAndSend(true); // With flush. + + if (log.isDebugEnabled()) + log.debug("Finished sending collected updates to remote reducers: " + job.id()); + + GridCompoundFuture fut = new GridCompoundFuture<>(); + + for (IgniteBiTuple> tup : sentMsgs.values()) + fut.add(tup.get2()); + + fut.markInitialized(); + + if (log.isDebugEnabled()) + log.debug("Collected futures to compound futures for flush: " + sentMsgs.size()); + + return fut; + } + + /** + * @param taskCtx Task context. + * @return Output. + * @throws IgniteCheckedException If failed. + */ + public HadoopTaskOutput output(HadoopTaskContext taskCtx) throws IgniteCheckedException { + switch (taskCtx.taskInfo().type()) { + case MAP: + assert !job.info().hasCombiner() : "The output creation is allowed if combiner has not been defined."; + + case COMBINE: + return new PartitionedOutput(taskCtx); + + default: + throw new IllegalStateException("Illegal type: " + taskCtx.taskInfo().type()); + } + } + + /** + * @param taskCtx Task context. + * @return Input. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("unchecked") + public HadoopTaskInput input(HadoopTaskContext taskCtx) throws IgniteCheckedException { + switch (taskCtx.taskInfo().type()) { + case REDUCE: + int reducer = taskCtx.taskInfo().taskNumber(); + + HadoopMultimap m = maps.get(reducer); + + if (m != null) + return m.input(taskCtx); + + return new HadoopTaskInput() { // Empty input. + @Override public boolean next() { + return false; + } + + @Override public Object key() { + throw new IllegalStateException(); + } + + @Override public Iterator values() { + throw new IllegalStateException(); + } + + @Override public void close() { + // No-op. + } + }; + + default: + throw new IllegalStateException("Illegal type: " + taskCtx.taskInfo().type()); + } + } + + /** + * Partitioned output. + */ + private class PartitionedOutput implements HadoopTaskOutput { + /** */ + private final HadoopTaskOutput[] adders = new HadoopTaskOutput[maps.length()]; + + /** */ + private HadoopPartitioner partitioner; + + /** */ + private final HadoopTaskContext taskCtx; + + /** + * Constructor. + * @param taskCtx Task context. + */ + private PartitionedOutput(HadoopTaskContext taskCtx) throws IgniteCheckedException { + this.taskCtx = taskCtx; + + if (needPartitioner) + partitioner = taskCtx.partitioner(); + } + + /** {@inheritDoc} */ + @Override public void write(Object key, Object val) throws IgniteCheckedException { + int part = 0; + + if (partitioner != null) { + part = partitioner.partition(key, val, adders.length); + + if (part < 0 || part >= adders.length) + throw new IgniteCheckedException("Invalid partition: " + part); + } + + HadoopTaskOutput out = adders[part]; + + if (out == null) + adders[part] = out = getOrCreateMap(maps, part).startAdding(taskCtx); + + out.write(key, val); + } + + /** {@inheritDoc} */ + @Override public void close() throws IgniteCheckedException { + for (HadoopTaskOutput adder : adders) { + if (adder != null) + adder.close(); + } + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java new file mode 100644 index 0000000..c350552a --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java @@ -0,0 +1,241 @@ +/* + * 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.ignite.internal.processors.hadoop.shuffle; + +import org.apache.ignite.*; +import org.apache.ignite.internal.processors.hadoop.*; +import org.apache.ignite.internal.processors.hadoop.message.*; +import org.apache.ignite.internal.util.tostring.*; +import org.apache.ignite.internal.util.typedef.internal.*; + +import java.io.*; +import java.util.concurrent.atomic.*; + +import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*; + +/** + * Shuffle message. + */ +public class HadoopShuffleMessage implements HadoopMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private static final AtomicLong ids = new AtomicLong(); + + /** */ + private static final byte MARKER_KEY = (byte)17; + + /** */ + private static final byte MARKER_VALUE = (byte)31; + + /** */ + @GridToStringInclude + private long msgId; + + /** */ + @GridToStringInclude + private HadoopJobId jobId; + + /** */ + @GridToStringInclude + private int reducer; + + /** */ + private byte[] buf; + + /** */ + @GridToStringInclude + private int off; + + /** + * + */ + public HadoopShuffleMessage() { + // No-op. + } + + /** + * @param size Size. + */ + public HadoopShuffleMessage(HadoopJobId jobId, int reducer, int size) { + assert jobId != null; + + buf = new byte[size]; + + this.jobId = jobId; + this.reducer = reducer; + + msgId = ids.incrementAndGet(); + } + + /** + * @return Message ID. + */ + public long id() { + return msgId; + } + + /** + * @return Job ID. + */ + public HadoopJobId jobId() { + return jobId; + } + + /** + * @return Reducer. + */ + public int reducer() { + return reducer; + } + + /** + * @return Buffer. + */ + public byte[] buffer() { + return buf; + } + + /** + * @return Offset. + */ + public int offset() { + return off; + } + + /** + * @param size Size. + * @param valOnly Only value wll be added. + * @return {@code true} If this message can fit additional data of this size + */ + public boolean available(int size, boolean valOnly) { + size += valOnly ? 5 : 10; + + if (off + size > buf.length) { + if (off == 0) { // Resize if requested size is too big. + buf = new byte[size]; + + return true; + } + + return false; + } + + return true; + } + + /** + * @param keyPtr Key pointer. + * @param keySize Key size. + */ + public void addKey(long keyPtr, int keySize) { + add(MARKER_KEY, keyPtr, keySize); + } + + /** + * @param valPtr Value pointer. + * @param valSize Value size. + */ + public void addValue(long valPtr, int valSize) { + add(MARKER_VALUE, valPtr, valSize); + } + + /** + * @param marker Marker. + * @param ptr Pointer. + * @param size Size. + */ + private void add(byte marker, long ptr, int size) { + buf[off++] = marker; + + UNSAFE.putInt(buf, BYTE_ARR_OFF + off, size); + + off += 4; + + UNSAFE.copyMemory(null, ptr, buf, BYTE_ARR_OFF + off, size); + + off += size; + } + + /** + * @param v Visitor. + */ + public void visit(Visitor v) throws IgniteCheckedException { + for (int i = 0; i < off;) { + byte marker = buf[i++]; + + int size = UNSAFE.getInt(buf, BYTE_ARR_OFF + i); + + i += 4; + + if (marker == MARKER_VALUE) + v.onValue(buf, i, size); + else if (marker == MARKER_KEY) + v.onKey(buf, i, size); + else + throw new IllegalStateException(); + + i += size; + } + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + jobId.writeExternal(out); + out.writeLong(msgId); + out.writeInt(reducer); + out.writeInt(off); + U.writeByteArray(out, buf); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + jobId = new HadoopJobId(); + + jobId.readExternal(in); + msgId = in.readLong(); + reducer = in.readInt(); + off = in.readInt(); + buf = U.readByteArray(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(HadoopShuffleMessage.class, this); + } + + /** + * Visitor. + */ + public static interface Visitor { + /** + * @param buf Buffer. + * @param off Offset. + * @param len Length. + */ + public void onKey(byte[] buf, int off, int len) throws IgniteCheckedException; + + /** + * @param buf Buffer. + * @param off Offset. + * @param len Length. + */ + public void onValue(byte[] buf, int off, int len) throws IgniteCheckedException; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimap.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimap.java deleted file mode 100644 index 32db722..0000000 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimap.java +++ /dev/null @@ -1,611 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; - -import org.apache.ignite.*; -import org.apache.ignite.internal.processors.hadoop.*; -import org.apache.ignite.internal.util.*; -import org.apache.ignite.internal.util.offheap.unsafe.*; -import org.apache.ignite.internal.util.typedef.internal.*; -import org.jetbrains.annotations.*; - -import java.io.*; -import java.util.*; -import java.util.concurrent.*; -import java.util.concurrent.atomic.*; - -/** - * Multimap for map reduce intermediate results. - */ -public class GridHadoopConcurrentHashMultimap extends GridHadoopHashMultimapBase { - /** */ - private final AtomicReference state = new AtomicReference<>(State.READING_WRITING); - - /** */ - private volatile AtomicLongArray oldTbl; - - /** */ - private volatile AtomicLongArray newTbl; - - /** */ - private final AtomicInteger keys = new AtomicInteger(); - - /** */ - private final CopyOnWriteArrayList adders = new CopyOnWriteArrayList<>(); - - /** */ - private final AtomicInteger inputs = new AtomicInteger(); - - /** - * @param jobInfo Job info. - * @param mem Memory. - * @param cap Initial capacity. - */ - public GridHadoopConcurrentHashMultimap(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem, int cap) { - super(jobInfo, mem); - - assert U.isPow2(cap); - - newTbl = oldTbl = new AtomicLongArray(cap); - } - - /** - * @return Number of keys. - */ - public long keys() { - int res = keys.get(); - - for (AdderImpl adder : adders) - res += adder.locKeys.get(); - - return res; - } - - /** - * @return Current table capacity. - */ - @Override public int capacity() { - return oldTbl.length(); - } - - /** - * @return Adder object. - * @param ctx Task context. - */ - @Override public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException { - if (inputs.get() != 0) - throw new IllegalStateException("Active inputs."); - - if (state.get() == State.CLOSING) - throw new IllegalStateException("Closed."); - - return new AdderImpl(ctx); - } - - /** {@inheritDoc} */ - @Override public void close() { - assert inputs.get() == 0 : inputs.get(); - assert adders.isEmpty() : adders.size(); - - state(State.READING_WRITING, State.CLOSING); - - if (keys() == 0) - return; - - super.close(); - } - - /** {@inheritDoc} */ - @Override protected long meta(int idx) { - return oldTbl.get(idx); - } - - /** - * Incrementally visits all the keys and values in the map. - * - * @param ignoreLastVisited Flag indicating that visiting must be started from the beginning. - * @param v Visitor. - * @return {@code false} If visiting was impossible due to rehashing. - */ - @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException { - if (!state.compareAndSet(State.READING_WRITING, State.VISITING)) { - assert state.get() != State.CLOSING; - - return false; // Can not visit while rehashing happens. - } - - AtomicLongArray tbl0 = oldTbl; - - for (int i = 0; i < tbl0.length(); i++) { - long meta = tbl0.get(i); - - while (meta != 0) { - long valPtr = value(meta); - - long lastVisited = ignoreLastVisited ? 0 : lastVisitedValue(meta); - - if (valPtr != lastVisited) { - v.onKey(key(meta), keySize(meta)); - - lastVisitedValue(meta, valPtr); // Set it to the first value in chain. - - do { - v.onValue(valPtr + 12, valueSize(valPtr)); - - valPtr = nextValue(valPtr); - } - while (valPtr != lastVisited); - } - - meta = collision(meta); - } - } - - state(State.VISITING, State.READING_WRITING); - - return true; - } - - /** {@inheritDoc} */ - @Override public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException { - inputs.incrementAndGet(); - - if (!adders.isEmpty()) - throw new IllegalStateException("Active adders."); - - State s = state.get(); - - if (s == State.CLOSING) - throw new IllegalStateException("Closed."); - - assert s != State.REHASHING; - - return new Input(taskCtx) { - @Override public void close() throws IgniteCheckedException { - if (inputs.decrementAndGet() < 0) - throw new IllegalStateException(); - - super.close(); - } - }; - } - - /** - * @param fromTbl Table. - */ - private void rehashIfNeeded(AtomicLongArray fromTbl) { - if (fromTbl.length() == Integer.MAX_VALUE) - return; - - long keys0 = keys(); - - if (keys0 < 3 * (fromTbl.length() >>> 2)) // New size has to be >= than 3/4 of capacity to rehash. - return; - - if (fromTbl != newTbl) // Check if someone else have done the job. - return; - - if (!state.compareAndSet(State.READING_WRITING, State.REHASHING)) { - assert state.get() != State.CLOSING; // Visiting is allowed, but we will not rehash. - - return; - } - - if (fromTbl != newTbl) { // Double check. - state(State.REHASHING, State.READING_WRITING); // Switch back. - - return; - } - - // Calculate new table capacity. - int newLen = fromTbl.length(); - - do { - newLen <<= 1; - } - while (newLen < keys0); - - if (keys0 >= 3 * (newLen >>> 2)) // Still more than 3/4. - newLen <<= 1; - - // This is our target table for rehashing. - AtomicLongArray toTbl = new AtomicLongArray(newLen); - - // Make the new table visible before rehashing. - newTbl = toTbl; - - // Rehash. - int newMask = newLen - 1; - - long failedMeta = 0; - - GridLongList collisions = new GridLongList(16); - - for (int i = 0; i < fromTbl.length(); i++) { // Scan source table. - long meta = fromTbl.get(i); - - assert meta != -1; - - if (meta == 0) { // No entry. - failedMeta = 0; - - if (!fromTbl.compareAndSet(i, 0, -1)) // Mark as moved. - i--; // Retry. - - continue; - } - - do { // Collect all the collisions before the last one failed to nullify or 0. - collisions.add(meta); - - meta = collision(meta); - } - while (meta != failedMeta); - - do { // Go from the last to the first to avoid 'in-flight' state for meta entries. - meta = collisions.remove(); - - int addr = keyHash(meta) & newMask; - - for (;;) { // Move meta entry to the new table. - long toCollision = toTbl.get(addr); - - collision(meta, toCollision); - - if (toTbl.compareAndSet(addr, toCollision, meta)) - break; - } - } - while (!collisions.isEmpty()); - - // Here 'meta' will be a root pointer in old table. - if (!fromTbl.compareAndSet(i, meta, -1)) { // Try to mark as moved. - failedMeta = meta; - - i--; // Retry the same address in table because new keys were added. - } - else - failedMeta = 0; - } - - // Now old and new tables will be the same again. - oldTbl = toTbl; - - state(State.REHASHING, State.READING_WRITING); - } - - /** - * Switch state. - * - * @param oldState Expected state. - * @param newState New state. - */ - private void state(State oldState, State newState) { - if (!state.compareAndSet(oldState, newState)) - throw new IllegalStateException(); - } - - /** - * @param meta Meta pointer. - * @return Value pointer. - */ - @Override protected long value(long meta) { - return mem.readLongVolatile(meta + 16); - } - - /** - * @param meta Meta pointer. - * @param oldValPtr Old value. - * @param newValPtr New value. - * @return {@code true} If succeeded. - */ - private boolean casValue(long meta, long oldValPtr, long newValPtr) { - return mem.casLong(meta + 16, oldValPtr, newValPtr); - } - - /** - * @param meta Meta pointer. - * @return Collision pointer. - */ - @Override protected long collision(long meta) { - return mem.readLongVolatile(meta + 24); - } - - /** - * @param meta Meta pointer. - * @param collision Collision pointer. - */ - @Override protected void collision(long meta, long collision) { - assert meta != collision : meta; - - mem.writeLongVolatile(meta + 24, collision); - } - - /** - * @param meta Meta pointer. - * @return Last visited value pointer. - */ - private long lastVisitedValue(long meta) { - return mem.readLong(meta + 32); - } - - /** - * @param meta Meta pointer. - * @param valPtr Last visited value pointer. - */ - private void lastVisitedValue(long meta, long valPtr) { - mem.writeLong(meta + 32, valPtr); - } - - /** - * Adder. Must not be shared between threads. - */ - private class AdderImpl extends AdderBase { - /** */ - private final Reader keyReader; - - /** */ - private final AtomicInteger locKeys = new AtomicInteger(); - - /** */ - private final Random rnd = new GridRandom(); - - /** - * @param ctx Task context. - * @throws IgniteCheckedException If failed. - */ - private AdderImpl(GridHadoopTaskContext ctx) throws IgniteCheckedException { - super(ctx); - - keyReader = new Reader(keySer); - - rehashIfNeeded(oldTbl); - - adders.add(this); - } - - /** - * @param in Data input. - * @param reuse Reusable key. - * @return Key. - * @throws IgniteCheckedException If failed. - */ - @Override public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException { - KeyImpl k = reuse == null ? new KeyImpl() : (KeyImpl)reuse; - - k.tmpKey = keySer.read(in, k.tmpKey); - - k.meta = add(k.tmpKey, null); - - return k; - } - - /** {@inheritDoc} */ - @Override public void write(Object key, Object val) throws IgniteCheckedException { - A.notNull(val, "val"); - - add(key, val); - } - - /** - * @param tbl Table. - */ - private void incrementKeys(AtomicLongArray tbl) { - locKeys.lazySet(locKeys.get() + 1); - - if (rnd.nextInt(tbl.length()) < 512) - rehashIfNeeded(tbl); - } - - /** - * @param keyHash Key hash. - * @param keySize Key size. - * @param keyPtr Key pointer. - * @param valPtr Value page pointer. - * @param collisionPtr Pointer to meta with hash collision. - * @param lastVisitedVal Last visited value pointer. - * @return Created meta page pointer. - */ - private long createMeta(int keyHash, int keySize, long keyPtr, long valPtr, long collisionPtr, long lastVisitedVal) { - long meta = allocate(40); - - mem.writeInt(meta, keyHash); - mem.writeInt(meta + 4, keySize); - mem.writeLong(meta + 8, keyPtr); - mem.writeLong(meta + 16, valPtr); - mem.writeLong(meta + 24, collisionPtr); - mem.writeLong(meta + 32, lastVisitedVal); - - return meta; - } - - /** - * @param key Key. - * @param val Value. - * @return Updated or created meta page pointer. - * @throws IgniteCheckedException If failed. - */ - private long add(Object key, @Nullable Object val) throws IgniteCheckedException { - AtomicLongArray tbl = oldTbl; - - int keyHash = U.hash(key.hashCode()); - - long newMetaPtr = 0; - - long valPtr = 0; - - if (val != null) { - valPtr = write(12, val, valSer); - int valSize = writtenSize() - 12; - - valueSize(valPtr, valSize); - } - - for (AtomicLongArray old = null;;) { - int addr = keyHash & (tbl.length() - 1); - - long metaPtrRoot = tbl.get(addr); // Read root meta pointer at this address. - - if (metaPtrRoot == -1) { // The cell was already moved by rehashing. - AtomicLongArray n = newTbl; // Need to read newTbl first here. - AtomicLongArray o = oldTbl; - - tbl = tbl == o ? n : o; // Trying to get the oldest table but newer than ours. - - old = null; - - continue; - } - - if (metaPtrRoot != 0) { // Not empty slot. - long metaPtr = metaPtrRoot; - - do { // Scan all the collisions. - if (keyHash(metaPtr) == keyHash && key.equals(keyReader.readKey(metaPtr))) { // Found key. - if (newMetaPtr != 0) // Deallocate new meta if one was allocated. - localDeallocate(key(newMetaPtr)); // Key was allocated first, so rewind to it's pointer. - - if (valPtr != 0) { // Add value if it exists. - long nextValPtr; - - // Values are linked to each other to a stack like structure. - // Replace the last value in meta with ours and link it as next. - do { - nextValPtr = value(metaPtr); - - nextValue(valPtr, nextValPtr); - } - while (!casValue(metaPtr, nextValPtr, valPtr)); - } - - return metaPtr; - } - - metaPtr = collision(metaPtr); - } - while (metaPtr != 0); - - // Here we did not find our key, need to check if it was moved by rehashing to the new table. - if (old == null) { // If the old table already set, then we will just try to update it. - AtomicLongArray n = newTbl; - - if (n != tbl) { // Rehashing happens, try to find the key in new table but preserve the old one. - old = tbl; - tbl = n; - - continue; - } - } - } - - if (old != null) { // We just checked new table but did not find our key as well as in the old one. - tbl = old; // Try to add new key to the old table. - - addr = keyHash & (tbl.length() - 1); - - old = null; - } - - if (newMetaPtr == 0) { // Allocate new meta page. - long keyPtr = write(0, key, keySer); - int keySize = writtenSize(); - - if (valPtr != 0) - nextValue(valPtr, 0); - - newMetaPtr = createMeta(keyHash, keySize, keyPtr, valPtr, metaPtrRoot, 0); - } - else // Update new meta with root pointer collision. - collision(newMetaPtr, metaPtrRoot); - - if (tbl.compareAndSet(addr, metaPtrRoot, newMetaPtr)) { // Try to replace root pointer with new one. - incrementKeys(tbl); - - return newMetaPtr; - } - } - } - - /** {@inheritDoc} */ - @Override public void close() throws IgniteCheckedException { - if (!adders.remove(this)) - throw new IllegalStateException(); - - keys.addAndGet(locKeys.get()); // Here we have race and #keys() method can return wrong result but it is ok. - - super.close(); - } - - /** - * Key. - */ - private class KeyImpl implements Key { - /** */ - private long meta; - - /** */ - private Object tmpKey; - - /** - * @return Meta pointer for the key. - */ - public long address() { - return meta; - } - - /** - * @param val Value. - */ - @Override public void add(Value val) { - int size = val.size(); - - long valPtr = allocate(size + 12); - - val.copyTo(valPtr + 12); - - valueSize(valPtr, size); - - long nextVal; - - do { - nextVal = value(meta); - - nextValue(valPtr, nextVal); - } - while(!casValue(meta, nextVal, valPtr)); - } - } - } - - /** - * Current map state. - */ - private enum State { - /** */ - REHASHING, - - /** */ - VISITING, - - /** */ - READING_WRITING, - - /** */ - CLOSING - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimap.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimap.java deleted file mode 100644 index 2795b77..0000000 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimap.java +++ /dev/null @@ -1,174 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; - -import org.apache.ignite.*; -import org.apache.ignite.internal.processors.hadoop.*; -import org.apache.ignite.internal.util.offheap.unsafe.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -/** - * Hash multimap. - */ -public class GridHadoopHashMultimap extends GridHadoopHashMultimapBase { - /** */ - private long[] tbl; - - /** */ - private int keys; - - /** - * @param jobInfo Job info. - * @param mem Memory. - * @param cap Initial capacity. - */ - public GridHadoopHashMultimap(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem, int cap) { - super(jobInfo, mem); - - assert U.isPow2(cap) : cap; - - tbl = new long[cap]; - } - - /** {@inheritDoc} */ - @Override public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException { - return new AdderImpl(ctx); - } - - /** - * Rehash. - */ - private void rehash() { - long[] newTbl = new long[tbl.length << 1]; - - int newMask = newTbl.length - 1; - - for (long meta : tbl) { - while (meta != 0) { - long collision = collision(meta); - - int idx = keyHash(meta) & newMask; - - collision(meta, newTbl[idx]); - - newTbl[idx] = meta; - - meta = collision; - } - } - - tbl = newTbl; - } - - /** - * @return Keys count. - */ - public int keys() { - return keys; - } - - /** {@inheritDoc} */ - @Override public int capacity() { - return tbl.length; - } - - /** {@inheritDoc} */ - @Override protected long meta(int idx) { - return tbl[idx]; - } - - /** - * Adder. - */ - private class AdderImpl extends AdderBase { - /** */ - private final Reader keyReader; - - /** - * @param ctx Task context. - * @throws IgniteCheckedException If failed. - */ - protected AdderImpl(GridHadoopTaskContext ctx) throws IgniteCheckedException { - super(ctx); - - keyReader = new Reader(keySer); - } - - /** - * @param keyHash Key hash. - * @param keySize Key size. - * @param keyPtr Key pointer. - * @param valPtr Value page pointer. - * @param collisionPtr Pointer to meta with hash collision. - * @return Created meta page pointer. - */ - private long createMeta(int keyHash, int keySize, long keyPtr, long valPtr, long collisionPtr) { - long meta = allocate(32); - - mem.writeInt(meta, keyHash); - mem.writeInt(meta + 4, keySize); - mem.writeLong(meta + 8, keyPtr); - mem.writeLong(meta + 16, valPtr); - mem.writeLong(meta + 24, collisionPtr); - - return meta; - } - - /** {@inheritDoc} */ - @Override public void write(Object key, Object val) throws IgniteCheckedException { - A.notNull(val, "val"); - - int keyHash = U.hash(key.hashCode()); - - // Write value. - long valPtr = write(12, val, valSer); - int valSize = writtenSize() - 12; - - valueSize(valPtr, valSize); - - // Find position in table. - int idx = keyHash & (tbl.length - 1); - - long meta = tbl[idx]; - - // Search for our key in collisions. - while (meta != 0) { - if (keyHash(meta) == keyHash && key.equals(keyReader.readKey(meta))) { // Found key. - nextValue(valPtr, value(meta)); - - value(meta, valPtr); - - return; - } - - meta = collision(meta); - } - - // Write key. - long keyPtr = write(0, key, keySer); - int keySize = writtenSize(); - - nextValue(valPtr, 0); - - tbl[idx] = createMeta(keyHash, keySize, keyPtr, valPtr, tbl[idx]); - - if (++keys > (tbl.length >>> 2) * 3) - rehash(); - } - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimapBase.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimapBase.java deleted file mode 100644 index 92854f1..0000000 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimapBase.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; - -import org.apache.ignite.*; -import org.apache.ignite.internal.processors.hadoop.*; -import org.apache.ignite.internal.util.offheap.unsafe.*; - -import java.util.*; - -/** - * Base class for hash multimaps. - */ -public abstract class GridHadoopHashMultimapBase extends GridHadoopMultimapBase { - /** - * @param jobInfo Job info. - * @param mem Memory. - */ - protected GridHadoopHashMultimapBase(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem) { - super(jobInfo, mem); - } - - /** {@inheritDoc} */ - @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException { - throw new UnsupportedOperationException("visit"); - } - - /** {@inheritDoc} */ - @Override public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException { - return new Input(taskCtx); - } - - /** - * @return Hash table capacity. - */ - public abstract int capacity(); - - /** - * @param idx Index in hash table. - * @return Meta page pointer. - */ - protected abstract long meta(int idx); - - /** - * @param meta Meta pointer. - * @return Key hash. - */ - protected int keyHash(long meta) { - return mem.readInt(meta); - } - - /** - * @param meta Meta pointer. - * @return Key size. - */ - protected int keySize(long meta) { - return mem.readInt(meta + 4); - } - - /** - * @param meta Meta pointer. - * @return Key pointer. - */ - protected long key(long meta) { - return mem.readLong(meta + 8); - } - - /** - * @param meta Meta pointer. - * @return Value pointer. - */ - protected long value(long meta) { - return mem.readLong(meta + 16); - } - /** - * @param meta Meta pointer. - * @param val Value pointer. - */ - protected void value(long meta, long val) { - mem.writeLong(meta + 16, val); - } - - /** - * @param meta Meta pointer. - * @return Collision pointer. - */ - protected long collision(long meta) { - return mem.readLong(meta + 24); - } - - /** - * @param meta Meta pointer. - * @param collision Collision pointer. - */ - protected void collision(long meta, long collision) { - assert meta != collision : meta; - - mem.writeLong(meta + 24, collision); - } - - /** - * Reader for key and value. - */ - protected class Reader extends ReaderBase { - /** - * @param ser Serialization. - */ - protected Reader(GridHadoopSerialization ser) { - super(ser); - } - - /** - * @param meta Meta pointer. - * @return Key. - */ - public Object readKey(long meta) { - assert meta > 0 : meta; - - try { - return read(key(meta), keySize(meta)); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } - } - } - - /** - * Task input. - */ - protected class Input implements GridHadoopTaskInput { - /** */ - private int idx = -1; - - /** */ - private long metaPtr; - - /** */ - private final int cap; - - /** */ - private final Reader keyReader; - - /** */ - private final Reader valReader; - - /** - * @param taskCtx Task context. - * @throws IgniteCheckedException If failed. - */ - public Input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException { - cap = capacity(); - - keyReader = new Reader(taskCtx.keySerialization()); - valReader = new Reader(taskCtx.valueSerialization()); - } - - /** {@inheritDoc} */ - @Override public boolean next() { - if (metaPtr != 0) { - metaPtr = collision(metaPtr); - - if (metaPtr != 0) - return true; - } - - while (++idx < cap) { // Scan table. - metaPtr = meta(idx); - - if (metaPtr != 0) - return true; - } - - return false; - } - - /** {@inheritDoc} */ - @Override public Object key() { - return keyReader.readKey(metaPtr); - } - - /** {@inheritDoc} */ - @Override public Iterator values() { - return new ValueIterator(value(metaPtr), valReader); - } - - /** {@inheritDoc} */ - @Override public void close() throws IgniteCheckedException { - keyReader.close(); - valReader.close(); - } - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimap.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimap.java deleted file mode 100644 index b8eb12c..0000000 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimap.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; - -import org.apache.ignite.*; -import org.apache.ignite.internal.processors.hadoop.*; -import org.jetbrains.annotations.*; - -import java.io.*; - -/** - * Multimap for hadoop intermediate results. - */ -@SuppressWarnings("PublicInnerClass") -public interface GridHadoopMultimap extends AutoCloseable { - /** - * Incrementally visits all the keys and values in the map. - * - * @param ignoreLastVisited Flag indicating that visiting must be started from the beginning. - * @param v Visitor. - * @return {@code false} If visiting was impossible. - */ - public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException; - - /** - * @param ctx Task context. - * @return Adder. - * @throws IgniteCheckedException If failed. - */ - public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException; - - /** - * @param taskCtx Task context. - * @return Task input. - * @throws IgniteCheckedException If failed. - */ - public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) - throws IgniteCheckedException; - - /** {@inheritDoc} */ - @Override public void close(); - - /** - * Adder. - */ - public interface Adder extends GridHadoopTaskOutput { - /** - * @param in Data input. - * @param reuse Reusable key. - * @return Key. - * @throws IgniteCheckedException If failed. - */ - public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException; - } - - /** - * Key add values to. - */ - public interface Key { - /** - * @param val Value. - */ - public void add(Value val); - } - - /** - * Value. - */ - public interface Value { - /** - * @return Size in bytes. - */ - public int size(); - - /** - * @param ptr Pointer. - */ - public void copyTo(long ptr); - } - - /** - * Key and values visitor. - */ - public interface Visitor { - /** - * @param keyPtr Key pointer. - * @param keySize Key size. - */ - public void onKey(long keyPtr, int keySize) throws IgniteCheckedException; - - /** - * @param valPtr Value pointer. - * @param valSize Value size. - */ - public void onValue(long valPtr, int valSize) throws IgniteCheckedException; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimapBase.java ---------------------------------------------------------------------- diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimapBase.java deleted file mode 100644 index 2d8660f..0000000 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimapBase.java +++ /dev/null @@ -1,368 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; - -import org.apache.ignite.*; -import org.apache.ignite.internal.processors.hadoop.*; -import org.apache.ignite.internal.processors.hadoop.shuffle.streams.*; -import org.apache.ignite.internal.util.*; -import org.apache.ignite.internal.util.offheap.unsafe.*; -import org.jetbrains.annotations.*; - -import java.io.*; -import java.util.*; -import java.util.concurrent.*; - -import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobProperty.*; - -/** - * Base class for all multimaps. - */ -public abstract class GridHadoopMultimapBase implements GridHadoopMultimap { - /** */ - protected final GridUnsafeMemory mem; - - /** */ - protected final int pageSize; - - /** */ - private final Collection allPages = new ConcurrentLinkedQueue<>(); - - /** - * @param jobInfo Job info. - * @param mem Memory. - */ - protected GridHadoopMultimapBase(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem) { - assert jobInfo != null; - assert mem != null; - - this.mem = mem; - - pageSize = get(jobInfo, SHUFFLE_OFFHEAP_PAGE_SIZE, 32 * 1024); - } - - /** - * @param ptrs Page pointers. - */ - private void deallocate(GridLongList ptrs) { - while (!ptrs.isEmpty()) - mem.release(ptrs.remove(), ptrs.remove()); - } - - /** - * @param valPtr Value page pointer. - * @param nextValPtr Next value page pointer. - */ - protected void nextValue(long valPtr, long nextValPtr) { - mem.writeLong(valPtr, nextValPtr); - } - - /** - * @param valPtr Value page pointer. - * @return Next value page pointer. - */ - protected long nextValue(long valPtr) { - return mem.readLong(valPtr); - } - - /** - * @param valPtr Value page pointer. - * @param size Size. - */ - protected void valueSize(long valPtr, int size) { - mem.writeInt(valPtr + 8, size); - } - - /** - * @param valPtr Value page pointer. - * @return Value size. - */ - protected int valueSize(long valPtr) { - return mem.readInt(valPtr + 8); - } - - /** {@inheritDoc} */ - @Override public void close() { - for (GridLongList list : allPages) - deallocate(list); - } - - /** - * Reader for key and value. - */ - protected class ReaderBase implements AutoCloseable { - /** */ - private Object tmp; - - /** */ - private final GridHadoopSerialization ser; - - /** */ - private final GridHadoopDataInStream in = new GridHadoopDataInStream(mem); - - /** - * @param ser Serialization. - */ - protected ReaderBase(GridHadoopSerialization ser) { - assert ser != null; - - this.ser = ser; - } - - /** - * @param valPtr Value page pointer. - * @return Value. - */ - public Object readValue(long valPtr) { - assert valPtr > 0 : valPtr; - - try { - return read(valPtr + 12, valueSize(valPtr)); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } - } - - /** - * Resets temporary object to the given one. - * - * @param tmp Temporary object for reuse. - */ - public void resetReusedObject(Object tmp) { - this.tmp = tmp; - } - - /** - * @param ptr Pointer. - * @param size Object size. - * @return Object. - */ - protected Object read(long ptr, long size) throws IgniteCheckedException { - in.buffer().set(ptr, size); - - tmp = ser.read(in, tmp); - - return tmp; - } - - /** {@inheritDoc} */ - @Override public void close() throws IgniteCheckedException { - ser.close(); - } - } - - /** - * Base class for adders. - */ - protected abstract class AdderBase implements Adder { - /** */ - protected final GridHadoopSerialization keySer; - - /** */ - protected final GridHadoopSerialization valSer; - - /** */ - private final GridHadoopDataOutStream out; - - /** */ - private long writeStart; - - /** Size and pointer pairs list. */ - private final GridLongList pages = new GridLongList(16); - - /** - * @param ctx Task context. - * @throws IgniteCheckedException If failed. - */ - protected AdderBase(GridHadoopTaskContext ctx) throws IgniteCheckedException { - valSer = ctx.valueSerialization(); - keySer = ctx.keySerialization(); - - out = new GridHadoopDataOutStream(mem) { - @Override public long move(long size) { - long ptr = super.move(size); - - if (ptr == 0) // Was not able to move - not enough free space. - ptr = allocateNextPage(size); - - assert ptr != 0; - - return ptr; - } - }; - } - - /** - * @param requestedSize Requested size. - * @return Next write pointer. - */ - private long allocateNextPage(long requestedSize) { - int writtenSize = writtenSize(); - - long newPageSize = Math.max(writtenSize + requestedSize, pageSize); - long newPagePtr = mem.allocate(newPageSize); - - pages.add(newPageSize); - pages.add(newPagePtr); - - GridHadoopOffheapBuffer b = out.buffer(); - - b.set(newPagePtr, newPageSize); - - if (writtenSize != 0) { - mem.copyMemory(writeStart, newPagePtr, writtenSize); - - b.move(writtenSize); - } - - writeStart = newPagePtr; - - return b.move(requestedSize); - } - - /** - * @return Fixed pointer. - */ - private long fixAlignment() { - GridHadoopOffheapBuffer b = out.buffer(); - - long ptr = b.pointer(); - - if ((ptr & 7L) != 0) { // Address is not aligned by octet. - ptr = (ptr + 8L) & ~7L; - - b.pointer(ptr); - } - - return ptr; - } - - /** - * @param off Offset. - * @param o Object. - * @return Page pointer. - * @throws IgniteCheckedException If failed. - */ - protected long write(int off, Object o, GridHadoopSerialization ser) throws IgniteCheckedException { - writeStart = fixAlignment(); - - if (off != 0) - out.move(off); - - ser.write(out, o); - - return writeStart; - } - - /** - * @param size Size. - * @return Pointer. - */ - protected long allocate(int size) { - writeStart = fixAlignment(); - - out.move(size); - - return writeStart; - } - - /** - * Rewinds local allocation pointer to the given pointer if possible. - * - * @param ptr Pointer. - */ - protected void localDeallocate(long ptr) { - GridHadoopOffheapBuffer b = out.buffer(); - - if (b.isInside(ptr)) - b.pointer(ptr); - else - b.reset(); - } - - /** - * @return Written size. - */ - protected int writtenSize() { - return (int)(out.buffer().pointer() - writeStart); - } - - /** {@inheritDoc} */ - @Override public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException { - throw new UnsupportedOperationException(); - } - - /** {@inheritDoc} */ - @Override public void close() throws IgniteCheckedException { - allPages.add(pages); - - keySer.close(); - valSer.close(); - } - } - - /** - * Iterator over values. - */ - protected class ValueIterator implements Iterator { - /** */ - private long valPtr; - - /** */ - private final ReaderBase valReader; - - /** - * @param valPtr Value page pointer. - * @param valReader Value reader. - */ - protected ValueIterator(long valPtr, ReaderBase valReader) { - this.valPtr = valPtr; - this.valReader = valReader; - } - - /** - * @param valPtr Head value pointer. - */ - public void head(long valPtr) { - this.valPtr = valPtr; - } - - /** {@inheritDoc} */ - @Override public boolean hasNext() { - return valPtr != 0; - } - - /** {@inheritDoc} */ - @Override public Object next() { - if (!hasNext()) - throw new NoSuchElementException(); - - Object res = valReader.readValue(valPtr); - - valPtr = nextValue(valPtr); - - return res; - } - - /** {@inheritDoc} */ - @Override public void remove() { - throw new UnsupportedOperationException(); - } - } -}