You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/07/03 09:36:48 UTC

[08/59] [abbrv] lucene-solr:jira/solr-10878: SOLR-10123: Upgraded the Analytics Component to version 2.0

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionCheckedDataReservation.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionCheckedDataReservation.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionCheckedDataReservation.java
new file mode 100644
index 0000000..73287fe
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionCheckedDataReservation.java
@@ -0,0 +1,35 @@
+/*
+ * 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.solr.analytics.stream.reservation;
+
+import java.util.function.BooleanSupplier;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionDataCollector;
+
+/**
+ * A reservation allows a {@link ReductionDataCollector} to specify a piece of data it needs to export from the shard.
+ * This data may, or may not, exist. The {@link ReductionDataCollector} need not check while importing/exporting since
+ * the Reader/Writers handle all checking.
+ */
+public abstract class ReductionCheckedDataReservation<A, E> extends ReductionDataReservation<A, E> {
+  protected final BooleanSupplier exists;
+  
+  protected ReductionCheckedDataReservation(A applier, E extractor, BooleanSupplier exists) {
+    super(applier, extractor);
+    this.exists = exists;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionDataArrayReservation.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionDataArrayReservation.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionDataArrayReservation.java
new file mode 100644
index 0000000..cbeece6
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionDataArrayReservation.java
@@ -0,0 +1,36 @@
+/*
+ * 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.solr.analytics.stream.reservation;
+
+import java.util.function.IntConsumer;
+import java.util.function.IntSupplier;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionDataCollector;
+
+/**
+ * A reservation allows a {@link ReductionDataCollector} to specify an array of data it needs to export from the shard.
+ */
+public abstract class ReductionDataArrayReservation<A, E> extends ReductionDataReservation<A, E> {
+  protected final IntConsumer sizeApplier;
+  protected final IntSupplier sizeExtractor;
+  
+  protected ReductionDataArrayReservation(A applier, IntConsumer sizeApplier, E extractor, IntSupplier sizeExtractor) {
+    super(applier, extractor);
+    this.sizeApplier = sizeApplier;
+    this.sizeExtractor = sizeExtractor;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionDataReservation.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionDataReservation.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionDataReservation.java
new file mode 100644
index 0000000..903016e
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/ReductionDataReservation.java
@@ -0,0 +1,53 @@
+/*
+ * 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.solr.analytics.stream.reservation;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionDataCollector;
+import org.apache.solr.analytics.stream.reservation.read.ReductionDataReader;
+import org.apache.solr.analytics.stream.reservation.write.ReductionDataWriter;
+
+/**
+ * A reservation allows a {@link ReductionDataCollector} to specify a piece of data it needs to export from the shard.
+ */
+public abstract class ReductionDataReservation<A, E> {
+  protected final A applier;
+  protected final E extractor;
+  
+  protected ReductionDataReservation(A applier, E extractor) {
+    this.applier = applier;
+    this.extractor = extractor;
+  }
+  
+  /**
+   * Generate a {@link ReductionDataReader} that merges the set of data this reservation represents.
+   * 
+   * @param input the shard input stream
+   * @return a reader from the given input
+   */
+  public abstract ReductionDataReader<A> createReadStream(DataInput input);
+  
+  /**
+   * Generate a {@link ReductionDataWriter} that exports the set of data this reservation represents.
+   * 
+   * @param output the shard output stream
+   * @return a writer to the given output
+   */
+  public abstract ReductionDataWriter<E> createWriteStream(DataOutput output);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringArrayReservation.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringArrayReservation.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringArrayReservation.java
new file mode 100644
index 0000000..c3c6989
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringArrayReservation.java
@@ -0,0 +1,45 @@
+/*
+ * 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.solr.analytics.stream.reservation;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import org.apache.solr.analytics.stream.reservation.read.StringDataArrayReader;
+import org.apache.solr.analytics.stream.reservation.write.StringDataArrayWriter;
+
+import java.util.function.IntConsumer;
+import java.util.function.IntSupplier;
+
+public class StringArrayReservation extends ReductionDataArrayReservation<Consumer<String>, Supplier<String>> {
+  
+  public StringArrayReservation(Consumer<String> applier, IntConsumer sizeApplier, Supplier<String> extractor, IntSupplier sizeExtractor) {
+    super(applier, sizeApplier, extractor, sizeExtractor);
+  }
+
+  @Override
+  public StringDataArrayReader createReadStream(DataInput input) {
+    return new StringDataArrayReader(input, applier, sizeApplier);
+  }
+
+  @Override
+  public StringDataArrayWriter createWriteStream(DataOutput output) {
+    return new StringDataArrayWriter(output, extractor, sizeExtractor);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringCheckedReservation.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringCheckedReservation.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringCheckedReservation.java
new file mode 100644
index 0000000..29c3614
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringCheckedReservation.java
@@ -0,0 +1,44 @@
+/*
+ * 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.solr.analytics.stream.reservation;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.util.function.Supplier;
+
+import org.apache.solr.analytics.stream.reservation.read.StringCheckedDataReader;
+import org.apache.solr.analytics.stream.reservation.write.StringCheckedDataWriter;
+
+import java.util.function.BooleanSupplier;
+import java.util.function.Consumer;
+
+public class StringCheckedReservation extends ReductionCheckedDataReservation<Consumer<String>, Supplier<String>> {
+  
+  public StringCheckedReservation(Consumer<String> applier, Supplier<String> extractor, BooleanSupplier exists) {
+    super(applier, extractor, exists);
+  }
+
+  @Override
+  public StringCheckedDataReader createReadStream(DataInput input) {
+    return new StringCheckedDataReader(input, applier);
+  }
+
+  @Override
+  public StringCheckedDataWriter createWriteStream(DataOutput output) {
+    return new StringCheckedDataWriter(output, extractor, exists);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringReservation.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringReservation.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringReservation.java
new file mode 100644
index 0000000..2601874
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/StringReservation.java
@@ -0,0 +1,43 @@
+/*
+ * 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.solr.analytics.stream.reservation;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.util.function.Supplier;
+
+import org.apache.solr.analytics.stream.reservation.read.StringDataReader;
+import org.apache.solr.analytics.stream.reservation.write.StringDataWriter;
+
+import java.util.function.Consumer;
+
+public class StringReservation extends ReductionDataReservation<Consumer<String>, Supplier<String>> {
+  
+  public StringReservation(Consumer<String> applier, Supplier<String> extractor) {
+    super(applier, extractor);
+  }
+
+  @Override
+  public StringDataReader createReadStream(DataInput input) {
+    return new StringDataReader(input, applier);
+  }
+
+  @Override
+  public StringDataWriter createWriteStream(DataOutput output) {
+    return new StringDataWriter(output, extractor);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/package-info.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/package-info.java
new file mode 100644
index 0000000..bfd832d
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Reservation classes for a single type of data being stored by one Reduction Data Collector.
+ * These reservations are imported/exported between shards during the streaming process.
+ */
+package org.apache.solr.analytics.stream.reservation;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanCheckedDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanCheckedDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanCheckedDataReader.java
new file mode 100644
index 0000000..49f6f00
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanCheckedDataReader.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.solr.analytics.util.function.BooleanConsumer;
+
+public class BooleanCheckedDataReader extends ReductionCheckedDataReader<BooleanConsumer> {
+  
+  public BooleanCheckedDataReader(DataInput inputStream, BooleanConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void checkedRead() throws IOException {
+    applier.accept(inputStream.readBoolean());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanDataArrayReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanDataArrayReader.java
new file mode 100644
index 0000000..ee929f4
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanDataArrayReader.java
@@ -0,0 +1,36 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.IntConsumer;
+
+import org.apache.solr.analytics.util.function.BooleanConsumer;
+
+public class BooleanDataArrayReader extends ReductionDataArrayReader<BooleanConsumer> {
+  
+  public BooleanDataArrayReader(DataInput inputStream, BooleanConsumer applier, IntConsumer signal) {
+    super(inputStream, applier, signal);
+  }
+  @Override
+  public void read(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      applier.accept(inputStream.readBoolean());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanDataReader.java
new file mode 100644
index 0000000..98e116d
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/BooleanDataReader.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.solr.analytics.util.function.BooleanConsumer;
+
+public class BooleanDataReader extends ReductionDataReader<BooleanConsumer> {
+  
+  public BooleanDataReader(DataInput inputStream, BooleanConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void read() throws IOException {
+    applier.accept(inputStream.readBoolean());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleCheckedDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleCheckedDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleCheckedDataReader.java
new file mode 100644
index 0000000..ec6a6e4
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleCheckedDataReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.DoubleConsumer;
+
+public class DoubleCheckedDataReader extends ReductionCheckedDataReader<DoubleConsumer> {
+  
+  public DoubleCheckedDataReader(DataInput inputStream, DoubleConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void checkedRead() throws IOException {
+    applier.accept(inputStream.readDouble());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleDataArrayReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleDataArrayReader.java
new file mode 100644
index 0000000..5690a45
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleDataArrayReader.java
@@ -0,0 +1,35 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.DoubleConsumer;
+import java.util.function.IntConsumer;
+
+public class DoubleDataArrayReader extends ReductionDataArrayReader<DoubleConsumer> {
+  
+  public DoubleDataArrayReader(DataInput inputStream, DoubleConsumer applier, IntConsumer signal) {
+    super(inputStream, applier, signal);
+  }
+  @Override
+  public void read(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      applier.accept(inputStream.readDouble());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleDataReader.java
new file mode 100644
index 0000000..42de9bf
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/DoubleDataReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.DoubleConsumer;
+
+public class DoubleDataReader extends ReductionDataReader<DoubleConsumer> {
+  
+  public DoubleDataReader(DataInput inputStream, DoubleConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void read() throws IOException {
+    applier.accept(inputStream.readDouble());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatCheckedDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatCheckedDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatCheckedDataReader.java
new file mode 100644
index 0000000..8139cfb
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatCheckedDataReader.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.solr.analytics.util.function.FloatConsumer;
+
+public class FloatCheckedDataReader extends ReductionCheckedDataReader<FloatConsumer> {
+  
+  public FloatCheckedDataReader(DataInput inputStream, FloatConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void checkedRead() throws IOException {
+    applier.accept(inputStream.readFloat());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatDataArrayReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatDataArrayReader.java
new file mode 100644
index 0000000..1157a11
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatDataArrayReader.java
@@ -0,0 +1,36 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.IntConsumer;
+
+import org.apache.solr.analytics.util.function.FloatConsumer;
+
+public class FloatDataArrayReader extends ReductionDataArrayReader<FloatConsumer> {
+  
+  public FloatDataArrayReader(DataInput inputStream, FloatConsumer applier, IntConsumer signal) {
+    super(inputStream, applier, signal);
+  }
+  @Override
+  public void read(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      applier.accept(inputStream.readFloat());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatDataReader.java
new file mode 100644
index 0000000..63067cd
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/FloatDataReader.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.solr.analytics.util.function.FloatConsumer;
+
+public class FloatDataReader extends ReductionDataReader<FloatConsumer> {
+  
+  public FloatDataReader(DataInput inputStream, FloatConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void read() throws IOException {
+    applier.accept(inputStream.readFloat());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntCheckedDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntCheckedDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntCheckedDataReader.java
new file mode 100644
index 0000000..e926e3e
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntCheckedDataReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.IntConsumer;
+
+public class IntCheckedDataReader extends ReductionCheckedDataReader<IntConsumer> {
+  
+  public IntCheckedDataReader(DataInput inputStream, IntConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void checkedRead() throws IOException {
+    applier.accept(inputStream.readInt());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntDataArrayReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntDataArrayReader.java
new file mode 100644
index 0000000..a0c1d86
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntDataArrayReader.java
@@ -0,0 +1,34 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.IntConsumer;
+
+public class IntDataArrayReader extends ReductionDataArrayReader<IntConsumer> {
+  
+  public IntDataArrayReader(DataInput inputStream, IntConsumer applier, IntConsumer signal) {
+    super(inputStream, applier, signal);
+  }
+  @Override
+  public void read(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      applier.accept(inputStream.readInt());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntDataReader.java
new file mode 100644
index 0000000..5b65055
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/IntDataReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.IntConsumer;
+
+public class IntDataReader extends ReductionDataReader<IntConsumer> {
+  
+  public IntDataReader(DataInput inputStream, IntConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void read() throws IOException {
+    applier.accept(inputStream.readInt());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongCheckedDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongCheckedDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongCheckedDataReader.java
new file mode 100644
index 0000000..7d9a7ff
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongCheckedDataReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.LongConsumer;
+
+public class LongCheckedDataReader extends ReductionCheckedDataReader<LongConsumer> {
+  
+  public LongCheckedDataReader(DataInput inputStream, LongConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void checkedRead() throws IOException {
+    applier.accept(inputStream.readLong());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongDataArrayReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongDataArrayReader.java
new file mode 100644
index 0000000..6c09f46
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongDataArrayReader.java
@@ -0,0 +1,35 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.IntConsumer;
+import java.util.function.LongConsumer;
+
+public class LongDataArrayReader extends ReductionDataArrayReader<LongConsumer> {
+  
+  public LongDataArrayReader(DataInput inputStream, LongConsumer applier, IntConsumer signal) {
+    super(inputStream, applier, signal);
+  }
+  @Override
+  public void read(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      applier.accept(inputStream.readLong());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongDataReader.java
new file mode 100644
index 0000000..bf4057d
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/LongDataReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.LongConsumer;
+
+public class LongDataReader extends ReductionDataReader<LongConsumer> {
+  
+  public LongDataReader(DataInput inputStream, LongConsumer applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void read() throws IOException {
+    applier.accept(inputStream.readLong());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionCheckedDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionCheckedDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionCheckedDataReader.java
new file mode 100644
index 0000000..ac98987
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionCheckedDataReader.java
@@ -0,0 +1,54 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+/**
+ * Abstract class to manage the reading and application of data from a {@link DataInput} stream.
+ * The data being read may not exist, so the reader first checks before reading.
+ */
+public abstract class ReductionCheckedDataReader<A> extends ReductionDataReader<A> {
+  
+  public ReductionCheckedDataReader(DataInput inputStream, A applier) {
+    super(inputStream, applier);
+  }
+  
+  @Override
+  /**
+   * Read a piece of data from the input stream and feed it to the applier.
+   * <br>
+   * First checks that the piece of data exists before reading.
+   * 
+   * @throws IOException if an exception occurs while reading from the input stream
+   */
+  public void read() throws IOException {
+    if (inputStream.readBoolean()) {
+      checkedRead();
+    }
+  }
+  
+  /**
+   * Read a piece of data from the input stream and feed it to the applier.
+   * <br>
+   * This piece of data is guaranteed to be there.
+   * 
+   * @throws IOException if an exception occurs while reading from the input stream
+   */
+  protected abstract void checkedRead() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataArrayReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataArrayReader.java
new file mode 100644
index 0000000..2a696d7
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataArrayReader.java
@@ -0,0 +1,54 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.IntConsumer;
+
+/**
+ * Abstract class to manage the reading and application of array data from a {@link DataInput} stream.
+ */
+public abstract class ReductionDataArrayReader<A> extends ReductionDataReader<A> {
+  protected final IntConsumer signal;
+  
+  public ReductionDataArrayReader(DataInput inputStream, A applier, IntConsumer signal) {
+    super(inputStream, applier);
+    
+    this.signal = signal;
+  }
+  
+  @Override
+  /**
+   * Read an array of data from the input stream and feed it to the applier, first signaling the size of the array.
+   * 
+   * @throws IOException if an exception occurs while reading from the input stream
+   */
+  public void read() throws IOException {
+    int size = inputStream.readInt();
+    signal.accept(size);
+    read(size);
+  }
+  
+  /**
+   * Read an array from the input stream, feeding each member to the applier.
+   * 
+   * @param size length of the array to read
+   * @throws IOException if an exception occurs while reading from the input stream
+   */
+  protected abstract void read(int size) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataReader.java
new file mode 100644
index 0000000..ec99621
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataReader.java
@@ -0,0 +1,40 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+/**
+ * Abstract class to manage the reading and application of data from a {@link DataInput} stream.
+ */
+public abstract class ReductionDataReader<A> {
+  protected final DataInput inputStream;
+  protected final A applier;
+  
+  public ReductionDataReader(DataInput inputStream, A applier) {
+    this.inputStream = inputStream;
+    this.applier = applier;
+  }
+  
+  /**
+   * Read a piece of data from the input stream and feed it to the applier.
+   * 
+   * @throws IOException if an exception occurs while reading from the input stream
+   */
+  public abstract void read() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringCheckedDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringCheckedDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringCheckedDataReader.java
new file mode 100644
index 0000000..400e990
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringCheckedDataReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.Consumer;
+
+public class StringCheckedDataReader extends ReductionCheckedDataReader<Consumer<String>> {
+  
+  public StringCheckedDataReader(DataInput inputStream, Consumer<String> applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void checkedRead() throws IOException {
+    applier.accept(inputStream.readUTF());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringDataArrayReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringDataArrayReader.java
new file mode 100644
index 0000000..048b84f
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringDataArrayReader.java
@@ -0,0 +1,35 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.IntConsumer;
+
+public class StringDataArrayReader extends ReductionDataArrayReader<Consumer<String>> {
+  
+  public StringDataArrayReader(DataInput inputStream, Consumer<String> applier, IntConsumer signal) {
+    super(inputStream, applier, signal);
+  }
+  @Override
+  public void read(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      applier.accept(inputStream.readUTF());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringDataReader.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringDataReader.java
new file mode 100644
index 0000000..d4d6bc4
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/StringDataReader.java
@@ -0,0 +1,34 @@
+/*
+ * 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.solr.analytics.stream.reservation.read;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.function.Consumer;
+
+public class StringDataReader extends ReductionDataReader<Consumer<String>> {
+  
+  public StringDataReader(DataInput inputStream, Consumer<String> applier) {
+    super(inputStream, applier);
+  }
+  @Override
+  public void read() throws IOException {
+    if (inputStream.readBoolean()) {
+      applier.accept(inputStream.readUTF());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/package-info.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/package-info.java
new file mode 100644
index 0000000..26bb7c5
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/read/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Reading classes for a single type of data being stored by one Reduction Data Collector.
+ * These writers are used to import data from shards during the streaming process.
+ */
+package org.apache.solr.analytics.stream.reservation.read;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanCheckedDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanCheckedDataWriter.java
new file mode 100644
index 0000000..6b4c1b7
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanCheckedDataWriter.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.BooleanSupplier;
+
+public class BooleanCheckedDataWriter extends ReductionCheckedDataWriter<BooleanSupplier> {
+  
+  public BooleanCheckedDataWriter(DataOutput output, BooleanSupplier extractor, BooleanSupplier existsSupplier) {
+    super(output, extractor, existsSupplier);
+  }
+
+  @Override
+  public void checkedWrite() throws IOException {
+    output.writeBoolean(extractor.getAsBoolean());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataArrayWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataArrayWriter.java
new file mode 100644
index 0000000..c188770
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataArrayWriter.java
@@ -0,0 +1,36 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.BooleanSupplier;
+import java.util.function.IntSupplier;
+
+public class BooleanDataArrayWriter extends ReductionDataArrayWriter<BooleanSupplier> {
+
+  public BooleanDataArrayWriter(DataOutput output, BooleanSupplier extractor, IntSupplier sizeSupplier) {
+    super(output, extractor, sizeSupplier);
+  }
+  
+  @Override
+  public void write(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      output.writeBoolean(extractor.getAsBoolean());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataWriter.java
new file mode 100644
index 0000000..f921a5d
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataWriter.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.BooleanSupplier;
+
+public class BooleanDataWriter extends ReductionDataWriter<BooleanSupplier> {
+  
+  public BooleanDataWriter(DataOutput output, BooleanSupplier extractor) {
+    super(output, extractor);
+  }
+
+  @Override
+  public void write() throws IOException {
+    output.writeBoolean(extractor.getAsBoolean());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleCheckedDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleCheckedDataWriter.java
new file mode 100644
index 0000000..376afee
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleCheckedDataWriter.java
@@ -0,0 +1,34 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.BooleanSupplier;
+import java.util.function.DoubleSupplier;
+
+public class DoubleCheckedDataWriter extends ReductionCheckedDataWriter<DoubleSupplier> {
+  
+  public DoubleCheckedDataWriter(DataOutput output, DoubleSupplier extractor, BooleanSupplier existsSupplier) {
+    super(output, extractor, existsSupplier);
+  }
+
+  @Override
+  public void checkedWrite() throws IOException {
+    output.writeDouble(extractor.getAsDouble());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataArrayWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataArrayWriter.java
new file mode 100644
index 0000000..830ceee
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataArrayWriter.java
@@ -0,0 +1,36 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+
+public class DoubleDataArrayWriter extends ReductionDataArrayWriter<DoubleSupplier> {
+
+  public DoubleDataArrayWriter(DataOutput output, DoubleSupplier extractor, IntSupplier sizeSupplier) {
+    super(output, extractor, sizeSupplier);
+  }
+  
+  @Override
+  public void write(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      output.writeDouble(extractor.getAsDouble());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataWriter.java
new file mode 100644
index 0000000..074b859
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataWriter.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.DoubleSupplier;
+
+public class DoubleDataWriter extends ReductionDataWriter<DoubleSupplier> {
+  
+  public DoubleDataWriter(DataOutput output, DoubleSupplier extractor) {
+    super(output, extractor);
+  }
+
+  @Override
+  public void write() throws IOException {
+    output.writeDouble(extractor.getAsDouble());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatCheckedDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatCheckedDataWriter.java
new file mode 100644
index 0000000..69c74b9
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatCheckedDataWriter.java
@@ -0,0 +1,35 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.BooleanSupplier;
+
+import org.apache.solr.analytics.util.function.FloatSupplier;
+
+public class FloatCheckedDataWriter extends ReductionCheckedDataWriter<FloatSupplier> {
+  
+  public FloatCheckedDataWriter(DataOutput output, FloatSupplier extractor, BooleanSupplier existsSupplier) {
+    super(output, extractor, existsSupplier);
+  }
+
+  @Override
+  public void checkedWrite() throws IOException {
+    output.writeFloat(extractor.getAsFloat());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatDataArrayWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatDataArrayWriter.java
new file mode 100644
index 0000000..0fd0d25
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatDataArrayWriter.java
@@ -0,0 +1,37 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.IntSupplier;
+
+import org.apache.solr.analytics.util.function.FloatSupplier;
+
+public class FloatDataArrayWriter extends ReductionDataArrayWriter<FloatSupplier> {
+
+  public FloatDataArrayWriter(DataOutput output, FloatSupplier extractor, IntSupplier sizeSupplier) {
+    super(output, extractor, sizeSupplier);
+  }
+  
+  @Override
+  public void write(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      output.writeFloat(extractor.getAsFloat());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatDataWriter.java
new file mode 100644
index 0000000..bc23f21
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/FloatDataWriter.java
@@ -0,0 +1,34 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.solr.analytics.util.function.FloatSupplier;
+
+public class FloatDataWriter extends ReductionDataWriter<FloatSupplier> {
+  
+  public FloatDataWriter(DataOutput output, FloatSupplier extractor) {
+    super(output, extractor);
+  }
+
+  @Override
+  public void write() throws IOException {
+    output.writeFloat(extractor.getAsFloat());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntCheckedDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntCheckedDataWriter.java
new file mode 100644
index 0000000..fd55cf7
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntCheckedDataWriter.java
@@ -0,0 +1,34 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.BooleanSupplier;
+import java.util.function.IntSupplier;
+
+public class IntCheckedDataWriter extends ReductionCheckedDataWriter<IntSupplier> {
+  
+  public IntCheckedDataWriter(DataOutput output, IntSupplier extractor, BooleanSupplier existsSupplier) {
+    super(output, extractor, existsSupplier);
+  }
+
+  @Override
+  public void checkedWrite() throws IOException {
+    output.writeInt(extractor.getAsInt());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntDataArrayWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntDataArrayWriter.java
new file mode 100644
index 0000000..144b27f
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntDataArrayWriter.java
@@ -0,0 +1,35 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.IntSupplier;
+
+public class IntDataArrayWriter extends ReductionDataArrayWriter<IntSupplier> {
+
+  public IntDataArrayWriter(DataOutput output, IntSupplier extractor, IntSupplier sizeSupplier) {
+    super(output, extractor, sizeSupplier);
+  }
+  
+  @Override
+  public void write(int size) throws IOException {
+    for (int i = 0; i < size; ++i) {
+      output.writeInt(extractor.getAsInt());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntDataWriter.java
new file mode 100644
index 0000000..fd2420a
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/IntDataWriter.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.IntSupplier;
+
+public class IntDataWriter extends ReductionDataWriter<IntSupplier> {
+  
+  public IntDataWriter(DataOutput output, IntSupplier extractor) {
+    super(output, extractor);
+  }
+
+  @Override
+  public void write() throws IOException {
+    output.writeInt(extractor.getAsInt());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/LongCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/LongCheckedDataWriter.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/LongCheckedDataWriter.java
new file mode 100644
index 0000000..e148e40
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/reservation/write/LongCheckedDataWriter.java
@@ -0,0 +1,34 @@
+/*
+ * 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.solr.analytics.stream.reservation.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.BooleanSupplier;
+import java.util.function.LongSupplier;
+
+public class LongCheckedDataWriter extends ReductionCheckedDataWriter<LongSupplier> {
+  
+  public LongCheckedDataWriter(DataOutput output, LongSupplier extractor, BooleanSupplier existsSupplier) {
+    super(output, extractor, existsSupplier);
+  }
+
+  @Override
+  public void checkedWrite() throws IOException {
+    output.writeLong(extractor.getAsLong());
+  }
+}
\ No newline at end of file