You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/11/02 11:16:35 UTC

[26/56] lucene-solr:jira/gradle: Add :solr:contrib:analytics module

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataArrayReader.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataArrayReader.java
new file mode 100644
index 0000000..2a696d7
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataReader.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/ReductionDataReader.java
new file mode 100644
index 0000000..ec99621
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringCheckedDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringCheckedDataReader.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringCheckedDataReader.java
new file mode 100644
index 0000000..400e990
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringDataArrayReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringDataArrayReader.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringDataArrayReader.java
new file mode 100644
index 0000000..048b84f
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringDataReader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringDataReader.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/StringDataReader.java
new file mode 100644
index 0000000..d4d6bc4
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/read/package-info.java b/solr/contrib/analytics/src/main/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/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanCheckedDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanCheckedDataWriter.java
new file mode 100644
index 0000000..6b4c1b7
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataArrayWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataArrayWriter.java
new file mode 100644
index 0000000..c188770
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/BooleanDataWriter.java
new file mode 100644
index 0000000..f921a5d
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleCheckedDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleCheckedDataWriter.java
new file mode 100644
index 0000000..376afee
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataArrayWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataArrayWriter.java
new file mode 100644
index 0000000..830ceee
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/DoubleDataWriter.java
new file mode 100644
index 0000000..074b859
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatCheckedDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatCheckedDataWriter.java
new file mode 100644
index 0000000..69c74b9
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatDataArrayWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatDataArrayWriter.java
new file mode 100644
index 0000000..0fd0d25
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/FloatDataWriter.java
new file mode 100644
index 0000000..bc23f21
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntCheckedDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntCheckedDataWriter.java
new file mode 100644
index 0000000..fd55cf7
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntDataArrayWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntDataArrayWriter.java
new file mode 100644
index 0000000..144b27f
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/IntDataWriter.java
new file mode 100644
index 0000000..fd2420a
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/LongCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/LongCheckedDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/LongCheckedDataWriter.java
new file mode 100644
index 0000000..e148e40
--- /dev/null
+++ b/solr/contrib/analytics/src/main/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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/LongDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/LongDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/LongDataWriter.java
new file mode 100644
index 0000000..3b8af52
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/LongDataWriter.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.LongSupplier;
+
+public class LongDataWriter extends ReductionDataWriter<LongSupplier> {
+  
+  public LongDataWriter(DataOutput output, LongSupplier extractor) {
+    super(output, extractor);
+  }
+
+  @Override
+  public void write() throws IOException {
+    output.writeLong(extractor.getAsLong());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionCheckedDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionCheckedDataWriter.java
new file mode 100644
index 0000000..a5a2273
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionCheckedDataWriter.java
@@ -0,0 +1,60 @@
+/*
+ * 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;
+
+/**
+ * Abstract class to manage the extraction and writing of data to a {@link DataOutput} stream.
+ * The data being written may not exist, so the writer first writes whether the data exists before writing the data.
+ */
+public abstract class ReductionCheckedDataWriter<C> extends ReductionDataWriter<C> {
+  private final BooleanSupplier existsSupplier;
+  
+  public ReductionCheckedDataWriter(DataOutput output, C extractor, BooleanSupplier existsSupplier) {
+    super(output, extractor);
+    
+    this.existsSupplier = existsSupplier;
+  }
+  
+  /**
+   * Write a piece of data, retrieved from the extractor, to the output stream.
+   * <br>
+   * First writes whether the data exists, then if it does exists writes the data.
+   * 
+   * @throws IOException if an exception occurs while writing to the output stream
+   */
+  @Override
+  public void write() throws IOException {
+    boolean exists = existsSupplier.getAsBoolean();
+    output.writeBoolean(exists);
+    if (exists) {
+      checkedWrite();
+    }
+  }
+  
+  /**
+   * Write a piece of data, retrieved from the extractor, to the output stream.
+   * <br>
+   * The data being written is guaranteed to exist.
+   * 
+   * @throws IOException if an exception occurs while writing to the output stream
+   */
+  protected abstract void checkedWrite() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionDataArrayWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionDataArrayWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionDataArrayWriter.java
new file mode 100644
index 0000000..29ba77e
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionDataArrayWriter.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.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.function.IntSupplier;
+
+/**
+ * Abstract class to manage the extraction and writing of array data to a {@link DataOutput} stream.
+ */
+public abstract class ReductionDataArrayWriter<C> extends ReductionDataWriter<C> {
+  private final IntSupplier sizeSupplier;
+  
+  public ReductionDataArrayWriter(DataOutput output, C extractor, IntSupplier sizeSupplier) {
+    super(output, extractor);
+    
+    this.sizeSupplier = sizeSupplier;
+  }
+  
+  /**
+   * Write an array of data, retrieved from the extractor, and its size, received from the sizeSupplier, to the output stream.
+   * 
+   * @throws IOException if an exception occurs while writing to the output stream
+   */
+  @Override
+  public void write() throws IOException {
+    int size = sizeSupplier.getAsInt();
+    output.writeInt(size);
+    write(size);
+  }
+  
+  /**
+   * Write an array of data, retrieved from the extractor, with the given size to the output stream.
+   * 
+   * @throws IOException if an exception occurs while writing to the output stream
+   */
+  protected abstract void write(int size) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionDataWriter.java
new file mode 100644
index 0000000..504a2be
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/ReductionDataWriter.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.write;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Abstract public class to manage the extraction and writing of data to a {@link DataOutput} stream.
+ */
+public abstract class ReductionDataWriter<E> {
+  protected final DataOutput output;
+  protected final E extractor;
+  
+  public ReductionDataWriter(DataOutput output, E extractor) {
+    this.output = output;
+    this.extractor = extractor;
+  }
+
+  /**
+   * Write a piece of data, retrieved from the extractor, to the output stream.
+   * 
+   * @throws IOException if an exception occurs while writing to the output stream
+   */
+  public abstract void write() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/StringCheckedDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/StringCheckedDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/StringCheckedDataWriter.java
new file mode 100644
index 0000000..6560a8f
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/StringCheckedDataWriter.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.Supplier;
+
+public class StringCheckedDataWriter extends ReductionCheckedDataWriter<Supplier<String>> {
+  
+  public StringCheckedDataWriter(DataOutput output, Supplier<String> extractor, BooleanSupplier existsSupplier) {
+    super(output, extractor, existsSupplier);
+  }
+
+  @Override
+  public void checkedWrite() throws IOException {
+    output.writeUTF(extractor.get());
+  }
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/StringDataWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/StringDataWriter.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/StringDataWriter.java
new file mode 100644
index 0000000..4aac07c
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/StringDataWriter.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.Supplier;
+
+public class StringDataWriter extends ReductionDataWriter<Supplier<String>> {
+  
+  public StringDataWriter(DataOutput output, Supplier<String> extractor) {
+    super(output, extractor);
+  }
+
+  @Override
+  public void write() throws IOException {
+    String temp = extractor.get();
+    output.writeBoolean(temp != null);
+    if (temp != null) {
+      output.writeUTF(temp);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/package-info.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/package-info.java
new file mode 100644
index 0000000..53a5168
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/stream/reservation/write/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.
+ */
+ 
+/** 
+ * Writing classes for a single type of data being stored by one Reduction Data Collector.
+ * These writers are used to export data between shards during the streaming process.
+ */
+package org.apache.solr.analytics.stream.reservation.write;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/AnalyticsResponseHeadings.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/AnalyticsResponseHeadings.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/AnalyticsResponseHeadings.java
new file mode 100644
index 0000000..00e0afb
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/AnalyticsResponseHeadings.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.util;
+
+/**
+ * Holds the headers for analytics responses.
+ */
+public class AnalyticsResponseHeadings {
+
+  public static final String COMPLETED_HEADER = "analytics_response";
+  public static final String RESULTS = "results";
+  public static final String GROUPINGS = "groupings";
+  public static final String FACET_VALUE = "value";
+  public static final String PIVOT_NAME = "pivot";
+  public static final String PIVOT_CHILDREN = "children";
+  
+  // Old Olap-style
+  public static final String COMPLETED_OLD_HEADER = "stats";
+  public static final String FIELD_FACETS = "fieldFacets";
+  public static final String RANGE_FACETS = "rangeFacets";
+  public static final String QUERY_FACETS = "queryFacets";
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/FacetRangeGenerator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/FacetRangeGenerator.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/FacetRangeGenerator.java
new file mode 100644
index 0000000..d2fb887
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/FacetRangeGenerator.java
@@ -0,0 +1,352 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.solr.analytics.facet.RangeFacet;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.FacetParams.FacetRangeInclude;
+import org.apache.solr.common.params.FacetParams.FacetRangeOther;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.NumericFieldType;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.util.DateMathParser;
+
+
+/**
+ * Calculates a set of {@link FacetRange}s for a given {@link RangeFacet}.
+ */
+public abstract class FacetRangeGenerator<T extends Comparable<T>> {
+  protected final SchemaField field;
+  protected final RangeFacet rangeFacet;
+  
+  public FacetRangeGenerator(final RangeFacet rangeFacet) {
+    this.field = rangeFacet.getField();
+    this.rangeFacet = rangeFacet;
+  }
+
+  /**
+   * Formats a Range endpoint for use as a range label name in the response.
+   * Default Impl just uses toString()
+   */
+  public String formatValue(final T val) {
+    return val.toString();
+  }
+  
+  /**
+   * Parses a String param into an Range endpoint value throwing 
+   * a useful exception if not possible
+   */
+  public final T getValue(final String rawval) {
+    try {
+      return parseVal(rawval);
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can't parse value "+rawval+" for field: " + field.getName(), e);
+    }
+  }
+  
+  /**
+   * Parses a String param into an Range endpoint. 
+   * Can throw a low level format exception as needed.
+   */
+  protected abstract T parseVal(final String rawval) throws java.text.ParseException;
+
+  /** 
+   * Parses a String param into a value that represents the gap and 
+   * can be included in the response, throwing 
+   * a useful exception if not possible.
+   *
+   * Note: uses Object as the return type instead of T for things like 
+   * Date where gap is just a DateMathParser string 
+   */
+  public final Object getGap(final String gap) {
+    try {
+      return parseGap(gap);
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can't parse gap "+gap+" for field: " + field.getName(), e);
+    }
+  }
+
+  /**
+   * Parses a String param into a value that represents the gap and 
+   * can be included in the response. 
+   * Can throw a low level format exception as needed.
+   *
+   * Default Impl calls parseVal
+   */
+  protected Object parseGap(final String rawval) throws java.text.ParseException {
+    return parseVal(rawval);
+  }
+
+  /**
+   * Adds the String gap param to a low Range endpoint value to determine 
+   * the corrisponding high Range endpoint value, throwing 
+   * a useful exception if not possible.
+   */
+  public final T addGap(T value, String gap) {
+    try {
+      return parseAndAddGap(value, gap);
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can't add gap "+gap+" to value " + value + " for field: " + field.getName(), e);
+    }
+  }
+  
+  /**
+   * Adds the String gap param to a low Range endpoint value to determine 
+   * the corrisponding high Range endpoint value.
+   * Can throw a low level format exception as needed.
+   */
+  protected abstract T parseAndAddGap(T value, String gap) throws java.text.ParseException;
+
+  public static class FacetRange {
+    public final String name;
+    public final String lower;
+    public final String upper;
+    public final boolean includeLower;
+    public final boolean includeUpper;
+    private final String facetValue;
+    
+    public FacetRange(String name, String lower, String upper, boolean includeLower, boolean includeUpper) {
+      this.name = name;
+      this.lower = lower;
+      this.upper = upper;
+      this.includeLower = includeLower;
+      this.includeUpper = includeUpper;
+      
+      String value = "(*";
+      if (lower != null) {
+        value = (includeLower ? "[" : "(") + lower;
+      }
+      value += " TO ";
+      if (upper == null) {
+        value += "*)";
+      } else {
+        value += upper + (includeUpper? "]" : ")");
+      }
+      facetValue = value;
+    }
+    
+    @Override
+    public String toString() {
+        return facetValue;
+    }
+  }
+  
+  public List<FacetRange> getRanges(){
+
+    final T start = getValue(rangeFacet.getStart());
+    T end = getValue(rangeFacet.getEnd()); // not final, hardend may change this
+    
+    if( end.compareTo(start) < 0 ){
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "range facet 'end' comes before 'start': "+end+" < "+start);
+    }
+    
+    // explicitly return the gap.  compute this early so we are more 
+    // likely to catch parse errors before attempting math
+    final List<String> gaps = rangeFacet.getGaps();
+    String gap = gaps.get(0);
+    
+    final EnumSet<FacetRangeInclude> include = rangeFacet.getInclude();
+        
+    T low = start;
+    
+    List<FacetRange> ranges = new ArrayList<>();
+    
+    int gapCounter = 0;
+    
+    while (low.compareTo(end) < 0) {
+      if (gapCounter<gaps.size()) {
+        gap = gaps.get(gapCounter++);
+      }
+      T high = addGap(low,gap);
+      if (end.compareTo(high) < 0) {
+        if (rangeFacet.isHardEnd()){
+          high = end;
+        } else {
+          end = high;
+        }
+      }
+      
+      if (high.compareTo(low) < 0) {
+        throw new SolrException (SolrException.ErrorCode.BAD_REQUEST, "range facet infinite loop (is gap negative? did the math overflow?)");
+      }
+      
+      if (high.compareTo(low) == 0) {
+        throw new SolrException (SolrException.ErrorCode.BAD_REQUEST, "range facet infinite loop: gap is either zero, or too small relative start/end and caused underflow: " + low + " + " + gap + " = " + high );
+      }
+      
+      final boolean includeLower = (include.contains(FacetRangeInclude.ALL) ||
+                                    include.contains(FacetRangeInclude.LOWER) ||
+                                   (include.contains(FacetRangeInclude.EDGE) && 
+                                   0 == low.compareTo(start)));
+      final boolean includeUpper = (include.contains(FacetRangeInclude.ALL) ||
+                                    include.contains(FacetRangeInclude.UPPER) ||
+                                   (include.contains(FacetRangeInclude.EDGE) && 
+                                   0 == high.compareTo(end)));
+      
+      final String lowS = formatValue(low);
+      final String highS = formatValue(high);
+
+      ranges.add( new FacetRange(lowS,lowS,highS,includeLower,includeUpper) );
+      low = high;
+    }
+    
+    final Set<FacetRangeOther> others = rangeFacet.getOthers();
+    if (null != others && 0 < others.size() ) {
+      
+      // no matter what other values are listed, we don't do
+      // anything if "none" is specified.
+      if( !others.contains(FacetRangeOther.NONE) ) {
+        
+        boolean all = others.contains(FacetRangeOther.ALL);
+
+        if (all || others.contains(FacetRangeOther.BEFORE)) {
+          // include upper bound if "outer" or if first gap doesn't already include it
+          ranges.add( new FacetRange(FacetRangeOther.BEFORE.toString(), 
+                                        null, formatValue(start), false, include.contains(FacetRangeInclude.OUTER) || include.contains(FacetRangeInclude.ALL) ||
+                                                            !(include.contains(FacetRangeInclude.LOWER) || include.contains(FacetRangeInclude.EDGE)) ) );
+          
+        }
+        if (all || others.contains(FacetRangeOther.AFTER)) {
+          // include lower bound if "outer" or if last gap doesn't already include it
+          ranges.add( new FacetRange(FacetRangeOther.AFTER.toString(), 
+                                        formatValue(end), null, include.contains(FacetRangeInclude.OUTER) || include.contains(FacetRangeInclude.ALL) ||
+                                                   !(include.contains(FacetRangeInclude.UPPER) || include.contains(FacetRangeInclude.EDGE)), false) );
+        }
+        if (all || others.contains(FacetRangeOther.BETWEEN)) {
+          ranges.add( new FacetRange(FacetRangeOther.BETWEEN.toString(), formatValue(start), formatValue(end),
+                                        include.contains(FacetRangeInclude.LOWER) || include.contains(FacetRangeInclude.EDGE) || include.contains(FacetRangeInclude.ALL),
+                                        include.contains(FacetRangeInclude.UPPER) || include.contains(FacetRangeInclude.EDGE) || include.contains(FacetRangeInclude.ALL)) );
+        }
+      }
+      
+    }
+  
+    return ranges;
+  }
+  
+  public static FacetRangeGenerator<? extends Comparable<?>> create(RangeFacet rangeFacet){
+    final SchemaField sf = rangeFacet.getField();
+    final FieldType ft = sf.getType();
+    final FacetRangeGenerator<?> calc;
+    if (ft instanceof NumericFieldType) {
+      switch (ft.getNumberType()) {
+        case FLOAT:
+          calc = new FloatFacetRangeGenerator(rangeFacet);
+          break;
+        case DOUBLE:
+          calc = new DoubleFacetRangeGenerator(rangeFacet);
+          break;
+        case INTEGER:
+          calc = new IntegerFacetRangeGenerator(rangeFacet);
+          break;
+        case LONG:
+          calc = new LongFacetRangeGenerator(rangeFacet);
+          break;
+        case DATE:
+          calc = new DateFacetRangeGenerator(rangeFacet, null);
+          break;
+        default:
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to range facet on numeric field of unexpected type: " + sf.getName());
+      }
+    } else {
+      throw new SolrException (SolrException.ErrorCode.BAD_REQUEST, "Unable to range facet on non-numeric field: " + sf);
+    } 
+    return calc;
+  }
+}
+class IntegerFacetRangeGenerator extends FacetRangeGenerator<Integer> {
+  public IntegerFacetRangeGenerator(final RangeFacet rangeFacet) { super(rangeFacet); }
+
+  @Override
+  protected Integer parseVal(String rawval) {
+    return Integer.valueOf(rawval);
+  }
+  @Override
+  public Integer parseAndAddGap(Integer value, String gap) {
+    return value.intValue() + Integer.valueOf(gap).intValue();
+  }
+}
+class LongFacetRangeGenerator extends FacetRangeGenerator<Long> {
+  public LongFacetRangeGenerator(final RangeFacet rangeFacet) { super(rangeFacet); }
+
+  @Override
+  protected Long parseVal(String rawval) {
+    return Long.valueOf(rawval);
+  }
+  @Override
+  public Long parseAndAddGap(Long value, String gap) {
+    return value.longValue() + Long.valueOf(gap).longValue();
+  }
+}
+
+class FloatFacetRangeGenerator extends FacetRangeGenerator<Float> {
+  public FloatFacetRangeGenerator(final RangeFacet rangeFacet) { super(rangeFacet); }
+
+  @Override
+  protected Float parseVal(String rawval) {
+    return Float.valueOf(rawval);
+  }
+  @Override
+  public Float parseAndAddGap(Float value, String gap) {
+    return value.floatValue() + Float.valueOf(gap).floatValue();
+  }
+}
+
+class DoubleFacetRangeGenerator extends FacetRangeGenerator<Double> {
+  public DoubleFacetRangeGenerator(final RangeFacet rangeFacet) { super(rangeFacet); }
+
+  @Override
+  protected Double parseVal(String rawval) {
+    return Double.valueOf(rawval);
+  }
+  @Override
+  public Double parseAndAddGap(Double value, String gap) {
+    return value.doubleValue() + Double.valueOf(gap).doubleValue();
+  }
+}
+class DateFacetRangeGenerator extends FacetRangeGenerator<Date> {
+  private final Date now;
+  public DateFacetRangeGenerator(final RangeFacet rangeFacet, final Date now) { 
+    super(rangeFacet); 
+    this.now = now;
+  }
+  
+  @Override
+  public String formatValue(Date val) {
+    return val.toInstant().toString();
+  }
+  @Override
+  protected Date parseVal(String rawval) {
+    return DateMathParser.parseMath(now, rawval);
+  }
+  @Override
+  protected Object parseGap(final String rawval) {
+    return rawval;
+  }
+  @Override
+  public Date parseAndAddGap(Date value, String gap) throws java.text.ParseException {
+    final DateMathParser dmp = new DateMathParser();
+    dmp.setNow(value);
+    return dmp.parseMath(gap);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/MedianCalculator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/MedianCalculator.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/MedianCalculator.java
new file mode 100644
index 0000000..541cff0
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/util/MedianCalculator.java
@@ -0,0 +1,131 @@
+/*
+ * 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.util;
+
+import java.util.List;
+
+/**
+ * Only used for testing.
+ * Medians are calculated with the {@link OrdinalCalculator} for actual analytics requests.
+ */
+public class MedianCalculator {
+
+  /**
+   * Calculates the median of the given list of numbers.
+   *
+   * @param list A list of {@link Comparable} {@link Number} objects
+   * @return The median of the given list as a double.
+   */
+  public static <T extends Number & Comparable<T>> double getMedian(List<T> list) {
+    int size = list.size() - 1;
+    if (size == -1) {
+      return 0;
+    }
+
+    select(list, .5 * size, 0, size);
+
+    int firstIdx = (int) (Math.floor(.5 * size));
+    int secondIdx = (firstIdx <= size && size % 2 == 1) ? firstIdx + 1 : firstIdx;
+    double result = list.get(firstIdx).doubleValue() * .5 + list.get(secondIdx).doubleValue() * .5;
+
+    return result;
+  }
+
+  private static <T extends Comparable<T>> void select(List<T> list, double place, int begin, int end) {
+    T split;
+    if (end - begin < 10) {
+      split = list.get((int) (Math.random() * (end - begin + 1)) + begin);
+    } else {
+      split = split(list, begin, end);
+    }
+
+    Point result = partition(list, begin, end, split);
+
+    if (place < result.low) {
+      select(list, place, begin, result.low);
+    } else if (place > result.high) {
+      select(list, place, result.high, end);
+    } else {
+      if (result.low == (int) (Math.floor(place)) && result.low > begin) {
+        select(list, result.low, begin, result.low);
+      }
+      if (result.high == (int) (Math.ceil(place)) && result.high < end) {
+        select(list, result.high, result.high, end);
+      }
+    }
+  }
+
+  private static <T extends Comparable<T>> T split(List<T> list, int begin, int end) {
+    T temp;
+    int num = (end - begin + 1);
+    int recursiveSize = (int) Math.sqrt((double) num);
+    int step = num / recursiveSize;
+    for (int i = 1; i < recursiveSize; i++) {
+      int swapFrom = i * step + begin;
+      int swapTo = i + begin;
+      temp = list.get(swapFrom);
+      list.set(swapFrom, list.get(swapTo));
+      list.set(swapTo, temp);
+    }
+    recursiveSize--;
+    select(list, recursiveSize / 2 + begin, begin, recursiveSize + begin);
+    return list.get(recursiveSize / 2 + begin);
+  }
+
+  private static <T extends Comparable<T>> Point partition(List<T> list, int begin, int end, T indexElement) {
+    T temp;
+    int left, right;
+    for (left = begin, right = end; left < right; left++, right--) {
+      while (list.get(left).compareTo(indexElement) < 0) {
+        left++;
+      }
+      while (right != begin - 1 && list.get(right).compareTo(indexElement) >= 0) {
+        right--;
+      }
+      if (right <= left) {
+        left--;
+        right++;
+        break;
+      }
+      temp = list.get(left);
+      list.set(left, list.get(right));
+      list.set(right, temp);
+    }
+    while (left != begin - 1 && list.get(left).compareTo(indexElement) >= 0) {
+      left--;
+    }
+    while (right != end + 1 && list.get(right).compareTo(indexElement) <= 0) {
+      right++;
+    }
+    int rightMove = right + 1;
+    while (rightMove < end + 1) {
+      if (list.get(rightMove).equals(indexElement)) {
+        temp = list.get(rightMove);
+        list.set(rightMove, list.get(right));
+        list.set(right, temp);
+        do {
+          right++;
+        } while (list.get(right).equals(indexElement));
+        if (rightMove <= right) {
+          rightMove = right;
+        }
+      }
+      rightMove++;
+    }
+    return new Point(left, right);
+  }
+}