MutableHashTable<Tuple2<Long, byte[]>, Long> table = new MutableHashTable<>( buildSerializer, probeSerializer, buildComparator, probeComparator, pairComparator, memory, ioMan, 16, false); table.open(buildInput, probeInput); while (table.nextRecord()) { MutableObjectIterator<Tuple2<Long, byte[]>> matches = table.getBuildSideIterator(); while (matches.next() != null) {} table.close();
@Override public void abort() { this.running = false; this.hashJoin.abort(); } }
throws Exception if (this.hashJoin.nextRecord()) final MutableObjectIterator<V2> buildSideIterator = this.hashJoin.getBuildSideIterator(); final V1 probeRecord = this.hashJoin.getCurrentProbeRecord(); V2 nextBuildSideRecord = buildSideIterator.next(this.nextBuildSideObject);
@Override public void close() { // close the join this.hashJoin.close(); // free the memory final List<MemorySegment> segments = this.hashJoin.getFreedMemory(); this.memManager.release(segments); }
this.writeBehindBuffersAvailable += buffersAvailable; releaseTable(); MemorySegment seg1 = getNextBuffer(); if (seg1 != null) { memory.add(seg1); MemorySegment seg2 = getNextBuffer(); if (seg2 != null) { memory.add(seg2); buildTableFromSpilledPartition(p); MemorySegment seg1 = getNextBuffer(); if (seg1 != null) { memory.add(seg1); MemorySegment seg2 = getNextBuffer(); if (seg2 != null) { memory.add(seg2); return nextRecord();
final MemorySegment[] table = new MemorySegment[numSegs]; ensureNumBuffersReturned(numSegs); final MemorySegment seg = getNextBuffer(); final byte partition = assignPartition(bucket, numPartitions); initBloomFilter(numBuckets);
/** * Opens the hash join. This method reads the build-side input and constructs the initial * hash table, gradually spilling partitions that do not fit into memory. * * @param buildSide Build side input. * @param probeSide Probe side input. * @throws IOException Thrown, if an I/O problem occurs while spilling a partition. */ public void open(final MutableObjectIterator<BT> buildSide, final MutableObjectIterator<PT> probeSide) throws IOException { open(buildSide, probeSide, false); }
@Override public void close() { if (partitionsBeingBuilt.size() == 0) { // partitions are cleared after the build phase. But we need to drop // memory with them. this.partitionsBeingBuilt.addAll(initialPartitions); } this.furtherPartitioning = true; // fake, to release table properly (close() will call releaseTable()) super.close(); } }
public <BT, PT> MutableHashTable<BT, PT> getHashJoin( TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator, TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator, TypePairComparator<PT, BT> pairComparator, MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction, boolean useBloomFilters) throws MemoryAllocationException { final int numPages = memManager.computeNumberOfPages(memoryFraction); final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages); return new MutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager, useBloomFilters); } }
final protected void buildBloomFilterForBucketsInPartition(int partNum, HashPartition<BT, PT> partition) { // Find all the buckets which belongs to this partition, and build bloom filter for each bucket(include its overflow buckets). final int bucketsPerSegment = this.bucketsPerSegmentMask + 1; int numSegs = this.buckets.length; // go over all segments that are part of the table for (int i = 0, bucket = 0; i < numSegs && bucket < numBuckets; i++) { final MemorySegment segment = this.buckets[i]; // go over all buckets in the segment for (int k = 0; k < bucketsPerSegment && bucket < numBuckets; k++, bucket++) { final int bucketInSegmentOffset = k * HASH_BUCKET_SIZE; byte partitionNumber = segment.get(bucketInSegmentOffset + HEADER_PARTITION_OFFSET); if (partitionNumber == partNum) { byte status = segment.get(bucketInSegmentOffset + HEADER_STATUS_OFFSET); if (status == BUCKET_STATUS_IN_MEMORY) { buildBloomFilterForBucket(bucketInSegmentOffset, segment, partition); } } } } }
/** * Set all the bucket memory except bucket header as the bit set of bloom filter, and use hash code of build records * to build bloom filter. */ final void buildBloomFilterForBucket(int bucketInSegmentPos, MemorySegment bucket, HashPartition<BT, PT> p) { final int count = bucket.getShort(bucketInSegmentPos + HEADER_COUNT_OFFSET); if (count <= 0) { return; } int[] hashCodes = new int[count]; // As the hashcode and bloom filter occupy same bytes, so we read all hashcode out at first and then write back to bloom filter. for (int i = 0; i < count; i++) { hashCodes[i] = bucket.getInt(bucketInSegmentPos + BUCKET_HEADER_LENGTH + i * HASH_CODE_LEN); } this.bloomFilter.setBitsLocation(bucket, bucketInSegmentPos + BUCKET_HEADER_LENGTH); for (int hashCode : hashCodes) { this.bloomFilter.addHash(hashCode); } buildBloomFilterForExtraOverflowSegments(bucketInSegmentPos, bucket, p); }
buildBloomFilterForBucketsInPartition(largestPartNum, p);
throws Exception if (this.hashJoin.nextRecord()) final MutableObjectIterator<V2> buildSideIterator = this.hashJoin.getBuildSideIterator(); final V1 probeRecord = this.hashJoin.getCurrentProbeRecord(); V2 nextBuildSideRecord = buildSideIterator.next(this.nextBuildSideObject);
@Override public void close() { // close the join this.hashJoin.close(); // free the memory final List<MemorySegment> segments = this.hashJoin.getFreedMemory(); this.memManager.release(segments); }
this.writeBehindBuffersAvailable += buffersAvailable; releaseTable(); MemorySegment seg1 = getNextBuffer(); if (seg1 != null) { memory.add(seg1); MemorySegment seg2 = getNextBuffer(); if (seg2 != null) { memory.add(seg2); buildTableFromSpilledPartition(p); MemorySegment seg1 = getNextBuffer(); if (seg1 != null) { memory.add(seg1); MemorySegment seg2 = getNextBuffer(); if (seg2 != null) { memory.add(seg2); return nextRecord();
final MemorySegment[] table = new MemorySegment[numSegs]; ensureNumBuffersReturned(numSegs); final MemorySegment seg = getNextBuffer(); final byte partition = assignPartition(bucket, numPartitions); initBloomFilter(numBuckets);
/** * Opens the hash join. This method reads the build-side input and constructs the initial * hash table, gradually spilling partitions that do not fit into memory. * * @param buildSide Build side input. * @param probeSide Probe side input. * @throws IOException Thrown, if an I/O problem occurs while spilling a partition. */ public void open(final MutableObjectIterator<BT> buildSide, final MutableObjectIterator<PT> probeSide) throws IOException { open(buildSide, probeSide, false); }
@Override public void close() { if (partitionsBeingBuilt.size() == 0) { // partitions are cleared after the build phase. But we need to drop // memory with them. this.partitionsBeingBuilt.addAll(initialPartitions); } this.furtherPartitioning = true; // fake, to release table properly (close() will call releaseTable()) super.close(); } }
public <BT, PT> MutableHashTable<BT, PT> getHashJoin( TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator, TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator, TypePairComparator<PT, BT> pairComparator, MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction, boolean useBloomFilters) throws MemoryAllocationException { final int numPages = memManager.computeNumberOfPages(memoryFraction); final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages); return new MutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager, useBloomFilters); } }
final protected void buildBloomFilterForBucketsInPartition(int partNum, HashPartition<BT, PT> partition) { // Find all the buckets which belongs to this partition, and build bloom filter for each bucket(include its overflow buckets). final int bucketsPerSegment = this.bucketsPerSegmentMask + 1; int numSegs = this.buckets.length; // go over all segments that are part of the table for (int i = 0, bucket = 0; i < numSegs && bucket < numBuckets; i++) { final MemorySegment segment = this.buckets[i]; // go over all buckets in the segment for (int k = 0; k < bucketsPerSegment && bucket < numBuckets; k++, bucket++) { final int bucketInSegmentOffset = k * HASH_BUCKET_SIZE; byte partitionNumber = segment.get(bucketInSegmentOffset + HEADER_PARTITION_OFFSET); if (partitionNumber == partNum) { byte status = segment.get(bucketInSegmentOffset + HEADER_STATUS_OFFSET); if (status == BUCKET_STATUS_IN_MEMORY) { buildBloomFilterForBucket(bucketInSegmentOffset, segment, partition); } } } } }