You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ro...@apache.org on 2021/09/03 10:12:33 UTC

[iotdb] 01/01: new access data structures

This is an automated email from the ASF dual-hosted git repository.

rong pushed a commit to branch nested-operations
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit f47791acebcd246b23600859dd9b67ed80f07b50
Author: Steve Yurong Su <ro...@apache.org>
AuthorDate: Fri Sep 3 17:07:04 2021 +0800

    new access data structures
---
 .../apache/iotdb/db/query/udf/api/access/Row.java  | 18 ++--
 ...ticSerializableTVListBackedSingleColumnRow.java | 99 ++++++++++++++++++++++
 ...SerializableTVListBackedSingleColumnWindow.java | 89 +++++++++++++++++++
 ...bleTVListBackedSingleColumnWindowIterator.java} | 30 +++----
 .../LayerPointReaderBackedSingleColumnRow.java     | 91 ++++++++++++++++++++
 .../access/{RowImpl.java => MultiColumnRow.java}   |  4 +-
 .../{RowWindowImpl.java => MultiColumnWindow.java} | 16 ++--
 ...torImpl.java => MultiColumnWindowIterator.java} |  8 +-
 .../db/query/udf/core/layer/IntermediateLayer.java | 42 ++++++++-
 9 files changed, 358 insertions(+), 39 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/api/access/Row.java b/server/src/main/java/org/apache/iotdb/db/query/udf/api/access/Row.java
index 5939a5d..eaf1f7c 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/api/access/Row.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/api/access/Row.java
@@ -22,6 +22,8 @@ package org.apache.iotdb.db.query.udf.api.access;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
 
+import java.io.IOException;
+
 public interface Row {
 
   /**
@@ -29,7 +31,7 @@ public interface Row {
    *
    * @return timestamp
    */
-  long getTime();
+  long getTime() throws IOException;
 
   /**
    * Returns the int value at the specified column in this row.
@@ -39,7 +41,7 @@ public interface Row {
    * @param columnIndex index of the specified column
    * @return the int value at the specified column in this row
    */
-  int getInt(int columnIndex);
+  int getInt(int columnIndex) throws IOException;
 
   /**
    * Returns the long value at the specified column in this row.
@@ -49,7 +51,7 @@ public interface Row {
    * @param columnIndex index of the specified column
    * @return the long value at the specified column in this row
    */
-  long getLong(int columnIndex);
+  long getLong(int columnIndex) throws IOException;
 
   /**
    * Returns the float value at the specified column in this row.
@@ -59,7 +61,7 @@ public interface Row {
    * @param columnIndex index of the specified column
    * @return the float value at the specified column in this row
    */
-  float getFloat(int columnIndex);
+  float getFloat(int columnIndex) throws IOException;
 
   /**
    * Returns the double value at the specified column in this row.
@@ -70,7 +72,7 @@ public interface Row {
    * @param columnIndex index of the specified column
    * @return the double value at the specified column in this row
    */
-  double getDouble(int columnIndex);
+  double getDouble(int columnIndex) throws IOException;
 
   /**
    * Returns the boolean value at the specified column in this row.
@@ -81,7 +83,7 @@ public interface Row {
    * @param columnIndex index of the specified column
    * @return the boolean value at the specified column in this row
    */
-  boolean getBoolean(int columnIndex);
+  boolean getBoolean(int columnIndex) throws IOException;
 
   /**
    * Returns the Binary value at the specified column in this row.
@@ -91,7 +93,7 @@ public interface Row {
    * @param columnIndex index of the specified column
    * @return the Binary value at the specified column in this row
    */
-  Binary getBinary(int columnIndex);
+  Binary getBinary(int columnIndex) throws IOException;
 
   /**
    * Returns the String value at the specified column in this row.
@@ -101,7 +103,7 @@ public interface Row {
    * @param columnIndex index of the specified column
    * @return the String value at the specified column in this row
    */
-  String getString(int columnIndex);
+  String getString(int columnIndex) throws IOException;
 
   /**
    * Returns the actual data type of the value at the specified column in this row.
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnRow.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnRow.java
new file mode 100644
index 0000000..1207f32
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnRow.java
@@ -0,0 +1,99 @@
+/*
+ * 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.iotdb.db.query.udf.core.access;
+
+import org.apache.iotdb.db.query.udf.api.access.Row;
+import org.apache.iotdb.db.query.udf.datastructure.tv.ElasticSerializableTVList;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Binary;
+
+import java.io.IOException;
+
+public class ElasticSerializableTVListBackedSingleColumnRow implements Row {
+
+  private final ElasticSerializableTVList tvList;
+  private int currentRowIndex;
+
+  public ElasticSerializableTVListBackedSingleColumnRow(
+      ElasticSerializableTVList tvList, int currentRowIndex) {
+    this.tvList = tvList;
+    this.currentRowIndex = currentRowIndex;
+  }
+
+  @Override
+  public long getTime() throws IOException {
+    return tvList.getTime(currentRowIndex);
+  }
+
+  @Override
+  public int getInt(int columnIndex) throws IOException {
+    return tvList.getInt(currentRowIndex);
+  }
+
+  @Override
+  public long getLong(int columnIndex) throws IOException {
+    return tvList.getLong(currentRowIndex);
+  }
+
+  @Override
+  public float getFloat(int columnIndex) throws IOException {
+    return tvList.getFloat(currentRowIndex);
+  }
+
+  @Override
+  public double getDouble(int columnIndex) throws IOException {
+    return tvList.getDouble(currentRowIndex);
+  }
+
+  @Override
+  public boolean getBoolean(int columnIndex) throws IOException {
+    return tvList.getBoolean(currentRowIndex);
+  }
+
+  @Override
+  public Binary getBinary(int columnIndex) throws IOException {
+    return tvList.getBinary(currentRowIndex);
+  }
+
+  @Override
+  public String getString(int columnIndex) throws IOException {
+    return tvList.getString(currentRowIndex);
+  }
+
+  @Override
+  public TSDataType getDataType(int columnIndex) {
+    return tvList.getDataType();
+  }
+
+  @Override
+  public boolean isNull(int columnIndex) {
+    return false;
+  }
+
+  @Override
+  public int size() {
+    return 1;
+  }
+
+  public Row seek(int currentRowIndex) {
+    this.currentRowIndex = currentRowIndex;
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnWindow.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnWindow.java
new file mode 100644
index 0000000..f91dd35
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnWindow.java
@@ -0,0 +1,89 @@
+/*
+ * 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.iotdb.db.query.udf.core.access;
+
+import org.apache.iotdb.db.query.udf.api.access.Row;
+import org.apache.iotdb.db.query.udf.api.access.RowIterator;
+import org.apache.iotdb.db.query.udf.api.access.RowWindow;
+import org.apache.iotdb.db.query.udf.datastructure.tv.ElasticSerializableTVList;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class ElasticSerializableTVListBackedSingleColumnWindow implements RowWindow {
+
+  private final ElasticSerializableTVList tvList;
+  private int beginIndex;
+  private int endIndex;
+  private int size;
+
+  private final ElasticSerializableTVListBackedSingleColumnRow row;
+  private ElasticSerializableTVListBackedSingleColumnWindowIterator rowIterator;
+
+  // [beginIndex, endIndex)
+  public ElasticSerializableTVListBackedSingleColumnWindow(ElasticSerializableTVList tvList) {
+    this.tvList = tvList;
+    beginIndex = 0;
+    endIndex = 0;
+    size = 0;
+
+    row = new ElasticSerializableTVListBackedSingleColumnRow(tvList, beginIndex);
+  }
+
+  @Override
+  public int windowSize() {
+    return size;
+  }
+
+  @Override
+  public Row getRow(int rowIndex) {
+    if (rowIndex < beginIndex || endIndex <= rowIndex) {
+      throw new ArrayIndexOutOfBoundsException(
+          String.format("Array index(%d) out of range [%d, %d).", rowIndex, beginIndex, endIndex));
+    }
+    return row.seek(beginIndex + rowIndex);
+  }
+
+  @Override
+  public TSDataType getDataType(int columnIndex) {
+    return tvList.getDataType();
+  }
+
+  @Override
+  public RowIterator getRowIterator() {
+    if (rowIterator == null) {
+      rowIterator =
+          new ElasticSerializableTVListBackedSingleColumnWindowIterator(
+              tvList, beginIndex, endIndex);
+    }
+
+    rowIterator.reset();
+    return rowIterator;
+  }
+
+  private RowWindow seek(int beginIndex, int endIndex) {
+    this.beginIndex = beginIndex;
+    this.endIndex = endIndex;
+    size = endIndex - beginIndex;
+
+    row.seek(beginIndex);
+    rowIterator = null;
+
+    return this;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowIteratorImpl.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnWindowIterator.java
similarity index 58%
copy from server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowIteratorImpl.java
copy to server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnWindowIterator.java
index 8c303f2..5991d50 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowIteratorImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/ElasticSerializableTVListBackedSingleColumnWindowIterator.java
@@ -21,38 +21,34 @@ package org.apache.iotdb.db.query.udf.core.access;
 
 import org.apache.iotdb.db.query.udf.api.access.Row;
 import org.apache.iotdb.db.query.udf.api.access.RowIterator;
-import org.apache.iotdb.db.query.udf.datastructure.primitive.IntList;
-import org.apache.iotdb.db.query.udf.datastructure.row.ElasticSerializableRowRecordList;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.db.query.udf.datastructure.tv.ElasticSerializableTVList;
 
 import java.io.IOException;
 
-public class RowIteratorImpl implements RowIterator {
+public class ElasticSerializableTVListBackedSingleColumnWindowIterator implements RowIterator {
 
-  private final ElasticSerializableRowRecordList rowRecordList;
-  private final IntList windowRowIndexes;
-  private final RowImpl row;
+  private final int beginIndex;
+  private final int size;
+  private final ElasticSerializableTVListBackedSingleColumnRow row;
   private int rowIndex;
 
-  public RowIteratorImpl(
-      ElasticSerializableRowRecordList rowRecordList,
-      int[] columnIndexes,
-      TSDataType[] dataTypes,
-      IntList windowRowIndexes) {
-    this.rowRecordList = rowRecordList;
-    this.windowRowIndexes = windowRowIndexes;
-    row = new RowImpl(columnIndexes, dataTypes);
+  // [beginIndex, endIndex)
+  public ElasticSerializableTVListBackedSingleColumnWindowIterator(
+      ElasticSerializableTVList tvList, int beginIndex, int endIndex) {
+    this.beginIndex = beginIndex;
+    size = endIndex - beginIndex;
+    row = new ElasticSerializableTVListBackedSingleColumnRow(tvList, beginIndex);
     rowIndex = -1;
   }
 
   @Override
   public boolean hasNextRow() {
-    return rowIndex < windowRowIndexes.size() - 1;
+    return rowIndex < size - 1;
   }
 
   @Override
   public Row next() throws IOException {
-    return row.setRowRecord(rowRecordList.getRowRecord(windowRowIndexes.get(++rowIndex)));
+    return row.seek(++rowIndex + beginIndex);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/LayerPointReaderBackedSingleColumnRow.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/LayerPointReaderBackedSingleColumnRow.java
new file mode 100644
index 0000000..a725878
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/LayerPointReaderBackedSingleColumnRow.java
@@ -0,0 +1,91 @@
+/*
+ * 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.iotdb.db.query.udf.core.access;
+
+import org.apache.iotdb.db.query.udf.api.access.Row;
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Binary;
+
+import java.io.IOException;
+
+public class LayerPointReaderBackedSingleColumnRow implements Row {
+
+  private final LayerPointReader layerPointReader;
+
+  public LayerPointReaderBackedSingleColumnRow(LayerPointReader layerPointReader) {
+    this.layerPointReader = layerPointReader;
+  }
+
+  @Override
+  public long getTime() throws IOException {
+    return layerPointReader.currentTime();
+  }
+
+  @Override
+  public int getInt(int columnIndex) throws IOException {
+    return layerPointReader.currentInt();
+  }
+
+  @Override
+  public long getLong(int columnIndex) throws IOException {
+    return layerPointReader.currentLong();
+  }
+
+  @Override
+  public float getFloat(int columnIndex) throws IOException {
+    return layerPointReader.currentFloat();
+  }
+
+  @Override
+  public double getDouble(int columnIndex) throws IOException {
+    return layerPointReader.currentDouble();
+  }
+
+  @Override
+  public boolean getBoolean(int columnIndex) throws IOException {
+    return layerPointReader.currentBoolean();
+  }
+
+  @Override
+  public Binary getBinary(int columnIndex) throws IOException {
+    return layerPointReader.currentBinary();
+  }
+
+  @Override
+  public String getString(int columnIndex) throws IOException {
+    return layerPointReader.currentBinary().getStringValue();
+  }
+
+  @Override
+  public TSDataType getDataType(int columnIndex) {
+    return layerPointReader.getDataType();
+  }
+
+  @Override
+  public boolean isNull(int columnIndex) {
+    return false;
+  }
+
+  @Override
+  public int size() {
+    return 1;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowImpl.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnRow.java
similarity index 95%
rename from server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowImpl.java
rename to server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnRow.java
index e186200..8baeca7 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnRow.java
@@ -23,14 +23,14 @@ import org.apache.iotdb.db.query.udf.api.access.Row;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
 
-public class RowImpl implements Row {
+public class MultiColumnRow implements Row {
 
   private final int[] columnIndexes;
   private final TSDataType[] dataTypes;
 
   private Object[] rowRecord;
 
-  public RowImpl(int[] columnIndexes, TSDataType[] dataTypes) {
+  public MultiColumnRow(int[] columnIndexes, TSDataType[] dataTypes) {
     this.columnIndexes = columnIndexes;
     this.dataTypes = dataTypes;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowWindowImpl.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnWindow.java
similarity index 86%
rename from server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowWindowImpl.java
rename to server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnWindow.java
index 3a225fa..0a01ad4 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowWindowImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnWindow.java
@@ -28,7 +28,7 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 import java.io.IOException;
 
-public class RowWindowImpl implements RowWindow {
+public class MultiColumnWindow implements RowWindow {
 
   private final ElasticSerializableRowRecordList rowRecordList;
 
@@ -36,10 +36,10 @@ public class RowWindowImpl implements RowWindow {
   private final TSDataType[] dataTypes;
   private final IntList windowRowIndexes;
 
-  private final RowImpl row;
-  private final RowIteratorImpl rowIterator;
+  private final MultiColumnRow row;
+  private MultiColumnWindowIterator rowIterator;
 
-  public RowWindowImpl(
+  public MultiColumnWindow(
       ElasticSerializableRowRecordList rowRecordList,
       int[] columnIndexes,
       TSDataType[] dataTypes,
@@ -48,8 +48,7 @@ public class RowWindowImpl implements RowWindow {
     this.columnIndexes = columnIndexes;
     this.dataTypes = dataTypes;
     this.windowRowIndexes = windowRowIndexes;
-    row = new RowImpl(columnIndexes, dataTypes);
-    rowIterator = new RowIteratorImpl(rowRecordList, columnIndexes, dataTypes, windowRowIndexes);
+    row = new MultiColumnRow(columnIndexes, dataTypes);
   }
 
   @Override
@@ -74,6 +73,11 @@ public class RowWindowImpl implements RowWindow {
 
   @Override
   public RowIterator getRowIterator() {
+    if (rowIterator == null) {
+      rowIterator =
+          new MultiColumnWindowIterator(rowRecordList, columnIndexes, dataTypes, windowRowIndexes);
+    }
+
     rowIterator.reset();
     return rowIterator;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowIteratorImpl.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnWindowIterator.java
similarity index 90%
rename from server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowIteratorImpl.java
rename to server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnWindowIterator.java
index 8c303f2..59de7fc 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/RowIteratorImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/access/MultiColumnWindowIterator.java
@@ -27,21 +27,21 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 import java.io.IOException;
 
-public class RowIteratorImpl implements RowIterator {
+public class MultiColumnWindowIterator implements RowIterator {
 
   private final ElasticSerializableRowRecordList rowRecordList;
   private final IntList windowRowIndexes;
-  private final RowImpl row;
+  private final MultiColumnRow row;
   private int rowIndex;
 
-  public RowIteratorImpl(
+  public MultiColumnWindowIterator(
       ElasticSerializableRowRecordList rowRecordList,
       int[] columnIndexes,
       TSDataType[] dataTypes,
       IntList windowRowIndexes) {
     this.rowRecordList = rowRecordList;
     this.windowRowIndexes = windowRowIndexes;
-    row = new RowImpl(columnIndexes, dataTypes);
+    row = new MultiColumnRow(columnIndexes, dataTypes);
     rowIndex = -1;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/udf/core/layer/IntermediateLayer.java b/server/src/main/java/org/apache/iotdb/db/query/udf/core/layer/IntermediateLayer.java
index cbac211..069af89 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/udf/core/layer/IntermediateLayer.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/udf/core/layer/IntermediateLayer.java
@@ -19,9 +19,47 @@
 
 package org.apache.iotdb.db.query.udf.core.layer;
 
+import org.apache.iotdb.db.query.udf.api.customizer.strategy.AccessStrategy;
+import org.apache.iotdb.db.query.udf.api.customizer.strategy.SlidingSizeWindowAccessStrategy;
+import org.apache.iotdb.db.query.udf.api.customizer.strategy.SlidingTimeWindowAccessStrategy;
 import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.db.query.udf.core.reader.LayerRowReader;
+import org.apache.iotdb.db.query.udf.core.reader.LayerRowWindowReader;
 
-public interface IntermediateLayer {
+public abstract class IntermediateLayer {
 
-  LayerPointReader constructPointReader();
+  protected final long queryId;
+  protected final float memoryBudgetInMB;
+
+  protected IntermediateLayer(long queryId, float memoryBudgetInMB) {
+    this.queryId = queryId;
+    this.memoryBudgetInMB = memoryBudgetInMB;
+  }
+
+  public abstract LayerPointReader constructPointReader();
+
+  public abstract LayerRowReader constructRowReader();
+
+  public final LayerRowWindowReader constructRowWindowReader(
+      AccessStrategy strategy, float memoryBudgetInMB) {
+    switch (strategy.getAccessStrategyType()) {
+      case SLIDING_TIME_WINDOW:
+        return constructRowSlidingTimeWindowReader(
+            (SlidingTimeWindowAccessStrategy) strategy, memoryBudgetInMB);
+      case SLIDING_SIZE_WINDOW:
+        return constructRowSlidingSizeWindowReader(
+            (SlidingSizeWindowAccessStrategy) strategy, memoryBudgetInMB);
+      default:
+        throw new IllegalStateException(
+            "Unexpected access strategy: " + strategy.getAccessStrategyType());
+    }
+  }
+
+  protected abstract LayerRowWindowReader constructRowSlidingSizeWindowReader(
+      SlidingSizeWindowAccessStrategy strategy, float memoryBudgetInMB);
+
+  protected abstract LayerRowWindowReader constructRowSlidingTimeWindowReader(
+      SlidingTimeWindowAccessStrategy strategy, float memoryBudgetInMB);
+
+  public abstract void updateEvictionUpperBound();
 }