@SuppressWarnings("unchecked") public synchronized void collect(K key, V value) throws IOException { reporter.progress(); if (key.getClass() != keyClass) { throw new IOException( "Type mismatch in key from map: expected " + keyClass.getName() + ", recieved " + key.getClass().getName()); } if (value.getClass() != valClass) { throw new IOException( "Type mismatch in value from map: expected " + valClass.getName() + ", recieved " + value.getClass().getName()); } if (sortSpillException != null) { throw (IOException) new IOException("Spill failed").initCause(sortSpillException); } try { // serialize key bytes into buffer int keystart = bufindex; keySerializer.serialize(key); if (bufindex < keystart) { // wrapped the key; reset required bb.reset(); keystart = 0; } // serialize value bytes into buffer int valstart = bufindex; valSerializer.serialize(value); int valend = bb.markRecord(); mapOutputByteCounter.increment( valend >= keystart ? valend - keystart : (bufvoid - keystart) + valend); if (keystart == bufindex) { // if emitted records make no writes, it's possible to wrap // accounting space without notice bb.write(new byte[0], 0, 0); } int partition = partitioner.getPartition(key, value, partitions); if (partition < 0 || partition >= partitions) { throw new IOException("Illegal partition for " + key + " (" + partition + ")"); } mapOutputRecordCounter.increment(1); // update accounting info int ind = kvindex * ACCTSIZE; kvoffsets[kvindex] = ind; kvindices[ind + PARTITION] = partition; kvindices[ind + KEYSTART] = keystart; kvindices[ind + VALSTART] = valstart; kvindex = (kvindex + 1) % kvoffsets.length; } catch (MapBufferTooSmallException e) { LOG.info("Record too large for in-memory buffer: " + e.getMessage()); spillSingleRecord(key, value); mapOutputRecordCounter.increment(1); return; } }
/** * Basic test of the ability to add to a buffer with a fixed capacity queue and to drain the * elements from the queue including tests of the non-blocking aspects of the API. * * @throws TimeoutException * @throws ExecutionException * @throws InterruptedException */ public void test_blockingBuffer() throws InterruptedException, ExecutionException, TimeoutException { final Integer e0 = new Integer(0); final Integer e1 = new Integer(1); final Integer e2 = new Integer(2); final int queueCapacity = 3; final BlockingQueue<Integer[]> queue = new ArrayBlockingQueue<Integer[]>(queueCapacity); final int chunkSize = 4; final long chunkTimeout = 1000; final TimeUnit chunkTimeoutUnit = TimeUnit.MILLISECONDS; /* * The test timeout is just a smidge longer than the chunk timeout. * * Note: use Long.MAX_VALUE iff debugging. */ // final long testTimeout = Long.MAX_VALUE; final long testTimeout = chunkTimeout + 20; final boolean ordered = false; final BlockingBuffer<Integer[]> buffer = new BlockingBuffer<Integer[]>(queue, chunkSize, chunkTimeout, chunkTimeoutUnit, ordered); // buffer is empty. assertTrue(buffer.isOpen()); assertTrue(buffer.isEmpty()); assertEquals("chunkCount", 0L, buffer.getChunksAddedCount()); assertEquals("elementCount", 0L, buffer.getElementsAddedCount()); final IAsynchronousIterator<Integer[]> itr = buffer.iterator(); // nothing available from the iterator (non-blocking test). assertFalse(itr.hasNext(1, TimeUnit.NANOSECONDS)); assertNull(itr.next(1, TimeUnit.NANOSECONDS)); // add an element to the buffer - should not block. buffer.add(new Integer[] {e0}); // should be one element and one chunk accepted by the buffer. assertTrue(buffer.isOpen()); assertFalse(buffer.isEmpty()); assertEquals("chunkCount", 1L, buffer.getChunksAddedCount()); assertEquals("elementCount", 1L, buffer.getElementsAddedCount()); // something should be available now (non-blocking). assertTrue(itr.hasNext(1, TimeUnit.NANOSECONDS)); // something should be available now (blocking). assertTrue(itr.hasNext()); // add another element to the buffer - should not block. buffer.add(new Integer[] {e1}); // should be two elements and two chunks accepted into the buffer assertTrue(buffer.isOpen()); assertFalse(buffer.isEmpty()); assertEquals("chunkCount", 2L, buffer.getChunksAddedCount()); assertEquals("elementCount", 2L, buffer.getElementsAddedCount()); final ReentrantLock lock = new ReentrantLock(); final Condition cond = lock.newCondition(); final AtomicBoolean proceedFlag = new AtomicBoolean(false); // future of task writing a 3rd element on the buffer. final Future<?> producerFuture = service.submit( new Callable<Void>() { public Void call() throws Exception { lock.lockInterruptibly(); try { if (!proceedFlag.get()) { cond.await(); } /* * add another element - should block until we take an * element using the iterator. */ buffer.add(new Integer[] {e2}); /* * itr.hasNext() will block until the buffer is closed. */ buffer.close(); } finally { lock.unlock(); } // done. return null; } }); // future of task draining the buffer. final Future<?> consumerFuture = service.submit( new Callable<Void>() { public Void call() throws Exception { try { lock.lockInterruptibly(); try { assertTrue(itr.hasNext()); // take the first chunk - two elements. if (log.isInfoEnabled()) log.info("Awaiting first chunk"); assertSameArray(new Integer[] {e0, e1}, itr.next(50, TimeUnit.MILLISECONDS)); if (log.isInfoEnabled()) log.info("Have first chunk"); /* * Verify that we obtained the first chunk before the * buffer was closed. Otherwise next() blocked * attempting to compile a full chunk until the producer * timeout, at which point the producer closed the * buffer and next() noticed the closed buffer and * returned. */ assertTrue(buffer.isOpen()); assertFalse("buffer was closed.", itr.isExhausted()); /* * Verify that nothing is available from the iterator * (non-blocking test). */ assertFalse(itr.hasNext(1, TimeUnit.NANOSECONDS)); assertNull(itr.next(1, TimeUnit.NANOSECONDS)); // Signal the producer that it should continue. proceedFlag.set(true); cond.signal(); } finally { lock.unlock(); } // should block until we close the buffer. assertTrue(itr.hasNext()); // last chunk assertSameArray(new Integer[] {e2}, itr.next()); // should be immediately false. assertFalse(itr.hasNext(1, TimeUnit.NANOSECONDS)); // should be immediately null. assertNull(itr.next(1, TimeUnit.NANOSECONDS)); // The synchronous API should also report an exhausted // itr. assertFalse(itr.hasNext()); try { itr.next(); fail("Expecting: " + NoSuchElementException.class); } catch (NoSuchElementException ex) { if (log.isInfoEnabled()) log.info("Ignoring expected exception: " + ex); } return null; } catch (Throwable t) { log.error("Consumer failed or blocked: " + t, t); throw new Exception(t); } } }); // wait a little bit for the producer future. producerFuture.get(testTimeout, chunkTimeoutUnit); // wait a little bit for the consumer future. consumerFuture.get(testTimeout, chunkTimeoutUnit); }
/** * Serialize the key, value to intermediate storage. When this method returns, kvindex must refer * to sufficient unused storage to store one METADATA. */ synchronized void collect(Object key, Object value, final int partition) throws IOException { if (key.getClass() != keyClass) { throw new IOException( "Type mismatch in key from map: expected " + keyClass.getName() + ", received " + key.getClass().getName()); } if (value.getClass() != valClass) { throw new IOException( "Type mismatch in value from map: expected " + valClass.getName() + ", received " + value.getClass().getName()); } if (partition < 0 || partition >= partitions) { throw new IOException( "Illegal partition for " + key + " (" + partition + ")" + ", TotalPartitions: " + partitions); } checkSpillException(); bufferRemaining -= METASIZE; if (bufferRemaining <= 0) { // start spill if the thread is not running and the soft limit has been // reached spillLock.lock(); try { do { if (!spillInProgress) { final int kvbidx = 4 * kvindex; final int kvbend = 4 * kvend; // serialized, unspilled bytes always lie between kvindex and // bufindex, crossing the equator. Note that any void space // created by a reset must be included in "used" bytes final int bUsed = distanceTo(kvbidx, bufindex); final boolean bufsoftlimit = bUsed >= softLimit; if ((kvbend + METASIZE) % kvbuffer.length != equator - (equator % METASIZE)) { // spill finished, reclaim space resetSpill(); bufferRemaining = Math.min(distanceTo(bufindex, kvbidx) - 2 * METASIZE, softLimit - bUsed) - METASIZE; continue; } else if (bufsoftlimit && kvindex != kvend) { // spill records, if any collected; check latter, as it may // be possible for metadata alignment to hit spill pcnt startSpill(); final int avgRec = (int) (mapOutputByteCounter.getValue() / mapOutputRecordCounter.getValue()); // leave at least half the split buffer for serialization data // ensure that kvindex >= bufindex final int distkvi = distanceTo(bufindex, kvbidx); final int newPos = (bufindex + Math.max( 2 * METASIZE - 1, Math.min(distkvi / 2, distkvi / (METASIZE + avgRec) * METASIZE))) % kvbuffer.length; setEquator(newPos); bufmark = bufindex = newPos; final int serBound = 4 * kvend; // bytes remaining before the lock must be held and limits // checked is the minimum of three arcs: the metadata space, the // serialization space, and the soft limit bufferRemaining = Math.min( // metadata max distanceTo(bufend, newPos), Math.min( // serialization max distanceTo(newPos, serBound), // soft limit softLimit)) - 2 * METASIZE; } } } while (false); } finally { spillLock.unlock(); } } try { // serialize key bytes into buffer int keystart = bufindex; keySerializer.serialize(key); if (bufindex < keystart) { // wrapped the key; must make contiguous bb.shiftBufferedKey(); keystart = 0; } // serialize value bytes into buffer final int valstart = bufindex; valSerializer.serialize(value); // It's possible for records to have zero length, i.e. the serializer // will perform no writes. To ensure that the boundary conditions are // checked and that the kvindex invariant is maintained, perform a // zero-length write into the buffer. The logic monitoring this could be // moved into collect, but this is cleaner and inexpensive. For now, it // is acceptable. bb.write(b0, 0, 0); // the record must be marked after the preceding write, as the metadata // for this record are not yet written int valend = bb.markRecord(); mapOutputRecordCounter.increment(1); mapOutputByteCounter.increment(distanceTo(keystart, valend, bufvoid)); // write accounting info kvmeta.put(kvindex + PARTITION, partition); kvmeta.put(kvindex + KEYSTART, keystart); kvmeta.put(kvindex + VALSTART, valstart); kvmeta.put(kvindex + VALLEN, distanceTo(valstart, valend)); // advance kvindex kvindex = (int) (((long) kvindex - NMETA + kvmeta.capacity()) % kvmeta.capacity()); } catch (MapBufferTooSmallException e) { LOG.info("Record too large for in-memory buffer: " + e.getMessage()); spillSingleRecord(key, value, partition); mapOutputRecordCounter.increment(1); return; } }