@Override public void coGroup( Iterator<Record> candidates, Iterator<Record> current, Collector<Record> out) throws Exception { if (!current.hasNext()) { throw new Exception("Error: Id not encountered before."); } Record old = current.next(); long oldId = old.getField(1, LongValue.class).getValue(); long minimumComponentID = Long.MAX_VALUE; while (candidates.hasNext()) { long candidateComponentID = candidates.next().getField(1, LongValue.class).getValue(); if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; } } if (minimumComponentID < oldId) { newComponentId.setValue(minimumComponentID); old.setField(1, newComponentId); out.collect(old); } }
/** * Computes a minimum aggregation on the distance of a data point to cluster centers. * * <p>Output Format: 0: centerID 1: pointVector 2: constant(1) (to enable combinable average * computation in the following reducer) */ @Override public void map(Record dataPointRecord, Collector<Record> out) { Point p = dataPointRecord.getField(1, Point.class); double nearestDistance = Double.MAX_VALUE; int centerId = -1; // check all cluster centers for (PointWithId center : centers) { // compute distance double distance = p.euclideanDistance(center.point); // update nearest cluster if necessary if (distance < nearestDistance) { nearestDistance = distance; centerId = center.id; } } // emit a new record with the center id and the data point. add a one to ease the // implementation of the average function with a combiner result.setField(0, new IntValue(centerId)); result.setField(1, p); result.setField(2, one); out.collect(result); }
@Override public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { String lineStr = new String(bytes, offset, numBytes); // replace reduce whitespaces and trim lineStr = lineStr.replaceAll("\\s+", " ").trim(); // build whitespace tokenizer StringTokenizer st = new StringTokenizer(lineStr, " "); // line must have at least three elements if (st.countTokens() < 3) { return null; } String rdfSubj = st.nextToken(); String rdfPred = st.nextToken(); String rdfObj = st.nextToken(); // we only want foaf:knows predicates if (!rdfPred.equals("<http://xmlns.com/foaf/0.1/knows>")) { return null; } // build node pair from subject and object fromNode.setValue(rdfSubj); toNode.setValue(rdfObj); target.setField(0, fromNode); target.setField(1, toNode); target.setField(2, pathLength); target.setField(3, hopCnt); target.setField(4, hopList); return target; }
public boolean readRecord(Record target, byte[] bytes, int offset, int numBytes) { StringValue str = this.theString; if (this.ascii) { str.setValueAscii(bytes, offset, numBytes); } else { ByteBuffer byteWrapper = this.byteWrapper; if (bytes != byteWrapper.array()) { byteWrapper = ByteBuffer.wrap(bytes, 0, bytes.length); this.byteWrapper = byteWrapper; } byteWrapper.limit(offset + numBytes); byteWrapper.position(offset); try { CharBuffer result = this.decoder.decode(byteWrapper); str.setValue(result); } catch (CharacterCodingException e) { byte[] copy = new byte[numBytes]; System.arraycopy(bytes, offset, copy, 0, numBytes); LOG.warn("Line could not be encoded: " + Arrays.toString(copy), e); return false; } } target.clear(); target.setField(this.pos, str); return true; }
@Override public Edge<VertexKey, EdgeValue> next() { Record next = input.next(); next.getFieldInto(0, keyHolder); next.getFieldInto(1, edgeValueHolder); edge.set(keyHolder, edgeValueHolder); return edge; }
@Override public void map(Record record, Collector<Record> out) throws Exception { double x = record.getField(1, DoubleValue.class).getValue(); double y = record.getField(2, DoubleValue.class).getValue(); double z = record.getField(3, DoubleValue.class).getValue(); record.setField(1, new Point(x, y, z)); out.collect(record); }
@Override public void writeRecord(Record record) throws IOException { int id = record.getField(0, IntValue.class).getValue(); Point p = record.getField(1, Point.class); byte[] bytes = String.format(format, id, p.x, p.y, p.z).getBytes(); this.stream.write(bytes); }
/** Reads all the center values from the broadcast variable into a collection. */ @Override public void open(Configuration parameters) throws Exception { Collection<Record> clusterCenters = this.getRuntimeContext().getBroadcastVariable("centers"); centers.clear(); for (Record r : clusterCenters) { centers.add( new PointWithId(r.getField(0, IntValue.class).getValue(), r.getField(1, Point.class))); } }
@Override public Record combineFirst(Iterator<Record> records) { Record next = null; long min = Long.MAX_VALUE; while (records.hasNext()) { next = records.next(); min = Math.min(min, next.getField(1, LongValue.class).getValue()); } newComponentId.setValue(min); next.setField(1, newComponentId); return next; }
public void sendMessageToAllNeighbors(Message m) { if (edgesUsed) { throw new IllegalStateException( "Can use either 'getOutgoingEdges()' or 'sendMessageToAllTargets()'."); } edgesUsed = true; while (edges.hasNext()) { Record next = edges.next(); VertexKey k = next.getField(1, this.keyClass); outValue.setField(0, k); outValue.setField(1, m); out.collect(outValue); } }
private final Record sumPointsAndCount(Iterator<Record> dataPoints) { Record next = null; p.clear(); int count = 0; // compute coordinate vector sum and count while (dataPoints.hasNext()) { next = dataPoints.next(); p.add(next.getField(1, Point.class)); count += next.getField(2, IntValue.class).getValue(); } next.setField(1, p); next.setField(2, new IntValue(count)); return next; }
/** * Filter "lineitem". * * <p>Output Schema: Key: orderkey Value: (partkey, suppkey, quantity, price) */ @Override public void map(Record record, Collector<Record> out) throws Exception { Tuple inputTuple = record.getField(1, Tuple.class); /* Extract the year from the date element of the order relation: */ /* pice = extendedprice * (1 - discount): */ float price = Float.parseFloat(inputTuple.getStringValueAt(5)) * (1 - Float.parseFloat(inputTuple.getStringValueAt(6))); /* Project (orderkey | partkey, suppkey, linenumber, quantity, extendedprice, discount, tax, ...) to (partkey, suppkey, quantity): */ inputTuple.project((0 << 0) | (1 << 1) | (1 << 2) | (0 << 3) | (1 << 4)); inputTuple.addAttribute("" + price); record.setField(1, inputTuple); out.collect(record); }
/** * Adds a record to the prepared statement. * * <p>When this method is called, the output format is guaranteed to be opened. * * @param record The records to add to the output. * @throws IOException Thrown, if the records could not be added due to an I/O problem. */ @Override public void writeRecord(Record record) throws IOException { try { for (int x = 0; x < record.getNumFields(); x++) { Value temp = record.getField(x, fieldClasses[x]); addValue(x + 1, temp); } upload.addBatch(); batchCount++; if (batchCount >= batchInterval) { upload.executeBatch(); batchCount = 0; } } catch (SQLException sqe) { throw new IllegalArgumentException("writeRecord() failed:\t", sqe); } catch (IllegalArgumentException iae) { throw new IllegalArgumentException("writeRecord() failed:\t", iae); } }
/** * The TextInputFormat seems to fail reading more than one record. I guess its an off by one * error. * * <p>The easiest workaround is to setParameter(TextInputFormat.CHARSET_NAME, "ASCII"); * * @throws IOException */ @Test public void testPositionBug() { final String FIRST = "First line"; final String SECOND = "Second line"; try { // create input file File tempFile = File.createTempFile("TextInputFormatTest", "tmp"); tempFile.deleteOnExit(); tempFile.setWritable(true); FileWriter writer = new FileWriter(tempFile); writer.append(FIRST).append('\n'); writer.append(SECOND).append('\n'); writer.close(); TextInputFormat inputFormat = new TextInputFormat(); inputFormat.setFilePath(tempFile.toURI().toString()); Configuration parameters = new Configuration(); inputFormat.configure(parameters); FileInputSplit[] splits = inputFormat.createInputSplits(1); assertTrue("expected at least one input split", splits.length >= 1); inputFormat.open(splits[0]); Record r = new Record(); assertNotNull("Expecting first record here", inputFormat.nextRecord(r)); assertEquals(FIRST, r.getField(0, StringValue.class).getValue()); assertNotNull("Expecting second record here", inputFormat.nextRecord(r)); assertEquals(SECOND, r.getField(0, StringValue.class).getValue()); assertNull("The input file is over", inputFormat.nextRecord(r)); } catch (Throwable t) { System.err.println("test failed with exception: " + t.getMessage()); t.printStackTrace(System.err); fail("Test erroneous"); } }
@Override public void map(Record record, Collector<Record> out) throws Exception { for (Record model : this.models) { // compute dot product between model and pair long product = 0; for (int i = 1; i <= NUM_FEATURES; i++) { product += model.getField(i, this.lft).getValue() * record.getField(i, this.rgt).getValue(); } this.prd.setValue(product); // construct result this.result.copyFrom(model, new int[] {0}, new int[] {0}); this.result.copyFrom(record, new int[] {0}, new int[] {1}); this.result.setField(2, this.prd); // emit result out.collect(this.result); } }
@Override public void writeRecord(Record record) throws IOException { StringBuilder line = new StringBuilder(); // append from-node line.append(record.getField(0, StringValue.class).toString()); line.append("|"); // append to-node line.append(record.getField(1, StringValue.class).toString()); line.append("|"); // append length line.append(record.getField(2, IntValue.class).toString()); line.append("|"); // append hopCnt line.append(record.getField(3, IntValue.class).toString()); line.append("|"); // append hopList line.append(record.getField(4, StringValue.class).toString()); line.append("|"); line.append("\n"); stream.write(line.toString().getBytes()); }
@Override public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { String lineStr = new String(bytes, offset, numBytes); StringTokenizer st = new StringTokenizer(lineStr, "|"); // path must have exactly 5 tokens (fromNode, toNode, length, hopCnt, hopList) if (st.countTokens() != 5) { return null; } this.fromNode.setValue(st.nextToken()); this.toNode.setValue(st.nextToken()); this.length.setValue(Integer.parseInt(st.nextToken())); this.hopCnt.setValue(Integer.parseInt(st.nextToken())); this.hopList.setValue(st.nextToken()); target.setField(0, fromNode); target.setField(1, toNode); target.setField(2, length); target.setField(3, hopCnt); target.setField(4, hopList); return target; }
/** * Splits the document into terms and emits a PactRecord (docId, term, tf) for each term of the * document. * * <p>Each input document has the format "docId, document contents". */ @Override public void map(Record record, Collector<Record> collector) { String document = record.getField(0, StringValue.class).toString(); // split document into a , separated list String data[] = document.split(","); int docID = Integer.parseInt(data[0]); // String docID = data[0]; document = data[1]; document = document.replaceAll("\\W", " ").toLowerCase(); StringTokenizer tokenizer = new StringTokenizer(document); HashSet<String> stopWords = Util.STOP_WORDS; Map<String, Integer> map = new HashMap<String, Integer>(); // to identify the frequency of each word in the document int co = 1; while (tokenizer.hasMoreElements()) { String word = tokenizer.nextToken(); if (stopWords.contains(word.toString())) { continue; } if (map.containsKey(word)) { // if the word added previously increment the count by one co++; map.put(word, co); } else { // add a new word to the map co = 1; map.put(word, co); } } Iterator iterator = map.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry pairs = (Map.Entry) iterator.next(); String word = pairs.getKey().toString(); int occur = Integer.parseInt(pairs.getValue().toString()); result.setField(0, new IntValue(docID)); result.setField(1, new StringValue(word)); result.setField(2, new IntValue(occur)); collector.collect(result); } }
/** Compute the new position (coordinate vector) of a cluster center. */ @Override public void reduce(Iterator<Record> points, Collector<Record> out) { Record sum = sumPointsAndCount(points); sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); out.collect(sum); }
@Override public void join(Record rec1, Record rec2, Collector<Record> out) throws Exception { // rec1 has matching start, rec2 matching end // Therefore, rec2's end node and rec1's start node are identical // First half of new path will be rec2, second half will be rec1 // Get from-node and to-node of new path final StringValue fromNode = rec2.getField(0, StringValue.class); final StringValue toNode = rec1.getField(1, StringValue.class); // Check whether from-node = to-node to prevent circles! if (fromNode.equals(toNode)) { return; } // Create new path outputRecord.setField(0, fromNode); outputRecord.setField(1, toNode); // Compute length of new path length.setValue( rec1.getField(2, IntValue.class).getValue() + rec2.getField(2, IntValue.class).getValue()); outputRecord.setField(2, length); // compute hop count int hops = rec1.getField(3, IntValue.class).getValue() + 1 + rec2.getField(3, IntValue.class).getValue(); hopCnt.setValue(hops); outputRecord.setField(3, hopCnt); // Concatenate hops lists and insert matching node StringBuilder sb = new StringBuilder(); // first path sb.append(rec2.getField(4, StringValue.class).getValue()); sb.append(" "); // common node sb.append(rec1.getField(0, StringValue.class).getValue()); // second path sb.append(" "); sb.append(rec1.getField(4, StringValue.class).getValue()); hopList.setValue(sb.toString().trim()); outputRecord.setField(4, hopList); out.collect(outputRecord); }
@Override public void convert(Record stratosphereRecord, K hadoopKey, V hadoopValue) { stratosphereRecord.setField(0, convert(hadoopKey)); stratosphereRecord.setField(1, convert(hadoopValue)); }
/* * This test is basically identical to the "testSpillingHashJoinWithMassiveCollisions" test, only that the number * of repeated values (causing bucket collisions) are large enough to make sure that their target partition no longer * fits into memory by itself and needs to be repartitioned in the recursion again. */ @Test public void testSpillingHashJoinWithTwoRecursions() throws IOException { // the following two values are known to have a hash-code collision on the first recursion // level. // we use them to make sure one partition grows over-proportionally large final int REPEATED_VALUE_1 = 40559; final int REPEATED_VALUE_2 = 92882; final int REPEATED_VALUE_COUNT_BUILD = 200000; final int REPEATED_VALUE_COUNT_PROBE = 5; final int NUM_KEYS = 1000000; final int BUILD_VALS_PER_KEY = 3; final int PROBE_VALS_PER_KEY = 10; // create a build input that gives 3 million pairs with 3 values sharing the same key, plus 400k // pairs with two colliding keys MutableObjectIterator<Record> build1 = new UniformRecordGenerator(NUM_KEYS, BUILD_VALS_PER_KEY, false); MutableObjectIterator<Record> build2 = new ConstantsKeyValuePairsIterator(REPEATED_VALUE_1, 17, REPEATED_VALUE_COUNT_BUILD); MutableObjectIterator<Record> build3 = new ConstantsKeyValuePairsIterator(REPEATED_VALUE_2, 23, REPEATED_VALUE_COUNT_BUILD); List<MutableObjectIterator<Record>> builds = new ArrayList<MutableObjectIterator<Record>>(); builds.add(build1); builds.add(build2); builds.add(build3); MutableObjectIterator<Record> buildInput = new UnionIterator<Record>(builds); // allocate the memory for the HashTable List<MemorySegment> memSegments; try { memSegments = this.memoryManager.allocatePages(MEM_OWNER, 896); } catch (MemoryAllocationException maex) { fail("Memory for the Join could not be provided."); return; } // create the map for validating the results HashMap<Integer, Long> map = new HashMap<Integer, Long>(NUM_KEYS); // ---------------------------------------------------------------------------------------- final ReOpenableMutableHashTable<Record, Record> join = new ReOpenableMutableHashTable<Record, Record>( this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator, memSegments, ioManager); for (int probe = 0; probe < NUM_PROBES; probe++) { // create a probe input that gives 10 million pairs with 10 values sharing a key MutableObjectIterator<Record> probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2); if (probe == 0) { join.open(buildInput, probeInput); } else { join.reopenProbe(probeInput); } Record record; final Record recordReuse = new Record(); while (join.nextRecord()) { int numBuildValues = 0; final Record probeRec = join.getCurrentProbeRecord(); int key = probeRec.getField(0, IntValue.class).getValue(); HashBucketIterator<Record, Record> buildSide = join.getBuildSideIterator(); if ((record = buildSide.next(recordReuse)) != null) { numBuildValues = 1; Assert.assertEquals( "Probe-side key was different than build-side key.", key, record.getField(0, IntValue.class).getValue()); } else { fail("No build side values found for a probe key."); } while ((record = buildSide.next(recordReuse)) != null) { numBuildValues++; Assert.assertEquals( "Probe-side key was different than build-side key.", key, record.getField(0, IntValue.class).getValue()); } Long contained = map.get(key); if (contained == null) { contained = new Long(numBuildValues); } else { contained = new Long(contained.longValue() + numBuildValues); } map.put(key, contained); } } join.close(); Assert.assertEquals("Wrong number of keys", NUM_KEYS, map.size()); for (Map.Entry<Integer, Long> entry : map.entrySet()) { long val = entry.getValue(); int key = entry.getKey(); if (key == REPEATED_VALUE_1 || key == REPEATED_VALUE_2) { Assert.assertEquals( "Wrong number of values in per-key cross product for key " + key, (PROBE_VALS_PER_KEY + REPEATED_VALUE_COUNT_PROBE) * (BUILD_VALS_PER_KEY + REPEATED_VALUE_COUNT_BUILD) * NUM_PROBES, val); } else { Assert.assertEquals( "Wrong number of values in per-key cross product for key " + key, PROBE_VALS_PER_KEY * BUILD_VALS_PER_KEY * NUM_PROBES, val); } } // ---------------------------------------------------------------------------------------- this.memoryManager.release(join.getFreedMemory()); }
@Override public void coGroup( Iterator<Record> inputRecords, Iterator<Record> concatRecords, Collector<Record> out) { // init minimum length and minimum path Record pathRec = null; StringValue path = null; if (inputRecords.hasNext()) { // path is in input paths pathRec = inputRecords.next(); } else { // path must be in concat paths pathRec = concatRecords.next(); } // get from node (common for all paths) StringValue fromNode = pathRec.getField(0, StringValue.class); // get to node (common for all paths) StringValue toNode = pathRec.getField(1, StringValue.class); // get length of path minLength.setValue(pathRec.getField(2, IntValue.class).getValue()); // store path and hop count path = new StringValue(pathRec.getField(4, StringValue.class)); shortestPaths.add(path); hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); // find shortest path of all input paths while (inputRecords.hasNext()) { pathRec = inputRecords.next(); IntValue length = pathRec.getField(2, IntValue.class); if (length.getValue() == minLength.getValue()) { // path has also minimum length add to list path = new StringValue(pathRec.getField(4, StringValue.class)); if (shortestPaths.add(path)) { hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); } } else if (length.getValue() < minLength.getValue()) { // path has minimum length minLength.setValue(length.getValue()); // clear lists hopCnts.clear(); shortestPaths.clear(); // get path and add path and hop count path = new StringValue(pathRec.getField(4, StringValue.class)); shortestPaths.add(path); hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); } } // find shortest path of all input and concatenated paths while (concatRecords.hasNext()) { pathRec = concatRecords.next(); IntValue length = pathRec.getField(2, IntValue.class); if (length.getValue() == minLength.getValue()) { // path has also minimum length add to list path = new StringValue(pathRec.getField(4, StringValue.class)); if (shortestPaths.add(path)) { hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); } } else if (length.getValue() < minLength.getValue()) { // path has minimum length minLength.setValue(length.getValue()); // clear lists hopCnts.clear(); shortestPaths.clear(); // get path and add path and hop count path = new StringValue(pathRec.getField(4, StringValue.class)); shortestPaths.add(path); hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); } } outputRecord.setField(0, fromNode); outputRecord.setField(1, toNode); outputRecord.setField(2, minLength); // emit all shortest paths for (StringValue shortestPath : shortestPaths) { outputRecord.setField(3, hopCnts.get(shortestPath)); outputRecord.setField(4, shortestPath); out.collect(outputRecord); } hopCnts.clear(); shortestPaths.clear(); }