[ASTERIXDB-3520][FUN]: Add capability to calculate resource size

- user model changes: no
- storage format changes: no
- interface changes: yes

Details:
- This feature introduces the storage_size datasource function
  which can be invoked as follows:
  storage_size("database", "dataverse", "dataset"[, "index")
  and return the size of the provided dataset/index.

Ext-ref: MB-59119

Change-Id: I02fb8ff0a87ff272aba05a676ccfb46ac370ebd2
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/18711
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Hussain Towaileb <hussainht@gmail.com>
Reviewed-by: Murtadha Hubail <mhubail@apache.org>
Tested-by: Hussain Towaileb <hussainht@gmail.com>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
index 45776a4..470ec34 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
@@ -85,6 +85,9 @@
     }
 
     protected boolean invalidArgs(List<Mutable<ILogicalExpression>> args) {
+        if (functionId.getArity() == FunctionIdentifier.VARARGS) {
+            return false;
+        }
         return args.size() != functionId.getArity();
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeDatasource.java
new file mode 100644
index 0000000..03d7bed
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeDatasource.java
@@ -0,0 +1,86 @@
+/*
+ * 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.asterix.app.function.collectionsize;
+
+import java.util.Objects;
+
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.api.IDatasourceFunction;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+
+public class StorageSizeDatasource extends FunctionDataSource {
+
+    private static final DataSourceId STORAGE_SIZE_DATASOURCE_ID =
+            new DataSourceId(StorageSizeRewriter.STORAGE_SIZE.getDatabase(),
+                    FunctionSignature.getDataverseName(StorageSizeRewriter.STORAGE_SIZE),
+                    StorageSizeRewriter.STORAGE_SIZE.getName());
+    private final String database;
+    private final DataverseName dataverse;
+    private final String collection;
+    private final String index;
+
+    StorageSizeDatasource(INodeDomain domain, String database, DataverseName dataverse, String collection, String index)
+            throws AlgebricksException {
+        super(STORAGE_SIZE_DATASOURCE_ID, StorageSizeRewriter.STORAGE_SIZE, domain);
+        this.database = database;
+        this.dataverse = dataverse;
+        this.collection = collection;
+        this.index = index;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public DataverseName getDataverse() {
+        return dataverse;
+    }
+
+    public String getCollection() {
+        return collection;
+    }
+
+    public String getIndex() {
+        return index;
+    }
+
+    @Override
+    protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
+            AlgebricksAbsolutePartitionConstraint locations) {
+        return new StorageSizeFunction(AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations()),
+                database, dataverse, collection, index);
+    }
+
+    @Override
+    protected boolean sameFunctionDatasource(FunctionDataSource other) {
+        if (!Objects.equals(this.functionId, other.getFunctionId())) {
+            return false;
+        }
+        StorageSizeDatasource that = (StorageSizeDatasource) other;
+        return Objects.equals(this.database, that.getDatabase()) && Objects.equals(this.dataverse, that.getDataverse())
+                && Objects.equals(this.collection, that.getCollection()) && Objects.equals(this.index, that.getIndex());
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeFunction.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeFunction.java
new file mode 100644
index 0000000..9f165bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeFunction.java
@@ -0,0 +1,83 @@
+/*
+ * 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.asterix.app.function.collectionsize;
+
+import static org.apache.asterix.app.message.ExecuteStatementRequestMessage.DEFAULT_NC_TIMEOUT_MILLIS;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.app.message.CalculateStorageSizeRequestMessage;
+import org.apache.asterix.app.message.CalculateStorageSizeResponseMessage;
+import org.apache.asterix.common.messaging.api.INCMessageBroker;
+import org.apache.asterix.common.messaging.api.MessageFuture;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.metadata.declared.AbstractDatasourceFunction;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class StorageSizeFunction extends AbstractDatasourceFunction {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final long serialVersionUID = 1L;
+
+    private final String database;
+    private final DataverseName dataverse;
+    private final String collection;
+    private final String index;
+
+    StorageSizeFunction(AlgebricksAbsolutePartitionConstraint locations, String database, DataverseName dataverse,
+            String collection, String index) {
+        super(locations);
+        this.database = database;
+        this.dataverse = dataverse;
+        this.collection = collection;
+        this.index = index;
+    }
+
+    @Override
+    public IRecordReader<char[]> createRecordReader(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException {
+        INCServiceContext serviceCtx = ctx.getJobletContext().getServiceContext();
+        INCMessageBroker messageBroker = (INCMessageBroker) serviceCtx.getMessageBroker();
+        MessageFuture messageFuture = messageBroker.registerMessageFuture();
+        long futureId = messageFuture.getFutureId();
+        CalculateStorageSizeRequestMessage request = new CalculateStorageSizeRequestMessage(serviceCtx.getNodeId(),
+                futureId, database, dataverse, collection, index);
+        try {
+            messageBroker.sendMessageToPrimaryCC(request);
+            CalculateStorageSizeResponseMessage response = (CalculateStorageSizeResponseMessage) messageFuture
+                    .get(DEFAULT_NC_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+            if (response.getFailure() != null) {
+                throw HyracksDataException.create(response.getFailure());
+            }
+            return new StorageSizeReader(response.getSize());
+        } catch (Exception e) {
+            LOGGER.info("Could not calculate collection size", e);
+            throw HyracksDataException.create(e);
+        } finally {
+            messageBroker.deregisterMessageFuture(futureId);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeReader.java
new file mode 100644
index 0000000..733d39c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeReader.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.asterix.app.function.collectionsize;
+
+import java.io.IOException;
+
+import org.apache.asterix.app.function.FunctionReader;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+
+public class StorageSizeReader extends FunctionReader {
+
+    private final long size;
+    private final CharArrayRecord record;
+    private boolean hasNext = true;
+
+    StorageSizeReader(long size) {
+        this.size = size;
+        record = new CharArrayRecord();
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+        return hasNext;
+    }
+
+    @Override
+    public IRawRecord<char[]> next() throws IOException {
+        hasNext = false;
+        record.reset();
+        String result = "{\"size\":" + size + "}";
+        record.append(result.toCharArray());
+        record.endRecord();
+        return record;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeRewriter.java
new file mode 100644
index 0000000..d551b7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeRewriter.java
@@ -0,0 +1,95 @@
+/*
+ * 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.asterix.app.function.collectionsize;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.TYPE_MISMATCH_FUNCTION;
+
+import java.util.List;
+
+import org.apache.asterix.app.function.FunctionRewriter;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+/**
+ * This function takes a collection's fully qualified name (database.scope.collection) and returns the collection's size
+ */
+
+public class StorageSizeRewriter extends FunctionRewriter {
+
+    public static final FunctionIdentifier STORAGE_SIZE =
+            FunctionConstants.newAsterix("storage-size", FunctionIdentifier.VARARGS);
+    public static final StorageSizeRewriter INSTANCE = new StorageSizeRewriter(STORAGE_SIZE);
+
+    private StorageSizeRewriter(FunctionIdentifier functionId) {
+        super(functionId);
+    }
+
+    @Override
+    protected FunctionDataSource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression function)
+            throws AlgebricksException {
+
+        if (function.getArguments().size() < 3 || function.getArguments().size() > 4) {
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_NUM_OF_ARGS, STORAGE_SIZE.getName());
+        }
+
+        verifyArgs(function.getArguments());
+        ILogicalExpression databaseExpr = function.getArguments().get(0).getValue();
+        ILogicalExpression scopeExpr = function.getArguments().get(1).getValue();
+        ILogicalExpression collectionExpr = function.getArguments().get(2).getValue();
+        ILogicalExpression indexExpr = null;
+        if (function.getArguments().size() == 4) {
+            indexExpr = function.getArguments().get(3).getValue();
+        }
+
+        String database = ConstantExpressionUtil.getStringConstant(databaseExpr);
+        DataverseName dataverse =
+                DataverseName.createSinglePartName(ConstantExpressionUtil.getStringConstant(scopeExpr));
+        String collection = ConstantExpressionUtil.getStringConstant(collectionExpr);
+        String index = indexExpr != null ? ConstantExpressionUtil.getStringConstant(indexExpr) : null;
+
+        return new StorageSizeDatasource(context.getComputationNodeDomain(), database, dataverse, collection, index);
+    }
+
+    private void verifyArgs(List<Mutable<ILogicalExpression>> args) throws CompilationException {
+        for (int i = 0; i < args.size(); i++) {
+            ConstantExpression expr = (ConstantExpression) args.get(i).getValue();
+            AsterixConstantValue value = (AsterixConstantValue) expr.getValue();
+            ATypeTag type = value.getObject().getType().getTypeTag();
+            if (type != ATypeTag.STRING) {
+                throw new CompilationException(TYPE_MISMATCH_FUNCTION, STORAGE_SIZE.getName(),
+                        ExceptionUtil.indexToPosition(i), ATypeTag.STRING, type);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeRequestMessage.java
new file mode 100644
index 0000000..1163e5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeRequestMessage.java
@@ -0,0 +1,73 @@
+/*
+ * 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.asterix.app.message;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.messaging.CCMessageBroker;
+import org.apache.asterix.utils.StorageUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class CalculateStorageSizeRequestMessage implements ICcAddressedMessage {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final long serialVersionUID = 1L;
+    private static final int FAILED_CALCULATED_SIZE = -1;
+    private final String nodeId;
+    private final long reqId;
+    private final String database;
+    private final DataverseName dataverse;
+    private final String collection;
+    private final String index;
+
+    public CalculateStorageSizeRequestMessage(String nodeId, long reqId, String database, DataverseName dataverse,
+            String collection, String index) {
+        this.nodeId = nodeId;
+        this.reqId = reqId;
+        this.database = database;
+        this.dataverse = dataverse;
+        this.collection = collection;
+        this.index = index;
+    }
+
+    @Override
+    public void handle(ICcApplicationContext appCtx) throws HyracksDataException {
+        CCMessageBroker messageBroker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+
+        try {
+            long size = StorageUtil.getCollectionSize(appCtx, database, dataverse, collection, index);
+            CalculateStorageSizeResponseMessage response =
+                    new CalculateStorageSizeResponseMessage(this.reqId, size, null);
+            messageBroker.sendApplicationMessageToNC(response, nodeId);
+        } catch (Exception ex) {
+            LOGGER.info("Failed to process request", ex);
+            try {
+                CalculateStorageSizeResponseMessage response =
+                        new CalculateStorageSizeResponseMessage(this.reqId, FAILED_CALCULATED_SIZE, ex);
+                messageBroker.sendApplicationMessageToNC(response, nodeId);
+            } catch (Exception ex2) {
+                LOGGER.info("Failed to process request", ex2);
+                throw HyracksDataException.create(ex2);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeResponseMessage.java
new file mode 100644
index 0000000..aa7989e
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeResponseMessage.java
@@ -0,0 +1,56 @@
+/*
+ * 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.asterix.app.message;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+import org.apache.asterix.common.messaging.api.MessageFuture;
+import org.apache.asterix.messaging.NCMessageBroker;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class CalculateStorageSizeResponseMessage implements INcAddressedMessage {
+
+    private static final long serialVersionUID = 1L;
+    private final long reqId;
+    private final long size;
+    private final Throwable failure;
+
+    public CalculateStorageSizeResponseMessage(long reqId, long size, Throwable failure) {
+        this.reqId = reqId;
+        this.size = size;
+        this.failure = failure;
+    }
+
+    @Override
+    public void handle(INcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
+        NCMessageBroker mb = (NCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        MessageFuture future = mb.deregisterMessageFuture(reqId);
+        if (future != null) {
+            future.complete(this);
+        }
+    }
+
+    public long getSize() {
+        return size;
+    }
+
+    public Throwable getFailure() {
+        return failure;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeRequestMessage.java
new file mode 100644
index 0000000..82b425b
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeRequestMessage.java
@@ -0,0 +1,96 @@
+/*
+ * 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.asterix.app.message;
+
+import java.util.function.Predicate;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.messaging.CcIdentifiedMessage;
+import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.messaging.NCMessageBroker;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class StorageSizeRequestMessage extends CcIdentifiedMessage implements INcAddressedMessage {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final long serialVersionUID = 1L;
+    private final long reqId;
+    private final String database;
+    private final String dataverse;
+    private final String collection;
+    private final String index;
+
+    public StorageSizeRequestMessage(long reqId, String database, String dataverse, String collection, String index) {
+        this.reqId = reqId;
+        this.database = database;
+        this.dataverse = dataverse;
+        this.collection = collection;
+        this.index = index;
+    }
+
+    @Override
+    public void handle(INcApplicationContext appCtx) throws HyracksDataException {
+        try {
+            Predicate<String> predicate = getPredicate();
+            IIOManager ioManager = appCtx.getPersistenceIoManager();
+            StorageSizeResponseMessage response =
+                    new StorageSizeResponseMessage(reqId, ioManager.getSize(predicate), null);
+            respond(appCtx, response);
+        } catch (Exception e) {
+            LOGGER.info("failed to get collection size", e);
+            StorageSizeResponseMessage response = new StorageSizeResponseMessage(reqId, 0, e);
+            respond(appCtx, response);
+        }
+    }
+
+    private Predicate<String> getPredicate() {
+        return path -> {
+            ResourceReference resourceReference = ResourceReference.of(path);
+            if (resourceReference.getDatabase().equals(database)
+                    && resourceReference.getDataverse().getCanonicalForm().equals(dataverse)
+                    && resourceReference.getDataset().equals(collection)) {
+                if (index != null) {
+                    return resourceReference.getIndex().equals(index);
+                }
+                return true;
+            }
+            return false;
+        };
+    }
+
+    private void respond(INcApplicationContext appCtx, StorageSizeResponseMessage response)
+            throws HyracksDataException {
+        NCMessageBroker messageBroker = (NCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        try {
+            messageBroker.sendMessageToPrimaryCC(response);
+        } catch (Exception e) {
+            LOGGER.info("failed to send collection size to cc", e);
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public boolean isWhispered() {
+        return true;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeResponseMessage.java
new file mode 100644
index 0000000..09850f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeResponseMessage.java
@@ -0,0 +1,76 @@
+/*
+ * 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.asterix.app.message;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.messaging.api.ICCMessageBroker;
+import org.apache.asterix.common.messaging.api.ICCMessageBroker.ResponseState;
+import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
+import org.apache.asterix.common.messaging.api.INcResponse;
+import org.apache.commons.lang3.tuple.MutablePair;
+
+public class StorageSizeResponseMessage implements ICcAddressedMessage, INcResponse {
+
+    private static final long serialVersionUID = 1L;
+    private final long reqId;
+    private final long size;
+    private final Throwable failure;
+
+    public StorageSizeResponseMessage(long reqId, long size, Throwable failure) {
+        this.reqId = reqId;
+        this.size = size;
+        this.failure = failure;
+    }
+
+    @Override
+    public void handle(ICcApplicationContext appCtx) {
+        ICCMessageBroker broker = (ICCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        broker.respond(reqId, this);
+    }
+
+    @Override
+    public void setResult(MutablePair<ResponseState, Object> result) {
+        if (failure != null) {
+            result.setLeft(ResponseState.FAILURE);
+            result.setRight(failure);
+            return;
+        }
+        setResponse(result);
+    }
+
+    private void setResponse(MutablePair<ResponseState, Object> result) {
+        switch (result.getKey()) {
+            case SUCCESS:
+                long currentSize = (long) result.getValue();
+                result.setValue(currentSize + size);
+                break;
+            case UNINITIALIZED:
+                result.setLeft(ResponseState.SUCCESS);
+                result.setValue(size);
+                break;
+            default:
+                break;
+        }
+    }
+
+    @Override
+    public boolean isWhispered() {
+        return true;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
index 9d39088..ea8d5fc 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
@@ -31,7 +31,9 @@
 import org.apache.asterix.app.function.StorageComponentsRewriter;
 import org.apache.asterix.app.function.TPCDSAllTablesDataGeneratorRewriter;
 import org.apache.asterix.app.function.TPCDSSingleTableDataGeneratorRewriter;
+import org.apache.asterix.app.function.collectionsize.StorageSizeRewriter;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.typecomputer.impl.AInt64TypeComputer;
 import org.apache.asterix.om.utils.RecordUtil;
 
 public class MetadataBuiltinFunctions {
@@ -106,6 +108,10 @@
                 true);
         BuiltinFunctions.addUnnestFun(QueryPartitionRewriter.QUERY_PARTITION, false);
         BuiltinFunctions.addDatasourceFunction(QueryPartitionRewriter.QUERY_PARTITION, QueryPartitionRewriter.INSTANCE);
+        // storage size
+        BuiltinFunctions.addFunction(StorageSizeRewriter.STORAGE_SIZE, AInt64TypeComputer.INSTANCE, true);
+        BuiltinFunctions.addUnnestFun(StorageSizeRewriter.STORAGE_SIZE, true);
+        BuiltinFunctions.addDatasourceFunction(StorageSizeRewriter.STORAGE_SIZE, StorageSizeRewriter.INSTANCE);
     }
 
     private MetadataBuiltinFunctions() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/StorageUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/StorageUtil.java
new file mode 100644
index 0000000..3eb9d08
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/StorageUtil.java
@@ -0,0 +1,102 @@
+/*
+ * 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.asterix.utils;
+
+import static org.apache.asterix.common.api.IClusterManagementWork.ClusterState.ACTIVE;
+import static org.apache.asterix.common.api.IClusterManagementWork.ClusterState.REBALANCE_REQUIRED;
+import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_BAD_CLUSTER_STATE;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.app.message.StorageSizeRequestMessage;
+import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.common.metadata.MetadataConstants;
+import org.apache.asterix.common.metadata.MetadataUtil;
+import org.apache.asterix.messaging.CCMessageBroker;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.hyracks.api.util.InvokeUtil;
+
+public class StorageUtil {
+
+    public static long getCollectionSize(ICcApplicationContext appCtx, String database, DataverseName dataverse,
+            String collection, String index) throws Exception {
+        IClusterManagementWork.ClusterState state = appCtx.getClusterStateManager().getState();
+        if (!(state == ACTIVE || state == REBALANCE_REQUIRED)) {
+            throw new RuntimeDataException(REJECT_BAD_CLUSTER_STATE, state);
+        }
+
+        if (!appCtx.getNamespaceResolver().isUsingDatabase()) {
+            database = MetadataConstants.DEFAULT_DATABASE;
+        }
+
+        IMetadataLockManager lockManager = appCtx.getMetadataLockManager();
+        MetadataProvider metadataProvider = MetadataProvider.createWithDefaultNamespace(appCtx);
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            lockManager.acquireDatabaseReadLock(metadataProvider.getLocks(), database);
+            lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), database, dataverse);
+            lockManager.acquireDatasetReadLock(metadataProvider.getLocks(), database, dataverse, collection);
+            Dataset dataset = metadataProvider.findDataset(database, dataverse, collection);
+            if (dataset == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, collection,
+                        MetadataUtil.dataverseName(database, dataverse, metadataProvider.isUsingDatabase()));
+            }
+
+            if (dataset.getDatasetType() != DatasetConfig.DatasetType.INTERNAL) {
+                throw new CompilationException(ErrorCode.STORAGE_SIZE_NOT_APPLICABLE_TO_TYPE, dataset.getDatasetType());
+            }
+
+            if (index != null) {
+                Index idx = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), database,
+                        dataverse, collection, index);
+                if (idx == null) {
+                    throw new CompilationException(ErrorCode.UNKNOWN_INDEX, index);
+                }
+            }
+
+            final List<String> ncs = new ArrayList<>(appCtx.getClusterStateManager().getParticipantNodes());
+            CCMessageBroker messageBroker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+
+            long reqId = messageBroker.newRequestId();
+            List<StorageSizeRequestMessage> requests = new ArrayList<>();
+            for (int i = 0; i < ncs.size(); i++) {
+                requests.add(new StorageSizeRequestMessage(reqId, database, dataverse.getCanonicalForm(), collection,
+                        index));
+            }
+            return (long) messageBroker.sendSyncRequestToNCs(reqId, ncs, requests, TimeUnit.SECONDS.toMillis(60), true);
+        } finally {
+            InvokeUtil.tryWithCleanups(() -> MetadataManager.INSTANCE.commitTransaction(mdTxnCtx),
+                    () -> metadataProvider.getLocks().unlock());
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/LocalCloudUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/LocalCloudUtil.java
index 9e3fdc4..b50d2f2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/LocalCloudUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/LocalCloudUtil.java
@@ -57,6 +57,10 @@
     }
 
     public static S3Mock startS3CloudEnvironment(boolean cleanStart) {
+        return startS3CloudEnvironment(cleanStart, false);
+    }
+
+    public static S3Mock startS3CloudEnvironment(boolean cleanStart, boolean createPlaygroundContainer) {
         if (cleanStart) {
             FileUtils.deleteQuietly(new File(MOCK_FILE_BACKEND));
         }
@@ -79,6 +83,12 @@
         S3Client client = builder.build();
         client.createBucket(CreateBucketRequest.builder().bucket(CLOUD_STORAGE_BUCKET).build());
         LOGGER.info("Created bucket {} for cloud storage", CLOUD_STORAGE_BUCKET);
+
+        // added for convenience since some non-external-based tests include an external collection test on this bucket
+        if (createPlaygroundContainer) {
+            client.createBucket(CreateBucketRequest.builder().bucket("playground").build());
+            LOGGER.info("Created bucket {}", "playground");
+        }
         client.close();
         return s3MockServer;
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageGCSTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageGCSTest.java
index d62f8f0..c532d9d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageGCSTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageGCSTest.java
@@ -27,6 +27,7 @@
 import java.util.Objects;
 import java.util.Random;
 
+import org.apache.asterix.api.common.LocalCloudUtil;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.test.common.TestExecutor;
 import org.apache.asterix.test.runtime.LangExecutionUtil;
@@ -76,6 +77,7 @@
 
     @BeforeClass
     public static void setUp() throws Exception {
+        LocalCloudUtil.startS3CloudEnvironment(true, true);
         Storage storage = StorageOptions.newBuilder().setHost(MOCK_SERVER_HOSTNAME)
                 .setCredentials(NoCredentials.getInstance()).setProjectId(MOCK_SERVER_PROJECT_ID).build().getService();
         cleanup(storage);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageTest.java
index 5f7a037..78f4e55 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.test.cloud_storage;
 
+import java.net.URI;
 import java.util.Collection;
 import java.util.List;
 
@@ -40,6 +41,12 @@
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.CreateBucketRequest;
+
 /**
  * Run tests in cloud deployment environment
  */
@@ -56,6 +63,11 @@
     private static final String DELTA_RESULT_PATH = "results_cloud";
     private static final String EXCLUDED_TESTS = "MP";
 
+    private static final String PLAYGROUND_CONTAINER = "playground";
+    private static final String MOCK_SERVER_REGION = "us-west-2";
+    private static final int MOCK_SERVER_PORT = 8001;
+    private static final String MOCK_SERVER_HOSTNAME = "http://127.0.0.1:" + MOCK_SERVER_PORT;
+
     public CloudStorageTest(TestCaseContext tcCtx) {
         this.tcCtx = tcCtx;
     }
@@ -68,6 +80,15 @@
         testExecutor.stripSubstring = "//DB:";
         LangExecutionUtil.setUp(CONFIG_FILE_NAME, testExecutor);
         System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, CONFIG_FILE_NAME);
+
+        // create the playground bucket and leave it empty, just for external collection-based tests
+        S3ClientBuilder builder = S3Client.builder();
+        URI endpoint = URI.create(MOCK_SERVER_HOSTNAME); // endpoint pointing to S3 mock server
+        builder.region(Region.of(MOCK_SERVER_REGION)).credentialsProvider(AnonymousCredentialsProvider.create())
+                .endpointOverride(endpoint);
+        S3Client client = builder.build();
+        client.createBucket(CreateBucketRequest.builder().bucket(PLAYGROUND_CONTAINER).build());
+        client.close();
     }
 
     @AfterClass
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageUnstableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageUnstableTest.java
index a517ed4..82a264a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageUnstableTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageUnstableTest.java
@@ -66,7 +66,7 @@
     @BeforeClass
     public static void setUp() throws Exception {
         System.setProperty(CloudRetryableRequestUtil.CLOUD_UNSTABLE_MODE, "true");
-        LocalCloudUtil.startS3CloudEnvironment(true);
+        LocalCloudUtil.startS3CloudEnvironment(true, true);
         TestExecutor testExecutor = new TestExecutor(DELTA_RESULT_PATH);
         testExecutor.executorId = "cloud";
         testExecutor.stripSubstring = "//DB:";
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NonCloudSqlppExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NonCloudSqlppExecutionTest.java
new file mode 100644
index 0000000..c403178
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NonCloudSqlppExecutionTest.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.asterix.test.runtime;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.api.common.LocalCloudUtil;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ runtime tests with the storage parallelism.
+ * This version of the tests runs the tests that aren't meant to be run in the cloud because they can have different
+ * output.
+ */
+@RunWith(Parameterized.class)
+public class NonCloudSqlppExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        final TestExecutor testExecutor = new TestExecutor();
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+        setNcEndpoints(testExecutor);
+        LocalCloudUtil.startS3CloudEnvironment(true, true);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "NonCloudSqlppExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "non_cloud_sqlpp_queries.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public NonCloudSqlppExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+    private static void setNcEndpoints(TestExecutor testExecutor) {
+        final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+        for (NodeControllerService nc : ncs) {
+            final String nodeId = nc.getId();
+            final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+            int apiPort = appCtx.getExternalProperties().getNcApiPort();
+            ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+        }
+        testExecutor.setNcEndPoints(ncEndPoints);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/non_cloud_sqlpp_queries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/non_cloud_sqlpp_queries.xml
new file mode 100644
index 0000000..f379c0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/non_cloud_sqlpp_queries.xml
@@ -0,0 +1,93 @@
+<!--
+ ! 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.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp"
+            QueryFileExtension=".sqlpp">
+  <test-group name="sqlpp_queries">
+    <test-group name="storage-size">
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="collection-exists">
+          <output-dir compare="Clean-JSON">collection-exists</output-dir>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="non-constant-argument">
+          <output-dir compare="Clean-JSON">non-constant-argument</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Compilation error: Function asterix.storage-size expects constant arguments while arg[0] is of type VARIABLE</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="collection-does-not-exist">
+          <output-dir compare="Clean-JSON">collection-does-not-exist</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Cannot find dataset with name doesNotExistCollection in dataverse testScope</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="non-string-argument">
+          <output-dir compare="Clean-JSON">non-string-argument</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Type mismatch: function storage-size expects its 1st input parameter to be of type string, but the actual input type is bigint</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="view-not-collection">
+          <output-dir compare="Clean-JSON">view-not-collection</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Cannot find dataset with name testCollection_vw in dataverse testScope</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="external-collection">
+          <output-dir compare="Clean-JSON">external-collection</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Retrieving storage size is not applicable to type: EXTERNAL.</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="index-exists">
+          <output-dir compare="Clean-JSON">index-exists</output-dir>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="index-does-not-exist">
+          <output-dir compare="Clean-JSON">index-does-not-exist</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Cannot find index with name testCollection_idx_fake</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="invalid-arguments-count">
+          <output-dir compare="Clean-JSON">invalid-arguments-count</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Invalid number of arguments for function storage-size</expected-error>
+          <expected-error>Invalid number of arguments for function storage-size</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="sanity">
+          <output-dir compare="Clean-JSON">sanity</output-dir>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp
new file mode 100644
index 0000000..dc66c29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "doesNotExistCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.040.query.sqlpp
new file mode 100644
index 0000000..d4442c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp
new file mode 100644
index 0000000..5ae3310
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
+
+CREATE TYPE test AS open {};
+CREATE EXTERNAL DATASET testExternalCollection(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("sessionToken"="dummySessionToken"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://127.0.0.1:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.040.query.sqlpp
new file mode 100644
index 0000000..9915cfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testExternalCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp
new file mode 100644
index 0000000..18ca3d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+CREATE INDEX testCollection_idx ON testCollection(some_value: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp
new file mode 100644
index 0000000..905a20a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx_fake") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.021.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.021.update.sqlpp
new file mode 100644
index 0000000..18ca3d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.021.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+CREATE INDEX testCollection_idx ON testCollection(some_value: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.040.query.sqlpp
new file mode 100644
index 0000000..cb60cb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp
new file mode 100644
index 0000000..d376541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp
new file mode 100644
index 0000000..b395ed5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testIndex", "tooManyArgs") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp
new file mode 100644
index 0000000..aea887f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+SELECT storage_size(v.id, v.id, v.id) FROM testCollection AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp
new file mode 100644
index 0000000..ddb1996
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size(1, 2, 3) AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.010.ddl.sqlpp
new file mode 100644
index 0000000..84d75e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.010.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection PRIMARY KEY (id: int);
+
+CREATE INDEX testCollection_idx ON testCollection(someField1: int);
+
+CREATE COLLECTION collectionAndIndexes PRIMARY KEY (id: int);
+CREATE COLLECTION collectionOnly PRIMARY KEY (id: int);
+CREATE COLLECTION indexOnly PRIMARY KEY (id: int)
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.020.update.sqlpp
new file mode 100644
index 0000000..f08857d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.020.update.sqlpp
@@ -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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {
+    "id": intVal,
+    "someField1": intVal,
+    "someField2": uuid(),
+    "someField3": {"firstName": uuid(), "lastName": uuid(), "location": {"lon": intVal * 2 + intVal, "lat": intVal * 7 + intVal}}
+    }
+    FROM range(1, 100000) intVal
+);
+
+INSERT INTO collectionAndIndexes({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection") as x)[0]});
+INSERT INTO collectionOnly({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection", "testCollection") as x)[0]});
+INSERT INTO indexOnly({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx") as x)[0]});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.040.query.sqlpp
new file mode 100644
index 0000000..f37a396
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.040.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+SELECT a.valueField > b.valueField AS collectionAndIndexesLargerThanCollectionOnly, b.valueField > c.valueField AS collectionOnlyLargerThanIndexOnly
+FROM collectionAndIndexes a,
+     collectionOnly b,
+     indexOnly c
+WHERE a.id = b.id
+  AND a.id = c.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp
new file mode 100644
index 0000000..3d9ea2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
+
+CREATE VIEW testCollection_vw(id int)
+DEFAULT NULL datetime "YYYY-MM-DD hh:mm:ss" AS
+SELECT t.id
+FROM testCollection t;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp
new file mode 100644
index 0000000..cd88a1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection_vw") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp
new file mode 100644
index 0000000..dc66c29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "doesNotExistCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.040.query.sqlpp
new file mode 100644
index 0000000..d4442c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp
new file mode 100644
index 0000000..5dcc7e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp
@@ -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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
+
+CREATE TYPE test AS open {};
+CREATE EXTERNAL DATASET testExternalCollection(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("sessionToken"="dummySessionToken"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://127.0.0.1:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.040.query.sqlpp
new file mode 100644
index 0000000..9915cfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testExternalCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp
new file mode 100644
index 0000000..24916e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+CREATE INDEX testCollection_idx ON testCollection(some_value: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp
new file mode 100644
index 0000000..905a20a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx_fake") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.021.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.021.update.sqlpp
new file mode 100644
index 0000000..24916e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.021.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+CREATE INDEX testCollection_idx ON testCollection(some_value: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.040.query.sqlpp
new file mode 100644
index 0000000..cb60cb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp
new file mode 100644
index 0000000..d376541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp
new file mode 100644
index 0000000..b395ed5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testIndex", "tooManyArgs") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp
new file mode 100644
index 0000000..2a0260e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+SELECT storage_size(v.id, v.id, v.id) FROM testCollection AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp
new file mode 100644
index 0000000..ddb1996
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size(1, 2, 3) AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.010.ddl.sqlpp
new file mode 100644
index 0000000..761ae9f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.010.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection PRIMARY KEY (id: int);
+
+CREATE INDEX testCollection_idx ON testCollection(someField1: int);
+
+CREATE COLLECTION collectionAndIndexes PRIMARY KEY (id: int);
+CREATE COLLECTION collectionOnly PRIMARY KEY (id: int);
+CREATE COLLECTION indexOnly PRIMARY KEY (id: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.020.update.sqlpp
new file mode 100644
index 0000000..44905ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.020.update.sqlpp
@@ -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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {
+    "id": intVal,
+    "someField1": intVal,
+    "someField2": uuid(),
+    "someField3": {"firstName": uuid(), "lastName": uuid(), "location": {"lon": intVal * 2 + intVal, "lat": intVal * 7 + intVal}}
+    }
+    FROM range(1, 100000) intVal
+);
+
+INSERT INTO collectionAndIndexes({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection") as x)[0]});
+INSERT INTO collectionOnly({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection", "testCollection") as x)[0]});
+INSERT INTO indexOnly({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx") as x)[0]});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.040.query.sqlpp
new file mode 100644
index 0000000..4ad1a6e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.040.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+SELECT a.valueField > b.valueField AS collectionAndIndexesLargerThanCollectionOnly, b.valueField > c.valueField AS collectionOnlyLargerThanIndexOnly
+FROM collectionAndIndexes a,
+     collectionOnly b,
+     indexOnly c
+WHERE a.id = b.id
+  AND a.id = c.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp
new file mode 100644
index 0000000..34a3239
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
+
+CREATE VIEW testCollection_vw(id int)
+DEFAULT NULL datetime "YYYY-MM-DD hh:mm:ss" AS
+SELECT t.id
+FROM testCollection t;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp
new file mode 100644
index 0000000..cd88a1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection_vw") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson
index b3f8c7c..d478e8a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson
@@ -16,6 +16,6 @@
 		"resultSize": "R{.*}",
 		"processedObjects": "R{.*}",
 		"bufferCacheHitRatio": "100.00%",
-		"bufferCachePageReadCount": 1
+		"bufferCachePageReadCount": "R{.*}"
 	}
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/collection-exists/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/collection-exists/result.040.regexjson
new file mode 100644
index 0000000..1e9e352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/collection-exists/result.040.regexjson
@@ -0,0 +1,3 @@
+{
+    "size": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/external-collection/result.002.ignore b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/external-collection/result.002.ignore
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/external-collection/result.002.ignore
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/index-exists/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/index-exists/result.040.regexjson
new file mode 100644
index 0000000..1e9e352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/index-exists/result.040.regexjson
@@ -0,0 +1,3 @@
+{
+    "size": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/sanity/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/sanity/result.040.regexjson
new file mode 100644
index 0000000..5cfd541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/sanity/result.040.regexjson
@@ -0,0 +1 @@
+{"collectionAndIndexesLargerThanCollectionOnly":true,"collectionOnlyLargerThanIndexOnly":true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/collection-exists/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/collection-exists/result.040.regexjson
new file mode 100644
index 0000000..1e9e352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/collection-exists/result.040.regexjson
@@ -0,0 +1,3 @@
+{
+    "size": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/external-collection/result.002.ignore b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/external-collection/result.002.ignore
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/external-collection/result.002.ignore
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/index-exists/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/index-exists/result.040.regexjson
new file mode 100644
index 0000000..1e9e352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/index-exists/result.040.regexjson
@@ -0,0 +1,3 @@
+{
+    "size": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/sanity/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/sanity/result.040.regexjson
new file mode 100644
index 0000000..5cfd541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/sanity/result.040.regexjson
@@ -0,0 +1 @@
+{"collectionAndIndexesLargerThanCollectionOnly":true,"collectionOnlyLargerThanIndexOnly":true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_cloud_storage.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_cloud_storage.xml
index 6a75394..03fa519 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_cloud_storage.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_cloud_storage.xml
@@ -44,5 +44,65 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="storage-size">
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="collection-exists">
+        <output-dir compare="Clean-JSON">collection-exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="non-constant-argument">
+        <output-dir compare="Clean-JSON">non-constant-argument</output-dir>
+        <expected-error>Compilation error: Function asterix.storage-size expects constant arguments while arg[0] is of type VARIABLE</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="collection-does-not-exist">
+        <output-dir compare="Clean-JSON">collection-does-not-exist</output-dir>
+        <expected-error>Cannot find dataset with name doesNotExistCollection in dataverse testDatabase.testScope</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="non-string-argument">
+        <output-dir compare="Clean-JSON">non-string-argument</output-dir>
+        <expected-error>Type mismatch: function storage-size expects its 1st input parameter to be of type string, but the actual input type is bigint</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="view-not-collection">
+        <output-dir compare="Clean-JSON">view-not-collection</output-dir>
+        <expected-error>Cannot find dataset with name testCollection_vw in dataverse testDatabase.testScope</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="external-collection">
+        <output-dir compare="Clean-JSON">external-collection</output-dir>
+        <expected-error>Retrieving storage size is not applicable to type: EXTERNAL.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="index-exists">
+        <output-dir compare="Clean-JSON">index-exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="index-does-not-exist">
+        <output-dir compare="Clean-JSON">index-does-not-exist</output-dir>
+        <expected-error>Cannot find index with name testCollection_idx_fake</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="invalid-arguments-count">
+        <output-dir compare="Clean-JSON">invalid-arguments-count</output-dir>
+        <expected-error>Invalid number of arguments for function storage-size</expected-error>
+        <expected-error>Invalid number of arguments for function storage-size</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="sanity">
+        <output-dir compare="Clean-JSON">sanity</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   &sqlpp_queries;
 </test-suite>
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/AbstractCloudIOManager.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/AbstractCloudIOManager.java
index da014a4..4ce382e 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/AbstractCloudIOManager.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/AbstractCloudIOManager.java
@@ -31,6 +31,7 @@
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Predicate;
 
 import org.apache.asterix.cloud.bulk.DeleteBulkCloudOperation;
 import org.apache.asterix.cloud.bulk.NoOpDeleteBulkCallBack;
@@ -70,6 +71,8 @@
 public abstract class AbstractCloudIOManager extends IOManager implements IPartitionBootstrapper, ICloudIOManager {
     private static final Logger LOGGER = LogManager.getLogger();
     private static final byte[] EMPTY_FILE_BYTES = "empty".getBytes();
+    private static final Predicate<String> NO_OP_LIST_FILES_FILTER = (path) -> true;
+
     protected final ICloudClient cloudClient;
     protected final ICloudGuardian guardian;
     protected final IWriteBufferProvider writeBufferProvider;
@@ -472,11 +475,20 @@
     }
 
     public long getTotalRemoteStorageSizeForNodeBytes() {
-        long size = 0;
-        for (CloudFile file : list()) {
-            size += file.getSize();
+        return getSize(NO_OP_LIST_FILES_FILTER);
+    }
+
+    @Override
+    public long getSize(Predicate<String> relativePathFilter) {
+        long totalSize = localIoManager.getSize(relativePathFilter);
+
+        // get uncached files from uncached files tracker
+        for (UncachedFileReference uncachedFile : getUncachedFiles()) {
+            if (relativePathFilter.test(uncachedFile.getRelativePath())) {
+                totalSize += uncachedFile.getSize();
+            }
         }
-        return size;
+        return totalSize;
     }
 
     @Override
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index b955fa2..319a7eb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -312,6 +312,7 @@
     TYPE_UNSUPPORTED_CSV_WRITE(1207),
     INVALID_CSV_SCHEMA(1208),
     MAXIMUM_VALUE_ALLOWED_FOR_PARAM(1209),
+    STORAGE_SIZE_NOT_APPLICABLE_TO_TYPE(1210),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index e450657..bbe7582 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -314,6 +314,7 @@
 1207 = '%1$s' type not supported in csv format
 1208 = Invalid Copy to CSV schema
 1209 = Maximum value allowed for '%1$s' is %2$s. Found %3$s
+1210 = Retrieving storage size is not applicable to type: %1$s.
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
index a6520c6..75cd23d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
@@ -24,6 +24,7 @@
 import java.nio.channels.WritableByteChannel;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Predicate;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -296,4 +297,12 @@
      * @param bulkOperation the operation to perform
      */
     void performBulkOperation(IIOBulkOperation bulkOperation) throws HyracksDataException;
+
+    /**
+     * Lists the storage files and apply the provided predicate to calculate the total size of the resource
+     *
+     * @param relativePathFilter predicate
+     * @return total size of resource accepting the predicate
+     */
+    long getSize(Predicate<String> relativePathFilter);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index 740940e..960f23b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -40,6 +40,7 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -67,6 +68,7 @@
      */
     private static final Logger LOGGER = LogManager.getLogger();
     private static final String WORKSPACE_FILE_SUFFIX = ".waf";
+    private static final String STORAGE_ROOT_DIR_NAME = "storage";
     private static final FilenameFilter WORKSPACE_FILES_FILTER = (dir, name) -> name.endsWith(WORKSPACE_FILE_SUFFIX);
     /*
      * Finals
@@ -601,6 +603,36 @@
         ((AbstractBulkOperation) bulkOperation).performOperation();
     }
 
+    @Override
+    public long getSize(Predicate<String> relativePathFilter) {
+        long totalSize = 0;
+
+        // get cached files (read from disk)
+        for (IODeviceHandle deviceHandle : getIODevices()) {
+            FileReference storageRoot = deviceHandle.createFileRef(STORAGE_ROOT_DIR_NAME);
+
+            Set<FileReference> deviceFiles;
+            try {
+                deviceFiles = list(storageRoot, IoUtil.NO_OP_FILTER);
+            } catch (Throwable th) {
+                LOGGER.info("Failed to get local storage files for root {}", storageRoot.getRelativePath(), th);
+                continue;
+            }
+
+            for (FileReference fileReference : deviceFiles) {
+                try {
+                    if (relativePathFilter.test(fileReference.getRelativePath())) {
+                        totalSize += fileReference.getFile().length();
+                    }
+                } catch (Throwable th) {
+                    LOGGER.info("Encountered issue for local storage file {}", fileReference.getRelativePath(), th);
+                }
+            }
+        }
+
+        return totalSize;
+    }
+
     public void setSpaceMaker(IDiskSpaceMaker spaceMaker) {
         this.spaceMaker = spaceMaker;
     }