@Test
  public void testBuildSecondWithMixedDataTypes() {
    try {
      MutableObjectIterator<IntPair> input1 = new UniformIntPairGenerator(500, 40, false);

      final TestData.TupleGenerator generator2 =
          new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
      final TestData.TupleGeneratorIterator input2 =
          new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);

      // collect expected data
      final Map<Integer, Collection<TupleIntPairMatch>> expectedMatchesMap =
          joinIntPairs(collectIntPairData(input1), collectTupleData(input2));

      final FlatJoinFunction<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>> matcher =
          new TupleIntPairMatchRemovingMatcher(expectedMatchesMap);
      final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();

      // reset the generators
      input1 = new UniformIntPairGenerator(500, 40, false);
      generator2.reset();
      input2.reset();

      // compare with iterator values
      ReusingBuildFirstHashJoinIterator<IntPair, Tuple2<Integer, String>, Tuple2<Integer, String>>
          iterator =
              new ReusingBuildFirstHashJoinIterator<>(
                  input1,
                  input2,
                  this.pairSerializer,
                  this.pairComparator,
                  this.recordSerializer,
                  this.record2Comparator,
                  this.recordPairPairComparator,
                  this.memoryManager,
                  this.ioManager,
                  this.parentTask,
                  1.0,
                  false,
                  true);

      iterator.open();

      while (iterator.callWithNextKey(matcher, collector)) ;

      iterator.close();

      // assert that each expected match was seen
      for (Entry<Integer, Collection<TupleIntPairMatch>> entry : expectedMatchesMap.entrySet()) {
        if (!entry.getValue().isEmpty()) {
          Assert.fail("Collection for key " + entry.getKey() + " is not empty");
        }
      }
    } catch (Exception e) {
      e.printStackTrace();
      Assert.fail("An exception occurred during the test: " + e.getMessage());
    }
  }
  @Test
  public void testBuildSecondWithHighNumberOfCommonKeys() {
    // the size of the left and right inputs
    final int INPUT_1_SIZE = 200;
    final int INPUT_2_SIZE = 100;

    final int INPUT_1_DUPLICATES = 10;
    final int INPUT_2_DUPLICATES = 2000;
    final int DUPLICATE_KEY = 13;

    try {
      TestData.TupleGenerator generator1 =
          new TestData.TupleGenerator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
      TestData.TupleGenerator generator2 =
          new TestData.TupleGenerator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);

      final TestData.TupleGeneratorIterator gen1Iter =
          new TestData.TupleGeneratorIterator(generator1, INPUT_1_SIZE);
      final TestData.TupleGeneratorIterator gen2Iter =
          new TestData.TupleGeneratorIterator(generator2, INPUT_2_SIZE);

      final TestData.TupleConstantValueIterator const1Iter =
          new TestData.TupleConstantValueIterator(
              DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES);
      final TestData.TupleConstantValueIterator const2Iter =
          new TestData.TupleConstantValueIterator(
              DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES);

      final List<MutableObjectIterator<Tuple2<Integer, String>>> inList1 = new ArrayList<>();
      inList1.add(gen1Iter);
      inList1.add(const1Iter);

      final List<MutableObjectIterator<Tuple2<Integer, String>>> inList2 = new ArrayList<>();
      inList2.add(gen2Iter);
      inList2.add(const2Iter);

      MutableObjectIterator<Tuple2<Integer, String>> input1 = new UnionIterator<>(inList1);
      MutableObjectIterator<Tuple2<Integer, String>> input2 = new UnionIterator<>(inList2);

      // collect expected data
      final Map<Integer, Collection<TupleMatch>> expectedMatchesMap =
          joinTuples(collectTupleData(input1), collectTupleData(input2));

      // re-create the whole thing for actual processing

      // reset the generators and iterators
      generator1.reset();
      generator2.reset();
      const1Iter.reset();
      const2Iter.reset();
      gen1Iter.reset();
      gen2Iter.reset();

      inList1.clear();
      inList1.add(gen1Iter);
      inList1.add(const1Iter);

      inList2.clear();
      inList2.add(gen2Iter);
      inList2.add(const2Iter);

      input1 = new UnionIterator<>(inList1);
      input2 = new UnionIterator<>(inList2);

      final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
      final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();

      ReusingBuildSecondHashJoinIterator<
              Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>>
          iterator =
              new ReusingBuildSecondHashJoinIterator<>(
                  input1,
                  input2,
                  this.recordSerializer,
                  this.record1Comparator,
                  this.recordSerializer,
                  this.record2Comparator,
                  this.recordPairComparator,
                  this.memoryManager,
                  ioManager,
                  this.parentTask,
                  1.0,
                  false,
                  true);

      iterator.open();

      while (iterator.callWithNextKey(matcher, collector)) ;

      iterator.close();

      // assert that each expected match was seen
      for (Entry<Integer, Collection<TupleMatch>> entry : expectedMatchesMap.entrySet()) {
        if (!entry.getValue().isEmpty()) {
          Assert.fail("Collection for key " + entry.getKey() + " is not empty");
        }
      }
    } catch (Exception e) {
      e.printStackTrace();
      Assert.fail("An exception occurred during the test: " + e.getMessage());
    }
  }
  private void doTest(
      TestData.TupleGeneratorIterator buildInput,
      TestData.TupleGeneratorIterator probeInput,
      TestData.TupleGenerator bgen,
      TestData.TupleGenerator pgen)
      throws Exception {
    // collect expected data
    final Map<Integer, Collection<TupleMatch>> expectedFirstMatchesMap =
        ReusingHashMatchIteratorITCase.matchSecondTupleFields(
            ReusingHashMatchIteratorITCase.collectTupleData(buildInput),
            ReusingHashMatchIteratorITCase.collectTupleData(probeInput));

    final List<Map<Integer, Collection<TupleMatch>>> expectedNMatchesMapList =
        new ArrayList<>(NUM_PROBES);
    final FlatJoinFunction[] nMatcher = new TupleMatchRemovingJoin[NUM_PROBES];
    for (int i = 0; i < NUM_PROBES; i++) {
      Map<Integer, Collection<TupleMatch>> tmp;
      expectedNMatchesMapList.add(tmp = deepCopy(expectedFirstMatchesMap));
      nMatcher[i] = new TupleMatchRemovingJoin(tmp);
    }

    final FlatJoinFunction firstMatcher = new TupleMatchRemovingJoin(expectedFirstMatchesMap);

    final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();

    // reset the generators
    bgen.reset();
    pgen.reset();
    buildInput.reset();
    probeInput.reset();

    // compare with iterator values
    ReusingBuildFirstReOpenableHashMatchIterator<
            Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>>
        iterator =
            new ReusingBuildFirstReOpenableHashMatchIterator<>(
                buildInput,
                probeInput,
                this.recordSerializer,
                this.record1Comparator,
                this.recordSerializer,
                this.record2Comparator,
                this.recordPairComparator,
                this.memoryManager,
                ioManager,
                this.parentTask,
                1.0,
                true);

    iterator.open();
    // do first join with both inputs
    while (iterator.callWithNextKey(firstMatcher, collector)) ;

    // assert that each expected match was seen for the first input
    for (Entry<Integer, Collection<TupleMatch>> entry : expectedFirstMatchesMap.entrySet()) {
      if (!entry.getValue().isEmpty()) {
        Assert.fail("Collection for key " + entry.getKey() + " is not empty");
      }
    }

    for (int i = 0; i < NUM_PROBES; i++) {
      pgen.reset();
      probeInput.reset();
      // prepare ..
      iterator.reopenProbe(probeInput);
      // .. and do second join
      while (iterator.callWithNextKey(nMatcher[i], collector)) ;

      // assert that each expected match was seen for the second input
      for (Entry<Integer, Collection<TupleMatch>> entry :
          expectedNMatchesMapList.get(i).entrySet()) {
        if (!entry.getValue().isEmpty()) {
          Assert.fail("Collection for key " + entry.getKey() + " is not empty");
        }
      }
    }

    iterator.close();
  }