You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2018/11/07 07:15:46 UTC

[01/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Repository: asterixdb
Updated Branches:
  refs/heads/master 8076fe97d -> 5aeba9b47


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index 315496d..e38514d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -41,6 +41,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.hyracks.util.trace.ITraceCategoryRegistry;
 import org.apache.hyracks.util.trace.ITracer;
 import org.apache.hyracks.util.trace.TraceCategoryRegistry;
@@ -99,13 +100,16 @@ public final class LSMBTreeTestContext extends OrderedIndexTestContext {
             lsmTree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits,
                     cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
                     ioScheduler, ioOpCallbackFactory, needKeyDupCheck, filterTypeTraits, filterCmp, btreefields,
-                    filterfields, true, metadataPageManagerFactory, updateAware, ITracer.NONE);
+                    filterfields, true, metadataPageManagerFactory, updateAware, ITracer.NONE,
+                    NoOpCompressorDecompressorFactory.INSTANCE);
         } else {
             lsmTree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits,
                     cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
                     ioScheduler, ioOpCallbackFactory, needKeyDupCheck, null, null, null, null, true,
-                    metadataPageManagerFactory, updateAware, new Tracer(LSMBTreeTestContext.class.getSimpleName(),
-                            ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
+                    metadataPageManagerFactory,
+                    updateAware, new Tracer(LSMBTreeTestContext.class.getSimpleName(),
+                            ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
+                    NoOpCompressorDecompressorFactory.INSTANCE);
         }
         LSMBTreeTestContext testCtx = new LSMBTreeTestContext(fieldSerdes, lsmTree, filtered);
         return testCtx;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheWithCompressionTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheWithCompressionTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheWithCompressionTest.java
new file mode 100644
index 0000000..a978bc5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheWithCompressionTest.java
@@ -0,0 +1,401 @@
+/*
+ * 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.hyracks.storage.common;
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.compression.SnappyCompressorDecompressorFactory;
+import org.apache.hyracks.storage.common.compression.file.CompressedFileReference;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
+import org.apache.hyracks.test.support.TestUtils;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class BufferCacheWithCompressionTest {
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final List<String> openFiles = new ArrayList<>();
+    private static final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+
+    private static final ICompressorDecompressor compDecomp =
+            (new SnappyCompressorDecompressorFactory()).createInstance();
+    private static final int PAGE_SIZE = 256;
+    private static final int NUM_PAGES = 10;
+    private static final int MAX_OPEN_DATA_FILES = 20;
+    //Additional file (LAF) for each compressed file
+    private static final int ACTUAL_MAX_OPEN_FILE = MAX_OPEN_DATA_FILES * 2;
+    private static final int HYRACKS_FRAME_SIZE = PAGE_SIZE;
+    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+
+    private static final Random rnd = new Random(50);
+
+    private FileReference getFileReference(IIOManager ioManager) throws HyracksDataException {
+        String fileName = simpleDateFormat.format(new Date()) + openFiles.size();
+        final FileReference fileRef = ioManager.resolve(fileName);
+        final CompressedFileReference cFileRef = new CompressedFileReference(fileRef.getDeviceHandle(), compDecomp,
+                fileRef.getRelativePath(), fileRef.getRelativePath() + ".dic");
+
+        openFiles.add(fileName);
+        openFiles.add(cFileRef.getLAFRelativePath());
+        return cFileRef;
+    }
+
+    @Test
+    public void interruptPinTest() throws Exception {
+        final int bufferCacheNumPages = 4;
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, bufferCacheNumPages, ACTUAL_MAX_OPEN_FILE);
+        IIOManager ioManager = TestStorageManagerComponentHolder.getIOManager();
+        IBufferCache bufferCache =
+                TestStorageManagerComponentHolder.getBufferCache(ctx.getJobletContext().getServiceContext());
+        final long duration = TimeUnit.SECONDS.toMillis(20);
+        final FileReference file = getFileReference(ioManager);
+        final int fileId = bufferCache.createFile(file);
+        final int numPages = 16;
+        bufferCache.openFile(fileId);
+        final ICompressedPageWriter writer = bufferCache.getCompressedPageWriter(fileId);
+        final IFIFOPageQueue queue = bufferCache.createFIFOQueue();
+        for (int i = 0; i < numPages; i++) {
+            long dpid = BufferedFileHandle.getDiskPageId(fileId, i);
+            ICachedPage page = bufferCache.confiscatePage(dpid);
+            writer.prepareWrite(page);
+            page.getBuffer().putInt(0, i);
+            queue.put(page, HaltOnFailureCallback.INSTANCE);
+        }
+        bufferCache.finishQueue();
+        writer.endWriting();
+        bufferCache.closeFile(fileId);
+        ExecutorService executor = Executors.newFixedThreadPool(bufferCacheNumPages);
+        MutableObject<Thread>[] readers = new MutableObject[bufferCacheNumPages];
+        Future<Void>[] futures = new Future[bufferCacheNumPages];
+        for (int i = 0; i < bufferCacheNumPages; i++) {
+            readers[i] = new MutableObject<>();
+            final int threadNumber = i;
+            futures[i] = executor.submit(new Callable<Void>() {
+                @Override
+                public Void call() throws Exception {
+                    synchronized (readers[threadNumber]) {
+                        readers[threadNumber].setValue(Thread.currentThread());
+                        readers[threadNumber].notifyAll();
+                    }
+                    // for duration, just read the pages one by one.
+                    // At the end, close the file
+                    bufferCache.openFile(fileId);
+                    final long start = System.currentTimeMillis();
+                    int pageNumber = 0;
+                    int totalReads = 0;
+                    int successfulReads = 0;
+                    int interruptedReads = 0;
+                    while (System.currentTimeMillis() - start < duration) {
+                        totalReads++;
+                        pageNumber = (pageNumber + 1) % numPages;
+                        try {
+                            long dpid = BufferedFileHandle.getDiskPageId(fileId, pageNumber);
+                            ICachedPage page = bufferCache.pin(dpid, false);
+                            successfulReads++;
+                            bufferCache.unpin(page);
+                        } catch (HyracksDataException hde) {
+                            interruptedReads++;
+                            // clear
+                            Thread.interrupted();
+                        }
+                    }
+                    bufferCache.closeFile(fileId);
+                    LOGGER.log(Level.INFO, "Total reads = " + totalReads + " Successful Reads = " + successfulReads
+                            + " Interrupted Reads = " + interruptedReads);
+                    return null;
+                }
+            });
+        }
+
+        for (int i = 0; i < bufferCacheNumPages; i++) {
+            synchronized (readers[i]) {
+                while (readers[i].getValue() == null) {
+                    readers[i].wait();
+                }
+            }
+        }
+        final long start = System.currentTimeMillis();
+
+        while (System.currentTimeMillis() - start < duration) {
+            for (int i = 0; i < bufferCacheNumPages; i++) {
+                readers[i].getValue().interrupt();
+            }
+            Thread.sleep(25); // NOSONAR Sleep so some reads are successful
+        }
+        try {
+            for (int i = 0; i < bufferCacheNumPages; i++) {
+                futures[i].get();
+            }
+        } finally {
+            bufferCache.deleteFile(fileId);
+            bufferCache.close();
+        }
+    }
+
+    /**
+     * Compressed files are immutable.
+     */
+    @Test
+    public void simpleOpenPinCloseTest() throws HyracksException {
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, ACTUAL_MAX_OPEN_FILE);
+        IBufferCache bufferCache =
+                TestStorageManagerComponentHolder.getBufferCache(ctx.getJobletContext().getServiceContext());
+
+        IIOManager ioManager = TestStorageManagerComponentHolder.getIOManager();
+        FileReference file = getFileReference(ioManager);
+        int fileId = bufferCache.createFile(file);
+        int num = 10;
+        int testPageId = 0;
+
+        bufferCache.openFile(fileId);
+        final ICompressedPageWriter writer = bufferCache.getCompressedPageWriter(fileId);
+
+        ICachedPage page = null;
+
+        // confiscating a page should succeed
+        page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, testPageId));
+        writer.prepareWrite(page);
+
+        for (int i = 0; i < num; i++) {
+            page.getBuffer().putInt(i * 4, i);
+        }
+        final IFIFOPageQueue queue = bufferCache.createFIFOQueue();
+        queue.put(page, HaltOnFailureCallback.INSTANCE);
+        bufferCache.finishQueue();
+        writer.endWriting();
+        bufferCache.closeFile(fileId);
+
+        // open file again
+        bufferCache.openFile(fileId);
+
+        // tryPin should succeed because page should still be cached
+        page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId), false);
+        Assert.assertNotNull(page);
+        try {
+            // verify contents of page
+            for (int i = 0; i < num; i++) {
+                Assert.assertEquals(page.getBuffer().getInt(i * 4), i);
+            }
+        } finally {
+            bufferCache.unpin(page);
+        }
+
+        bufferCache.closeFile(fileId);
+        bufferCache.close();
+    }
+
+    @Test
+    public void contentCheckingMaxOpenFilesTest() throws HyracksException {
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, ACTUAL_MAX_OPEN_FILE);
+        IBufferCache bufferCache =
+                TestStorageManagerComponentHolder.getBufferCache(ctx.getJobletContext().getServiceContext());
+        IIOManager ioManager = TestStorageManagerComponentHolder.getIOManager();
+
+        List<Integer> fileIds = new ArrayList<>();
+        Map<Integer, ArrayList<Integer>> pageContents = new HashMap<>();
+        int num = 10;
+        int testPageId = 0;
+
+        // open max number of files and write some stuff into their first page
+        for (int i = 0; i < MAX_OPEN_DATA_FILES; i++) {
+            FileReference file = getFileReference(ioManager);
+            int fileId = bufferCache.createFile(file);
+            fileIds.add(fileId);
+            bufferCache.openFile(fileId);
+            final ICompressedPageWriter writer = bufferCache.getCompressedPageWriter(fileId);
+            ICachedPage page = null;
+            page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, testPageId));
+            writer.prepareWrite(page);
+            ArrayList<Integer> values = new ArrayList<>();
+            for (int j = 0; j < num; j++) {
+                int x = Math.abs(rnd.nextInt());
+                page.getBuffer().putInt(j * 4, x);
+                values.add(x);
+            }
+            pageContents.put(fileId, values);
+            final IFIFOPageQueue queue = bufferCache.createFIFOQueue();
+            queue.put(page, HaltOnFailureCallback.INSTANCE);
+            bufferCache.finishQueue();
+            writer.endWriting();
+        }
+
+        boolean exceptionThrown = false;
+
+        // since all files are open, next open should fail
+        try {
+            FileReference file = getFileReference(ioManager);
+            int fileId = bufferCache.createFile(file);
+            bufferCache.openFile(fileId);
+        } catch (HyracksDataException e) {
+            exceptionThrown = true;
+        }
+        Assert.assertTrue(exceptionThrown);
+
+        // close a few random files
+        ArrayList<Integer> closedFileIds = new ArrayList<>();
+        int filesToClose = 5;
+        for (int i = 0; i < filesToClose; i++) {
+            int ix = Math.abs(rnd.nextInt()) % fileIds.size();
+            bufferCache.closeFile(fileIds.get(ix));
+            closedFileIds.add(fileIds.get(ix));
+            fileIds.remove(ix);
+        }
+
+        // now open a few new files
+        for (int i = 0; i < filesToClose; i++) {
+            FileReference file = getFileReference(ioManager);
+            int fileId = bufferCache.createFile(file);
+            bufferCache.openFile(fileId);
+            fileIds.add(fileId);
+        }
+
+        // since all files are open, next open should fail
+        exceptionThrown = false;
+        try {
+            FileReference file = getFileReference(ioManager);
+            int fileId = bufferCache.createFile(file);
+            bufferCache.openFile(fileId);
+        } catch (HyracksDataException e) {
+            exceptionThrown = true;
+        }
+        Assert.assertTrue(exceptionThrown);
+
+        // close a few random files again
+        for (int i = 0; i < filesToClose; i++) {
+            int ix = Math.abs(rnd.nextInt()) % fileIds.size();
+            bufferCache.closeFile(fileIds.get(ix));
+            closedFileIds.add(fileIds.get(ix));
+            fileIds.remove(ix);
+        }
+
+        // now open those closed files again and verify their contents
+        for (int i = 0; i < filesToClose; i++) {
+            int closedFileId = closedFileIds.get(i);
+            bufferCache.openFile(closedFileId);
+            fileIds.add(closedFileId);
+
+            // pin first page and verify contents
+            ICachedPage page = null;
+            page = bufferCache.pin(BufferedFileHandle.getDiskPageId(closedFileId, testPageId), false);
+            try {
+                ArrayList<Integer> values = pageContents.get(closedFileId);
+                for (int j = 0; j < values.size(); j++) {
+                    Assert.assertEquals(values.get(j).intValue(), page.getBuffer().getInt(j * 4));
+                }
+            } finally {
+                bufferCache.unpin(page);
+            }
+        }
+
+        for (Integer i : fileIds) {
+            bufferCache.closeFile(i.intValue());
+        }
+
+        bufferCache.close();
+    }
+
+    @Test
+    public void interruptedConcurrentReadTest() throws Exception {
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, 200, ACTUAL_MAX_OPEN_FILE);
+        IBufferCache bufferCache =
+                TestStorageManagerComponentHolder.getBufferCache(ctx.getJobletContext().getServiceContext());
+        IIOManager ioManager = TestStorageManagerComponentHolder.getIOManager();
+        FileReference file = getFileReference(ioManager);
+        int fileId = bufferCache.createFile(file);
+        int testPageId = 0;
+        bufferCache.openFile(fileId);
+        bufferCache.getCompressedPageWriter(fileId).endWriting();
+
+        final int expectedPinCount = 100;
+        final AtomicInteger actualPinCount = new AtomicInteger(0);
+        Thread innocentReader = new Thread(() -> {
+            Thread interruptedReader = null;
+            try {
+                for (int i = 0; i < expectedPinCount; i++) {
+                    ICachedPage aPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId), false);
+                    bufferCache.unpin(aPage);
+                    ((CachedPage) aPage).invalidate();
+                    actualPinCount.incrementAndGet();
+                    if (i % 10 == 0) {
+                        // start an interruptedReader that will cause the channel to closed
+                        interruptedReader = new Thread(() -> {
+                            try {
+                                Thread.currentThread().interrupt();
+                                bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId + 1), false);
+                            } catch (Exception e) {
+                                e.printStackTrace();
+                            }
+                        });
+                        interruptedReader.start();
+                    }
+                }
+                if (interruptedReader != null) {
+                    interruptedReader.join();
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        innocentReader.start();
+        innocentReader.join();
+        // make sure that all reads by the innocentReader succeeded
+        Assert.assertEquals(actualPinCount.get(), expectedPinCount);
+        // close file
+        bufferCache.closeFile(fileId);
+    }
+
+    @AfterClass
+    public static void cleanup() throws Exception {
+        for (String s : openFiles) {
+            File f = new File(s);
+            f.deleteOnExit();
+        }
+    }
+
+}


[10/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
[ASTERIXDB-2422][STO] Introduce compressed storage

- user model changes: yes
  - Add new configuration in the with-caluse to enable compression
  - Add new nc configuration in the config file
- storage format changes: yes
  - Pages of the primary index can be compressed
  - Add a companion file (Look Aside File) with the compressed index
  - Allow optional values in the LocalResource
  - Add compression information in Metadata.Dataset
- interface changes: yes
  - ICCApplicationContext:
    - Add getCompressionManager()
  - IBufferCache:
    - Add getCompressedFileWriter(int fileId)
  - ICachedPageInternal:
    - Add setCompressedPageOffset(long offset)
    - Add getCompressedPageOffset()
    - Add setCompressedPageSize(int size)
    - Add getCompressedPageSize()

Details:
  - Add new integration test for this patch
  - Fix ASTERIXDB-2464
  - Add ddl-with-clause type validator

Additional details in the design document:
https://cwiki.apache.org/confluence/display/ASTERIXDB/Compression+in+AsterixDB

Change-Id: Idde6f37c810c30c7f1a5ee8bcbc1e3e5f4410031
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2857
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <mh...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/5aeba9b4
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/5aeba9b4
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/5aeba9b4

Branch: refs/heads/master
Commit: 5aeba9b475fc714edf953bd88ada7281a2d4937e
Parents: 8076fe9
Author: Wail Alkowaileet <wa...@gmail.com>
Authored: Tue Nov 6 18:52:34 2018 -0800
Committer: Wail Alkowaileet <wa...@gmail.com>
Committed: Tue Nov 6 23:12:28 2018 -0800

----------------------------------------------------------------------
 .../compression/incompressible_large_object.adm |   1 +
 .../data/compression/incompressible_object.adm  |   2 +
 .../app/io/PersistedResourceRegistry.java       |  28 ++
 .../asterix/app/translator/QueryTranslator.java |   4 +-
 .../SqlppExecutionWithCompresisionTest.java     |  64 +++
 .../src/test/resources/cc-compression.conf      |  57 +++
 .../large-page/large-page.1.ddl.sqlpp           |  36 ++
 .../large-page/large-page.2.update.sqlpp        |  25 ++
 .../large-page/large-page.3.query.sqlpp         |  24 ++
 .../small-page/small-page.1.ddl.sqlpp           |  36 ++
 .../small-page/small-page.2.update.sqlpp        |  25 ++
 .../small-page/small-page.3.query.sqlpp         |  24 ++
 .../invalid-compression-scheme.1.ddl.sqlpp      |  36 ++
 .../scheme-none/scheme-none.1.ddl.sqlpp         |  42 ++
 .../scheme-none/scheme-none.2.update.sqlpp      |  25 ++
 .../scheme-none/scheme-none.3.query.sqlpp       |  27 ++
 .../scheme-snappy/scheme-snappy.1.ddl.sqlpp     |  43 ++
 .../scheme-snappy/scheme-snappy.2.update.sqlpp  |  25 ++
 .../scheme-snappy/scheme-snappy.3.query.sqlpp   |  27 ++
 .../cross-dv02/cross-dv02.3.query.sqlpp         |  14 +-
 .../cross-dv04/cross-dv04.3.query.sqlpp         |  14 +-
 .../cross-dv19/cross-dv19.3.query.sqlpp         |  15 +-
 .../missing-non-optional.1.ddl.sqlpp            |  41 ++
 .../type-mismatch/type-mismatch.1.ddl.sqlpp     |  42 ++
 .../unsupported-field.1.ddl.sqlpp               |  43 ++
 .../unsupported-subfield.1.ddl.sqlpp            |  43 ++
 .../all_datasets_compressed.1.ddl.sqlpp         |  66 +++
 .../all_datasets_compressed.10.post.http        |  20 +
 .../all_datasets_compressed.11.get.http         |  20 +
 .../all_datasets_compressed.12.get.http         |  20 +
 .../all_datasets_compressed.13.query.sqlpp      |  35 ++
 .../all_datasets_compressed.14.query.sqlpp      |  22 +
 .../all_datasets_compressed.15.query.sqlpp      |  22 +
 .../all_datasets_compressed.16.query.sqlpp      |  22 +
 .../all_datasets_compressed.2.update.sqlpp      |  24 ++
 .../all_datasets_compressed.3.query.sqlpp       |  35 ++
 .../all_datasets_compressed.4.post.http         |  20 +
 .../all_datasets_compressed.5.get.http          |  20 +
 .../all_datasets_compressed.6.get.http          |  20 +
 .../all_datasets_compressed.7.query.sqlpp       |  35 ++
 .../all_datasets_compressed.8.query.sqlpp       |  22 +
 .../all_datasets_compressed.9.query.sqlpp       |  22 +
 .../single_dataset_compressed.1.ddl.sqlpp       |  49 +++
 .../single_dataset_compressed.10.query.sqlpp    |  22 +
 .../single_dataset_compressed.11.query.sqlpp    |  22 +
 .../single_dataset_compressed.2.update.sqlpp    |  23 ++
 .../single_dataset_compressed.3.query.sqlpp     |  22 +
 .../single_dataset_compressed.4.post.http       |  20 +
 .../single_dataset_compressed.5.get.http        |  20 +
 .../single_dataset_compressed.6.query.sqlpp     |  22 +
 .../single_dataset_compressed.7.query.sqlpp     |  22 +
 .../single_dataset_compressed.8.post.http       |  20 +
 .../single_dataset_compressed.9.get.http        |  20 +
 ...le_dataset_with_index_compressed.1.ddl.sqlpp |  51 +++
 ...le_dataset_with_index_compressed.10.get.http |  20 +
 ...dataset_with_index_compressed.11.query.sqlpp |  25 ++
 ...dataset_with_index_compressed.12.query.sqlpp |  22 +
 ...dataset_with_index_compressed.13.query.sqlpp |  25 ++
 ...dataset_with_index_compressed.2.update.sqlpp |  23 ++
 ..._dataset_with_index_compressed.3.query.sqlpp |  25 ++
 ...le_dataset_with_index_compressed.4.post.http |  20 +
 ...gle_dataset_with_index_compressed.5.get.http |  20 +
 ..._dataset_with_index_compressed.6.query.sqlpp |  25 ++
 ..._dataset_with_index_compressed.7.query.sqlpp |  22 +
 ..._dataset_with_index_compressed.8.query.sqlpp |  25 ++
 ...le_dataset_with_index_compressed.9.post.http |  20 +
 .../single_dataverse_compressed.1.ddl.sqlpp     |  65 +++
 .../single_dataverse_compressed.10.post.http    |  20 +
 .../single_dataverse_compressed.11.get.http     |  20 +
 .../single_dataverse_compressed.12.get.http     |  20 +
 .../single_dataverse_compressed.13.query.sqlpp  |  36 ++
 .../single_dataverse_compressed.14.query.sqlpp  |  22 +
 .../single_dataverse_compressed.15.query.sqlpp  |  22 +
 .../single_dataverse_compressed.2.update.sqlpp  |  26 ++
 .../single_dataverse_compressed.3.query.sqlpp   |  36 ++
 .../single_dataverse_compressed.4.post.http     |  20 +
 .../single_dataverse_compressed.5.get.http      |  20 +
 .../single_dataverse_compressed.6.get.http      |  20 +
 .../single_dataverse_compressed.7.query.sqlpp   |  36 ++
 .../single_dataverse_compressed.8.query.sqlpp   |  22 +
 .../single_dataverse_compressed.9.query.sqlpp   |  22 +
 .../bulkload.1.sto.cmd                          |  19 +
 .../bulkload.10.post.http                       |  19 +
 .../bulkload.11.pollget.http                    |  21 +
 .../bulkload.12.query.sqlpp                     |  22 +
 .../bulkload.13.sto.cmd                         |  19 +
 .../bulkload.14.sto.cmd                         |  19 +
 .../bulkload.2.sto.cmd                          |  19 +
 .../bulkload.3.pollget.http                     |  21 +
 .../bulkload.4.pollget.http                     |  21 +
 .../bulkload.5.ddl.sqlpp                        |  48 +++
 .../bulkload.6.update.sqlpp                     |  23 ++
 .../bulkload.7.sto.cmd                          |  19 +
 .../bulkload.8.sto.cmd                          |  19 +
 .../bulkload.9.post.http                        |  19 +
 .../flushed_component_compressed.1.sto.cmd      |  19 +
 .../flushed_component_compressed.10.post.http   |  19 +
 .../flushed_component_compressed.11.post.http   |  19 +
 ...flushed_component_compressed.12.pollget.http |  21 +
 .../flushed_component_compressed.13.query.sqlpp |  22 +
 .../flushed_component_compressed.14.sto.cmd     |  19 +
 .../flushed_component_compressed.15.sto.cmd     |  19 +
 .../flushed_component_compressed.2.sto.cmd      |  19 +
 .../flushed_component_compressed.3.pollget.http |  21 +
 .../flushed_component_compressed.4.pollget.http |  21 +
 .../flushed_component_compressed.5.ddl.sqlpp    |  57 +++
 .../flushed_component_compressed.6.update.sqlpp |  22 +
 .../flushed_component_compressed.7.sleep.sqlpp  |  19 +
 .../flushed_component_compressed.8.sto.cmd      |  19 +
 .../flushed_component_compressed.9.sto.cmd      |  19 +
 .../src/test/resources/runtimets/rebalance.xml  |  21 +
 .../test/resources/runtimets/replication.xml    |  11 +
 .../large-page/large-page.1.adm                 |   1 +
 .../small-page/small-page.1.adm                 |   2 +
 .../compression/scheme-none/scheme-none.1.adm   | 100 +++++
 .../scheme-snappy/scheme-snappy.1.adm           | 100 +++++
 .../cross-dataverse/cross-dv02/cross-dv02.3.ast |  68 +++-
 .../cross-dataverse/cross-dv04/cross-dv04.3.ast |  68 +++-
 .../cross-dataverse/cross-dv19/cross-dv19.3.ast |  73 +++-
 .../resources/runtimets/testsuite_sqlpp.xml     |  59 +++
 .../common/config/StorageProperties.java        |  10 +-
 .../common/dataflow/ICcApplicationContext.java  |   8 +-
 .../asterix/common/exceptions/ErrorCode.java    |   4 +-
 .../common/storage/ICompressionManager.java     |  49 +++
 .../main/resources/asx_errormsg/en.properties   |  10 +-
 asterixdb/asterix-lang-common/pom.xml           |   4 +
 .../common/statement/CreateFeedStatement.java   |   8 +-
 .../lang/common/statement/DatasetDecl.java      |  92 ++---
 .../common/util/ConfigurationTypeValidator.java | 191 +++++++++
 .../lang/common/util/ConfigurationUtil.java     |  74 ++++
 .../common/util/DatasetDeclParametersUtil.java  |  97 +++++
 .../lang/common/util/ExpressionUtils.java       |   9 +-
 .../lang/common/util/MergePolicyUtils.java      |  67 ----
 .../metadata/bootstrap/MetadataBootstrap.java   |   4 +-
 .../metadata/bootstrap/MetadataRecordTypes.java |   5 +
 .../metadata/dataset/hints/DatasetHints.java    |   3 +
 .../declared/BTreeResourceFactoryProvider.java  |  13 +-
 .../metadata/declared/MetadataProvider.java     |   5 +
 .../asterix/metadata/entities/Dataset.java      |  28 +-
 .../DatasetTupleTranslator.java                 |  73 +++-
 .../DatasetTupleTranslatorTest.java             |   8 +-
 .../IndexTupleTranslatorTest.java               |   8 +-
 .../asterix/object/base/AdmObjectNode.java      |   8 +
 .../runtime/compression/CompressionManager.java | 128 ++++++
 .../runtime/utils/CcApplicationContext.java     |  12 +-
 .../compression/ICompressorDecompressor.java    |  64 +++
 .../ICompressorDecompressorFactory.java         |  39 ++
 .../apache/hyracks/api/io/FileReference.java    |  12 +-
 .../api/io/IPersistedResourceRegistry.java      |  12 +
 .../lsm/btree/LSMBTreeOperatorTestHelper.java   |   3 +-
 .../hyracks/storage/am/btree/impls/BTree.java   |   8 +-
 .../AppendOnlyLinkedMetadataPageManager.java    |   7 +-
 .../am/common/impls/AbstractTreeIndex.java      |  16 +-
 .../dataflow/ExternalBTreeLocalResource.java    |   6 +-
 .../ExternalBTreeLocalResourceFactory.java      |   3 +-
 .../ExternalBTreeWithBuddyLocalResource.java    |   6 +-
 ...ernalBTreeWithBuddyLocalResourceFactory.java |   3 +-
 .../btree/dataflow/LSMBTreeLocalResource.java   |  20 +-
 .../dataflow/LSMBTreeLocalResourceFactory.java  |   8 +-
 .../lsm/btree/impls/LSMBTreeDiskComponent.java  |   9 +-
 .../am/lsm/btree/impls/LSMBTreeFileManager.java |  20 +-
 .../am/lsm/btree/utils/LSMBTreeUtil.java        |  11 +-
 .../impls/AbstractLSMIndexFileManager.java      |  34 +-
 .../hyracks/storage/am/rtree/impls/RTree.java   |   9 +-
 .../hyracks/hyracks-storage-common/pom.xml      |   5 +
 .../AbstractBufferedFileIOManager.java          | 287 +++++++++++++
 .../storage/common/buffercache/BufferCache.java | 238 ++++-------
 .../buffercache/BufferCacheHeaderHelper.java    |  90 +++++
 .../storage/common/buffercache/CachedPage.java  |  21 +
 .../common/buffercache/IBufferCache.java        |   8 +
 .../common/buffercache/ICachedPageInternal.java |   8 +
 .../compression/NoOpCompressorDecompressor.java |  46 +++
 .../NoOpCompressorDecompressorFactory.java      |  47 +++
 .../SnappyCompressorDecompressor.java           |  66 +++
 .../SnappyCompressorDecompressorFactory.java    |  48 +++
 .../compression/file/CompressedFileManager.java | 285 +++++++++++++
 .../file/CompressedFileReference.java           |  86 ++++
 .../compression/file/ICompressedPageWriter.java |  48 +++
 .../common/compression/file/LAFWriter.java      | 260 ++++++++++++
 .../common/compression/file/NoOpLAFWriter.java  |  44 ++
 .../storage/common/file/BufferedFileHandle.java | 133 +++++-
 .../file/CompressedBufferedFileHandle.java      | 261 ++++++++++++
 .../am/lsm/btree/LSMBTreeExamplesTest.java      |   4 +-
 ...MBTreeModificationOperationCallbackTest.java |   4 +-
 .../LSMBTreeSearchOperationCallbackTest.java    |   4 +-
 .../am/lsm/btree/LSMBTreeUpdateInPlaceTest.java |   4 +-
 .../btree/impl/TestLsmBtreeLocalResource.java   |   6 +-
 .../impl/TestLsmBtreeLocalResourceFactory.java  |   3 +-
 .../multithread/LSMBTreeMultiThreadTest.java    |   4 +-
 .../am/lsm/btree/perf/LSMTreeRunner.java        |   5 +-
 .../am/lsm/btree/util/LSMBTreeTestContext.java  |  10 +-
 .../common/BufferCacheWithCompressionTest.java  | 401 +++++++++++++++++++
 192 files changed, 6536 insertions(+), 424 deletions(-)
----------------------------------------------------------------------



[06/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.6.query.sqlpp
new file mode 100644
index 0000000..ca42f3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.6.query.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 tpch;
+
+select value count(*) from LineItem;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.7.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.7.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
new file mode 100644
index 0000000..1a677fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
@@ -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.
+ */
+
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1,asterix_nc2"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.9.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.9.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.9.get.http
new file mode 100644
index 0000000..360a01f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.9.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.1.ddl.sqlpp
new file mode 100644
index 0000000..4aef0e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.1.ddl.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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 tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType)
+primary key l_orderkey,l_linenumber
+with {"storage-block-compression": {"scheme": "snappy"}};
+
+create index lineitem_shipdateIx on LineItem (l_shipdate);
+create index lineitem_receiptdateIx on LineItem (l_receiptdate);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.10.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.10.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.10.get.http
new file mode 100644
index 0000000..360a01f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.10.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.11.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.11.query.sqlpp
new file mode 100644
index 0000000..a82d537
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.11.query.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 tpch;
+
+select value l
+from LineItem l
+where l_shipdate="1994-01-20"
+order by l_orderkey, l_linenumber;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.12.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.12.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.12.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.13.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.13.query.sqlpp
new file mode 100644
index 0000000..d8fda37
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.13.query.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 tpch;
+
+explain select value l
+from LineItem l
+where l_shipdate="1994-01-20"
+order by l_orderkey, l_linenumber;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.2.update.sqlpp
new file mode 100644
index 0000000..8a59946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 tpch;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+      (`format`=`delimited-text`),(`delimiter`=`|`));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.3.query.sqlpp
new file mode 100644
index 0000000..a82d537
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.3.query.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 tpch;
+
+select value l
+from LineItem l
+where l_shipdate="1994-01-20"
+order by l_orderkey, l_linenumber;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
new file mode 100644
index 0000000..737c1fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
@@ -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.
+ */
+
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.5.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.5.get.http
new file mode 100644
index 0000000..360a01f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.5.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.6.query.sqlpp
new file mode 100644
index 0000000..a82d537
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.6.query.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 tpch;
+
+select value l
+from LineItem l
+where l_shipdate="1994-01-20"
+order by l_orderkey, l_linenumber;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.7.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.7.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.8.query.sqlpp
new file mode 100644
index 0000000..d8fda37
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.8.query.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 tpch;
+
+explain select value l
+from LineItem l
+where l_shipdate="1994-01-20"
+order by l_orderkey, l_linenumber;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
new file mode 100644
index 0000000..1a677fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
@@ -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.
+ */
+
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1,asterix_nc2"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.1.ddl.sqlpp
new file mode 100644
index 0000000..466a1d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.1.ddl.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * 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 tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create dataset LineItem(LineItemType)
+primary key l_orderkey,l_linenumber
+with {"storage-block-compression": {"scheme": "snappy"}};
+
+create  dataset Orders(OrderType)
+primary key o_orderkey
+with {"storage-block-compression": {"scheme": "snappy"}};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
new file mode 100644
index 0000000..5a1c111
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
@@ -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.
+ */
+
+/admin/rebalance?dataverseName=tpch&nodes="asterix_nc1,asterix_nc2"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.11.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.11.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.11.get.http
new file mode 100644
index 0000000..360a01f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.11.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.12.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.12.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.12.get.http
new file mode 100644
index 0000000..4288376
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.12.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch&datasetName=Orders

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.13.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.13.query.sqlpp
new file mode 100644
index 0000000..ce9e993
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.13.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.14.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.14.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.14.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.15.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.15.query.sqlpp
new file mode 100644
index 0000000..0505471
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.15.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "Orders";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.2.update.sqlpp
new file mode 100644
index 0000000..44a89bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.2.update.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.
+ */
+
+use tpch;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),
+      (`delimiter`=`|`));
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),
+      (`delimiter`=`|`));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.3.query.sqlpp
new file mode 100644
index 0000000..ce9e993
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.3.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
new file mode 100644
index 0000000..d8f5812
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
@@ -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.
+ */
+
+/admin/rebalance?dataverseName=tpch&nodes="asterix_nc1"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.5.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.5.get.http
new file mode 100644
index 0000000..360a01f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.5.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.6.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.6.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.6.get.http
new file mode 100644
index 0000000..4288376
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.6.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch&datasetName=Orders

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.7.query.sqlpp
new file mode 100644
index 0000000..ce9e993
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.7.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.8.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.8.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.9.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.9.query.sqlpp
new file mode 100644
index 0000000..0505471
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.9.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "Orders";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.1.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.1.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.1.sto.cmd
new file mode 100644
index 0000000..1947749
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.1.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc2 /addReplica 2 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.10.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.10.post.http
new file mode 100644
index 0000000..a3ea801
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.10.post.http
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.11.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.11.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.11.pollget.http
new file mode 100644
index 0000000..32e2f78
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.11.pollget.http
@@ -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.
+ */
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.12.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.12.query.sqlpp
new file mode 100644
index 0000000..cd777cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.12.query.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 tpch;
+
+select value count(*) from LineItem;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.13.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.13.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.13.sto.cmd
new file mode 100644
index 0000000..1e192f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.13.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc2 /removeReplica 2 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.14.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.14.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.14.sto.cmd
new file mode 100644
index 0000000..530432f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.14.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc2 /removeReplica 3 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.2.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.2.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.2.sto.cmd
new file mode 100644
index 0000000..f3810f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.2.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc2 /addReplica 3 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.3.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.3.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.3.pollget.http
new file mode 100644
index 0000000..4ea16d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.3.pollget.http
@@ -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.
+ */
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.4.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.4.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.4.pollget.http
new file mode 100644
index 0000000..22558bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.4.pollget.http
@@ -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.
+ */
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.5.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.5.ddl.sqlpp
new file mode 100644
index 0000000..babf76f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.5.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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 tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create dataset LineItem(LineItemType)
+primary key l_orderkey,l_linenumber
+with {"storage-block-compression": {"scheme": "snappy"}};
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.6.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.6.update.sqlpp
new file mode 100644
index 0000000..e53f462
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.6.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 tpch;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+      (`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.7.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.7.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.7.sto.cmd
new file mode 100644
index 0000000..389cf68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.7.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc1 /promote 2 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.8.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.8.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.8.sto.cmd
new file mode 100644
index 0000000..257f26a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.8.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc1 /promote 3 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.9.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.9.post.http
new file mode 100644
index 0000000..36e1d00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.9.post.http
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.1.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.1.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.1.sto.cmd
new file mode 100644
index 0000000..1947749
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.1.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc2 /addReplica 2 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.10.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.10.post.http
new file mode 100644
index 0000000..36e1d00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.10.post.http
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.11.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.11.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.11.post.http
new file mode 100644
index 0000000..a3ea801
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.11.post.http
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.12.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.12.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.12.pollget.http
new file mode 100644
index 0000000..32e2f78
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.12.pollget.http
@@ -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.
+ */
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file


[07/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
index a7034e2..d4c217b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -59,6 +59,7 @@ import org.apache.asterix.metadata.utils.SecondaryCorrelatedTreeIndexOperationsH
 import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
 import org.apache.asterix.om.pointables.nonvisitor.AListPointable;
 import org.apache.asterix.om.pointables.nonvisitor.ARecordPointable;
+import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
@@ -248,6 +249,9 @@ public class PersistedResourceRegistry implements IPersistedResourceRegistry {
         REGISTERED_CLASSES.put("UTF8NGramTokenFactory", UTF8NGramTokenFactory.class);
         REGISTERED_CLASSES.put("UTF8WordTokenFactory", UTF8WordTokenFactory.class);
         REGISTERED_CLASSES.put("RTreePolicyType", RTreePolicyType.class);
+
+        //ICompressorDecompressorFactory
+        CompressionManager.registerCompressorDecompressorsFactoryClasses(REGISTERED_CLASSES);
     }
 
     @Override
@@ -265,6 +269,30 @@ public class PersistedResourceRegistry implements IPersistedResourceRegistry {
     }
 
     @Override
+    public IJsonSerializable deserializeOrDefault(JsonNode json, Class<? extends IJsonSerializable> defaultClass)
+            throws HyracksDataException {
+        if (json != null) {
+            return deserialize(json);
+        }
+
+        return deserializeDefault(defaultClass);
+    }
+
+    private IJsonSerializable deserializeDefault(Class<? extends IJsonSerializable> defaultClass)
+            throws HyracksDataException {
+        //Ensure it is registered
+        final String resourceId = getResourceId(defaultClass);
+        try {
+            Class<? extends IJsonSerializable> clazz = getResourceClass(resourceId);
+            //Using static method (fromJson)
+            Method method = clazz.getMethod(DESERIALIZATION_METHOD, IPersistedResourceRegistry.class, JsonNode.class);
+            return (IJsonSerializable) method.invoke(null, this, null);
+        } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
     public ObjectNode getClassIdentifier(Class<? extends IJsonSerializable> clazz, long version) {
         final ObjectNode objectNode = JSON_MAPPER.createObjectNode();
         objectNode.put(IPersistedResourceRegistry.TYPE_FIELD_ID, getResourceId(clazz));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index cffa178..3638e55 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -554,6 +554,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         String nodegroupName = ngNameId == null ? null : ngNameId.getValue();
         String compactionPolicy = dd.getCompactionPolicy();
         Map<String, String> compactionPolicyProperties = dd.getCompactionPolicyProperties();
+        String compressionScheme = metadataProvider.getCompressionManager()
+                .getDdlOrDefaultCompressionScheme(dd.getDatasetCompressionScheme());
         boolean defaultCompactionPolicy = compactionPolicy == null;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
@@ -652,7 +654,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             dataset = new Dataset(dataverseName, datasetName, itemTypeDataverseName, itemTypeName,
                     metaItemTypeDataverseName, metaItemTypeName, ngName, compactionPolicy, compactionPolicyProperties,
                     datasetDetails, dd.getHints(), dsType, DatasetIdFactory.generateDatasetId(),
-                    MetadataUtil.PENDING_ADD_OP);
+                    MetadataUtil.PENDING_ADD_OP, compressionScheme);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
             if (dd.getDatasetType() == DatasetType.INTERNAL) {
                 JobSpecification jobSpec = DatasetUtil.createDatasetJobSpec(dataset, metadataProvider);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionWithCompresisionTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionWithCompresisionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionWithCompresisionTest.java
new file mode 100644
index 0000000..af50bde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionWithCompresisionTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.util.Collection;
+
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+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++ storage runtime tests with the Snappy compression.
+ */
+@RunWith(Parameterized.class)
+public class SqlppExecutionWithCompresisionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-compression.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppExecutionWithCompressionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_sqlpp.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppExecutionWithCompresisionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/cc-compression.conf
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/cc-compression.conf b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
new file mode 100644
index 0000000..904707a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
@@ -0,0 +1,57 @@
+; 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.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=21MB
+storage.memorycomponent.globalbudget=512MB
+storage.compression.block=snappy
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+
+[common]
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+messaging.frame.size=4096
+messaging.frame.count=512

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.1.ddl.sqlpp
new file mode 100644
index 0000000..ad8ef0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test incompressible pages
+ * Expected Res : Success
+ * Date         : 24 Jul 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.2.update.sqlpp
new file mode 100644
index 0000000..3c4c3c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.2.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 test;
+
+set `compiler.sortmemory` "64MB";
+
+load dataset Compressed using localfs
+(("path" = "asterix_nc1://data/compression/incompressible_large_object.adm"), ("format" = "adm"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.3.query.sqlpp
new file mode 100644
index 0000000..01959ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+use test;
+
+select value array_count(c.array_doubles)
+from Compressed as c;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.1.ddl.sqlpp
new file mode 100644
index 0000000..ad8ef0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test incompressible pages
+ * Expected Res : Success
+ * Date         : 24 Jul 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.2.update.sqlpp
new file mode 100644
index 0000000..6c414c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.2.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 test;
+
+set `compiler.sortmemory` "64MB";
+
+load dataset Compressed using localfs
+(("path" = "asterix_nc1://data/compression/incompressible_object.adm"), ("format" = "adm"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.3.query.sqlpp
new file mode 100644
index 0000000..01959ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+use test;
+
+select value array_count(c.array_doubles)
+from Compressed as c;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/invalid-compression-scheme/invalid-compression-scheme.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/invalid-compression-scheme/invalid-compression-scheme.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/invalid-compression-scheme/invalid-compression-scheme.1.ddl.sqlpp
new file mode 100644
index 0000000..92be222
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/invalid-compression-scheme/invalid-compression-scheme.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test providing invalid compression scheme
+ * Expected Res : Error: Throws Unknown compression scheme zip. Supported schemes are [lz4,lz4hc,snappy,none]
+ * Date         : 24 Jul 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as {
+  id: int
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {"storage-block-compression": {"scheme": "zip"}};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.1.ddl.sqlpp
new file mode 100644
index 0000000..2885755
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test compression without compression
+ * Expected Res : Success
+ * Date         : 24 Jul 2018
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType)
+primary key id
+with {"storage-block-compression": {"scheme": "none"}};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.2.update.sqlpp
new file mode 100644
index 0000000..86eacdf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.2.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.3.query.sqlpp
new file mode 100644
index 0000000..02f8073
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.3.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+select element paper
+from  DBLP1 as paper
+order by paper.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.1.ddl.sqlpp
new file mode 100644
index 0000000..a27a0be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test compression with snappy
+ * Expected Res : Success
+ * Date         : 24 Jul 2018
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType)
+primary key id
+with {"storage-block-compression": {"scheme": "snappy"}};
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.2.update.sqlpp
new file mode 100644
index 0000000..86eacdf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.2.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.3.query.sqlpp
new file mode 100644
index 0000000..02f8073
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.3.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+select element paper
+from  DBLP1 as paper
+order by paper.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp
index e9b2b51..d71e6f5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp
@@ -24,7 +24,19 @@
  * Date         : 28th Aug 2012
  */
 
-select element l
+select l.DataverseName,
+  l.DatasetName,
+  l.DatatypeDataverseName,
+  l.DatatypeName,
+  l.DatasetType,
+  l.GroupName,
+  l.CompactionPolicy,
+  l.CompactionPolicyProperties,
+  l.InternalDetails,
+  l.`Hints`,
+  l.Timestamp,
+  l.DatasetId,
+  l.PendingOp
 from  `Metadata`.`Dataset` as l
 where ((l.DataverseName = 'student') or (l.DataverseName = 'teacher'))
 order by l.DatasetName

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp
index b9c51a3..5435d9e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp
@@ -27,7 +27,19 @@
  * Date         : 28th Aug 2012
  */
 
-select element l
+select l.DataverseName,
+  l.DatasetName,
+  l.DatatypeDataverseName,
+  l.DatatypeName,
+  l.DatasetType,
+  l.GroupName,
+  l.CompactionPolicy,
+  l.CompactionPolicyProperties,
+  l.InternalDetails,
+  l.`Hints`,
+  l.Timestamp,
+  l.DatasetId,
+  l.PendingOp
 from  `Metadata`.`Dataset` as l
 where ((l.DataverseName = 'student') or (l.DataverseName = 'teacher'))
 order by l.DatasetName

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp
index a74c316..c7ef7cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp
@@ -22,7 +22,20 @@
  * Date         : Sep 20 2012
  */
 
-select element l
+select l.DataverseName,
+  l.DatasetName,
+  l.DatatypeDataverseName,
+  l.DatatypeName,
+  l.DatasetType,
+  l.GroupName,
+  l.CompactionPolicy,
+  l.CompactionPolicyProperties,
+  l.ExternalDetails,
+  l.InternalDetails,
+  l.`Hints`,
+  l.Timestamp,
+  l.DatasetId,
+  l.PendingOp
 from  `Metadata`.`Dataset` as l
 where ((l.DataverseName = 'test1') or (l.DataverseName = 'test2') or (l.DataverseName = 'TwitterData'))
 order by l.DataverseName, l.DatasetName, l.DatatypeDataverseName

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/missing-non-optional/missing-non-optional.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/missing-non-optional/missing-non-optional.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/missing-non-optional/missing-non-optional.1.ddl.sqlpp
new file mode 100644
index 0000000..246a380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/missing-non-optional/missing-non-optional.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test
+ * Expected Res : Excpetion: WITH_FIELD_MUST_CONTAIN_SUB_FIELD
+ * Date         : 1 Nov 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {
+  "merge-policy": {
+    "parameters": { "max-mergable-component-size": 16384, "max-tolerance-component-count": 3 }
+  }
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/type-mismatch/type-mismatch.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/type-mismatch/type-mismatch.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/type-mismatch/type-mismatch.1.ddl.sqlpp
new file mode 100644
index 0000000..4fea63f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/type-mismatch/type-mismatch.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test
+ * Expected Res : Excpetion: WITH_FIELD_MUST_BE_OF_TYPE
+ * Date         : 1 Nov 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {
+  "merge-policy": {
+    "name": "correlated-prefix",
+    "parameters": { "max-mergable-component-size": "16384", "max-tolerance-component-count": 3 }
+  }
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-field/unsupported-field.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-field/unsupported-field.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-field/unsupported-field.1.ddl.sqlpp
new file mode 100644
index 0000000..2119c61
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-field/unsupported-field.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test
+ * Expected Res : Excpetion: UNSUPPORTED_WITH_FIELD
+ * Date         : 1 Nov 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {
+  "merge-policy": {
+    "name": "correlated-prefix",
+    "parameters": { "max-mergable-component-size": 16384, "max-tolerance-component-count": 3 }
+  },
+  "unknown-field": "is not accepted"
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-subfield/unsupported-subfield.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-subfield/unsupported-subfield.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-subfield/unsupported-subfield.1.ddl.sqlpp
new file mode 100644
index 0000000..81f7b51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-subfield/unsupported-subfield.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test
+ * Expected Res : Excpetion: WITH_FIELD_MUST_CONTAIN_SUB_FIELD
+ * Date         : 1 Nov 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {
+  "merge-policy": {
+    "name": "correlated-prefix",
+    "parameters": { "max-mergable-component-size": 16384, "max-tolerance-component-count": 3 },
+    "unknown-subfield": "is not accepted"
+  }
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.1.ddl.sqlpp
new file mode 100644
index 0000000..06c5840
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.1.ddl.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * 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 tpch1 if exists;
+create  dataverse tpch1;
+
+drop  dataverse tpch2 if exists;
+create  dataverse tpch2;
+
+
+create type tpch1.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type tpch2.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create dataset tpch1.LineItem(LineItemType)
+primary key l_orderkey,l_linenumber
+with {"storage-block-compression": {"scheme": "snappy"}};
+
+create  dataset tpch2.Orders(OrderType)
+primary key o_orderkey
+with {"storage-block-compression": {"scheme": "snappy"}};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
new file mode 100644
index 0000000..156c15e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
@@ -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.
+ */
+
+/admin/rebalance?nodes="asterix_nc1,asterix_nc2"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.11.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.11.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.11.get.http
new file mode 100644
index 0000000..e5fe873
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.11.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch1&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.12.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.12.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.12.get.http
new file mode 100644
index 0000000..6e444ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.12.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch2&datasetName=Orders

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.13.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.13.query.sqlpp
new file mode 100644
index 0000000..a30c374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.13.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  tpch1.LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM tpch2.Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.14.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.14.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.14.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.15.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.15.query.sqlpp
new file mode 100644
index 0000000..0505471
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.15.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "Orders";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.16.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.16.query.sqlpp
new file mode 100644
index 0000000..7edd168
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.16.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "Dataset";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.2.update.sqlpp
new file mode 100644
index 0000000..373ab5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+load  dataset tpch1.LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+      (`format`=`delimited-text`), (`delimiter`=`|`));
+
+load  dataset tpch2.Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+      (`format`=`delimited-text`), (`delimiter`=`|`));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.3.query.sqlpp
new file mode 100644
index 0000000..a30c374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.3.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  tpch1.LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM tpch2.Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
new file mode 100644
index 0000000..4e2b902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
@@ -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.
+ */
+
+/admin/rebalance?nodes="asterix_nc1"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.5.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.5.get.http
new file mode 100644
index 0000000..e5fe873
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.5.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch1&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.6.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.6.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.6.get.http
new file mode 100644
index 0000000..6e444ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.6.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch2&datasetName=Orders

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.7.query.sqlpp
new file mode 100644
index 0000000..a30c374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.7.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  tpch1.LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM tpch2.Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.8.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.8.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.9.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.9.query.sqlpp
new file mode 100644
index 0000000..0505471
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.9.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "Orders";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.1.ddl.sqlpp
new file mode 100644
index 0000000..df26d68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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 tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType)
+primary key l_orderkey,l_linenumber
+with {"storage-block-compression": {"scheme": "snappy"}};
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.10.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.10.query.sqlpp
new file mode 100644
index 0000000..ca42f3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.10.query.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 tpch;
+
+select value count(*) from LineItem;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.11.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.11.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.11.query.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.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.2.update.sqlpp
new file mode 100644
index 0000000..8a59946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 tpch;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+      (`format`=`delimited-text`),(`delimiter`=`|`));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.3.query.sqlpp
new file mode 100644
index 0000000..ca42f3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.3.query.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 tpch;
+
+select value count(*) from LineItem;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
new file mode 100644
index 0000000..737c1fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
@@ -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.
+ */
+
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.5.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.5.get.http
new file mode 100644
index 0000000..360a01f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.5.get.http
@@ -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.
+ */
+
+/connector?dataverseName=tpch&datasetName=LineItem


[09/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/data/compression/incompressible_large_object.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/data/compression/incompressible_large_object.adm b/asterixdb/asterix-app/data/compression/incompressible_large_object.adm
new file mode 100644
index 0000000..f8e1aac
--- /dev/null
+++ b/asterixdb/asterix-app/data/compression/incompressible_large_object.adm
@@ -0,0 +1 @@
+{"id":0, "array_doubles":[0.8353859890622932, 0.22500827673705914, 0.22157022791391168, 0.32035245758794395, 0.34939474725656017, 0.5554287293317939, 0.06613512953798861, 0.9681347508140643, 0.6623415178580739, 0.5919439929535106, 0.05212040031639453, 0.3892855164316662, 0.8844497593574515, 0.6074932969350857, 0.7711957613447981, 0.8631583218045213, 0.1770457827094719, 0.4525687775376275, 0.8036709263952709, 0.5019105296004952, 0.40277490848861697, 0.3190257275538467, 0.7110984656234003, 0.47492940777795445, 0.6588714514705232, 0.8422160078737491, 0.6587109939654401, 0.7413350665701102, 0.8347485080603378, 0.01498504222097563, 0.6551497676477125, 0.8641987866071527, 0.9416184262349903, 0.5638318471854301, 0.4565123230471907, 0.23499879702437088, 0.7245505917442545, 0.02053900526300123, 0.8975670677530877, 0.6979679511672306, 0.9863184718241791, 0.7118370974009355, 0.7850529576017674, 0.13137044593192237, 0.12758276331945362, 0.28517618646209486, 0.5163029978250586, 0.016377992191920
 93, 0.6394033369931067, 0.19043485821420558, 0.27100002466638606, 0.40409840155892873, 0.6717675614750906, 0.745404515677119, 0.964694454966989, 0.9774422135729012, 0.3457680486842587, 0.027145395602065636, 0.14978284412603105, 0.09689613474086856, 0.1772583959126579, 0.851817571134352, 0.7029676527431258, 0.8179046088632121, 0.836260348127726, 0.5323490307372913, 0.4477877164800176, 0.8724437851767138, 0.47909147244416905, 0.9363096377075704, 0.826630340403748, 0.5185049307619617, 0.0591278224570877, 0.6056187878344298, 0.3587053620587216, 0.9785870656178437, 0.3473107559787938, 0.34522114147657645, 0.898782433631359, 0.9119317000304087, 0.8749535739799605, 0.7396238130336652, 0.6460626445222623, 0.995114802462112, 0.6490973667424694, 0.5141953106193369, 0.4433764232814532, 0.2217446083229403, 0.28461222664174113, 0.5223847738684045, 0.15396944033783244, 0.9743962859771625, 0.14743461508521505, 0.4105277061797251, 0.8064587833050655, 0.2705472366999413, 0.20977679144649508, 0.68866
 25866980012, 0.7581122349133979, 0.839159905125601, 0.580840032967774, 0.018789554722417168, 0.342423334544698, 0.8456916113612959, 0.0955084362426809, 0.021261763607677175, 0.3390254486454911, 0.15069384533534935, 0.9186953318108272, 0.795506176239649, 0.1323084784948293, 0.9607958089401623, 0.8544597784952086, 0.6530294504855796, 0.6096719468989508, 0.6107507786732062, 0.9367059800745291, 0.2686721024230544, 0.691168728727381, 0.0013555592010559003, 0.92222336647191, 0.5703316191423874, 0.8793637744292353, 0.9104370043899721, 0.5070212780975464, 0.20788364967407158, 0.3715525360297284, 0.9927316460777484, 0.44335624313740585, 0.3687874777328216, 0.0860545310122598, 0.7893680959550368, 0.24754639259535172, 0.04087043020742853, 0.4201169616155035, 0.47879385372934047, 0.8576121464237345, 0.5620434058822834, 0.6790345554522537, 0.4183716243074923, 0.424977533820756, 0.4422071989205446, 0.8944112317086809, 0.7495028636692472, 0.4059125754468552, 0.9943679332633895, 0.18406028257852403
 , 0.4690186185609001, 0.25609149191727487, 0.2699868860840089, 0.896859601201321, 0.40571157235176947, 0.32558764303904175, 0.15784836022047277, 0.5329487541371086, 0.4832648479788375, 0.0018208421281796783, 0.2762365786044213, 0.8329754628279173, 0.49946678078557394, 0.7544183688493702, 0.46497106817803957, 0.4896688642550392, 0.4064833256880278, 0.4986460577435724, 0.2668571583537266, 0.6508375244374913, 0.7380925988988857, 0.6130114153387053, 0.7773661705853161, 0.7549604023395863, 0.397168991286312, 0.02183574359078644, 0.6368683230634229, 0.26885938730935377, 0.4903368813041652, 0.9850376750522303, 0.7414979635370078, 0.5815120643082735, 0.2765971441756202, 0.6250662364010331, 0.45673994414693575, 0.6488840558988345, 0.5828619403295261, 0.7648657934523091, 0.7320825197263998, 0.9817300836078994, 0.2317167114370684, 0.35095198602327304, 0.5355910622654452, 0.221634123363029, 0.12536063689782695, 0.5921219388371786, 0.4540135793971187, 0.8601280196190044, 0.6345596166045365, 0.95
 70176434432645, 0.13703794438703332, 0.6215494538255102, 0.8525793260886928, 0.3427072240032266, 0.5035778211793611, 0.6523698093894362, 0.09144750451720662, 0.051643351535156756, 0.273472138075992, 0.9380429939713437, 0.7215287654149819, 0.6958038006037093, 0.4397412103692946, 0.3029965532382284, 0.08527703645335194, 0.784514589186673, 0.7545500585208672, 0.940206646690063, 0.7521641124716618, 0.5464009638064378, 0.1662871871624697, 0.6638689241066048, 0.8241652118915335, 0.48610740162357824, 0.4156753346107207, 0.9819977723027192, 0.3498157234856416, 0.28287253407244817, 0.6755881763331116, 0.645355917066487, 0.15973561072652742, 0.9007407177517643, 0.20225483116999843, 0.5449381532331123, 0.5976513057298175, 0.19913686110876494, 0.4547917694659974, 0.7023015382906737, 0.6883270942425279, 0.9077267587060027, 0.03551005794922535, 0.6253562782319632, 0.2574514408036823, 0.5609303964423067, 0.1052025918462095, 0.48680063418280517, 0.39840611944126914, 0.8751507839382795, 0.6356250389
 441056, 0.42320435733103856, 0.46697899449085467, 0.09572652984533137, 0.9756412274652144, 0.7498148862765809, 0.47746389955380575, 0.04081154411321142, 0.7280115852304494, 0.03948127381327704, 0.37222428863484525, 0.9258304497949561, 0.860561549524682, 0.8236606121865943, 0.14167966219114514, 0.5648536647917183, 0.30847382719846694, 0.13929286202418534, 0.1828068860195442, 0.21418171574656708, 0.06370743501572396, 0.2673991970545776, 0.6016313199188558, 0.14862302469664457, 0.8930526588012075, 0.7875804714413934, 0.38881533470452445, 0.04239548866817067, 0.3780905013276109, 0.5071616691135529, 0.5273346929521358, 0.3642767474222771, 0.9283317783373297, 0.9145493473886707, 0.12736622199054393, 0.2905738915486633, 0.7830696124956467, 0.5534386982990959, 0.6258926509105323, 0.08967856490764692, 0.7879863136591094, 0.11782791547116012, 0.9371637280031505, 0.538418409074168, 0.9093139644056136, 0.1273670323413092, 0.19503533568053544, 0.6931114174047233, 0.03135539595090464, 0.758389373
 4582627, 0.3893408428052567, 0.2571751334615425, 0.5724305040150481, 0.7780438951358732, 0.7658327056653681, 0.8623377874910653, 0.8498199456404653, 0.557266580833382, 0.3328635934887805, 0.8119435197740464, 0.9260372758419227, 0.822900339476027, 0.19786083914167718, 0.22253987872720027, 0.8767667887191496, 0.15009177639683136, 0.7427795420137674, 0.04388027144375972, 0.03506383076060504, 0.19316507149326823, 0.8401029613309663, 0.13767003965074642, 0.7869206129025924, 0.3064921811963376, 0.4095918182887037, 0.45182947381115246, 0.9200598060679294, 0.5164497612403659, 0.6208665658502489, 0.339743282937226, 0.3608246360387287, 0.7713729352150592, 0.9580396884901152, 0.24291021642679111, 0.3732931918388648, 0.018487125796476822, 0.17999801983285701, 0.2806029933235842, 0.9402516526569102, 0.9351182984316809, 0.9335674223040599, 0.7715930975970625, 0.09741934038054767, 0.0760533730226648, 0.406675762613563, 0.3506909067145595, 0.45032636831792083, 0.789742661540051, 0.9447620791080007,
  0.1367966060497391, 0.9675020177373276, 0.5031055547401347, 0.996063226498628, 0.35935274464056044, 0.41414047159179135, 0.8780226024167288, 0.10004199912740752, 0.9565347806518504, 0.4671677254189486, 0.90216590174722, 0.8881030887038365, 0.9212108323563561, 0.635641628994703, 0.19264100463268452, 0.22687698466402306, 0.5896476124422604, 0.5996747328655427, 0.6629836968980145, 0.7017336477050021, 0.643134779275433, 0.4765776217438785, 0.23237464546087383, 0.611821398636598, 0.3241448672050964, 0.8246776986442755, 0.4405064531848849, 0.5278090150403909, 0.2563416468868692, 0.9032769512227845, 0.6723566049520555, 0.24964160511528533, 0.30169518772720305, 0.8214694326500231, 0.19419128832388055, 0.6940041169752231, 0.08815660089319388, 0.7290471245649632, 0.3554445086498037, 0.8398187779248226, 0.19779141473608708, 0.12948105728453518, 0.12068786790450436, 0.36498569787880086, 0.5179656142771164, 0.7279717758854606, 0.5617260523746306, 0.018708613756401427, 0.9271233374610145, 0.9668
 215163894344, 0.06192427798944056, 0.09113081891332186, 0.31189125528849304, 0.5458880655361705, 0.7631929930115235, 0.5656387895988371, 0.8936892672991418, 0.777638054985954, 0.22062693845538173, 0.6690739470273791, 0.9807228662890946, 0.5843255054634809, 0.32061468865236553, 0.5712259766004212, 0.5551777125845686, 0.30907871545431376, 0.2653058062018421, 0.927363822379401, 0.5556323537110611, 0.2831873475707364, 0.8296429010187638, 0.8061930341052319, 0.32115018286374475, 0.26863803362138083, 0.15673619558593543, 0.30051156426248304, 0.42620702172647407, 0.47795607804467377, 0.07317845922901622, 0.7789681438212199, 0.05455658693963339, 0.8626674348749306, 0.27796847266058755, 0.3047110322925114, 0.13533027500316142, 0.7572038639757042, 0.7870316840992571, 0.220346843139974, 0.6961144095502548, 0.7668312828051563, 0.6998967494084836, 0.2785128656781246, 0.27544203292609903, 0.9961896516933901, 0.6959072390977359, 0.9834983375858704, 0.781082870173694, 0.3292901624481307, 0.66441793
 14067685, 0.0017333886977698443, 0.8807095092961701, 0.4923237556964084, 0.5267452400870454, 0.597597237679028, 0.8679634052873876, 0.057392209070833755, 0.6172124015769943, 0.05587952953091713, 0.9596401179292597, 0.37828148113615734, 0.9564492378191886, 0.9633621523438356, 0.14865758071622392, 0.6886542806392905, 0.3721775220919771, 0.039371260943652064, 0.06266737423428237, 0.23854695400942505, 0.7010329872746882, 0.9287821425864429, 0.759613842452502, 0.3968136731507218, 0.23954512533534977, 8.425237111984174E-4, 0.029241217916994078, 0.10516820283037687, 0.20321668486585276, 0.733312460122401, 0.5634943392421479, 0.9337320652125436, 0.19971448145927762, 0.703802074222364, 0.30895399196663575, 0.258505120422696, 0.6255042523192753, 0.5588208652504856, 0.4208928155383046, 0.7579326667206011, 0.4548316973692854, 0.5859354630176191, 0.5012333913135502, 0.5087511931399413, 0.10577286234774252, 0.6755654792673146, 0.22163532220675486, 0.45055196744682746, 0.9401470868357267, 0.526329
 3100343238, 0.008133415149142786, 0.8840105590997338, 0.48240581554005024, 0.9214858456984, 0.19519437850584187, 0.9694872168509081, 0.627181503813748, 0.7838992248718403, 0.9447548076596834, 0.9007113199927351, 0.9460002690527081, 0.5030880885850223, 0.26974839978057585, 0.5425210532778811, 0.7007331712206972, 0.7203026049052722, 0.701826088220545, 0.8421366904037744, 0.3246807849709996, 0.35620564093689133, 0.4994172702627482, 0.25020442130683584, 0.941755870592959, 0.17061756270302275, 0.8049766036483089, 0.7208549166566659, 0.9600501318681526, 0.5549165582498513, 0.829205450733648, 0.5992788750376741, 0.8981400621240695, 0.5750691634228464, 0.9652325321939834, 0.8508295416694933, 0.38080859800273126, 0.5068990764631086, 0.7171838953219764, 0.6374199648746411, 0.6754282698926716, 0.23508432049913364, 0.46526738363347153, 0.20499089567783968, 0.23107301871500596, 0.923506228089565, 0.38344198092903414, 0.41008238091049687, 0.4294954002619078, 0.3069564781420412, 0.2759382157111784
 , 0.9840505308737686, 0.16042651569209077, 0.8028408853210908, 0.7053406117950739, 0.810228647327581, 0.893226095668618, 0.39640158542339843, 0.5476386121039944, 0.0723392935535232, 0.6327309031712448, 0.9141127516474774, 0.02859803797501681, 0.12172221228299929, 0.5192195459147763, 0.8672862296996823, 0.3709998582283819, 0.3064506847665206, 0.860732065847133, 0.521558216305488, 0.4137391851589741, 0.8071663477145705, 0.07055356860689055, 0.22780941179524872, 0.9514715463223165, 0.7324708363579607, 0.3234903902123122, 0.3362044068322294, 0.08744028047493746, 0.3409702176248248, 0.7416390141551527, 0.20741935968733538, 0.5923127406510057, 0.27757579227272944, 0.5736669340763476, 0.933758838327004, 0.11829951955501028, 0.1384230747942764, 0.14119574818451508, 0.3877463192236421, 0.1305251950384586, 0.5117780702134279, 0.29169703601321595, 0.12690567432089994, 0.5865654030403541, 0.8815582052515809, 0.7193813785281924, 0.5008326664599363, 0.7202594481000216, 0.4256030966626583, 0.79388
 20718341421, 0.03029016922162886, 0.7673537038730573, 0.7925874537919323, 0.4711041510191172, 0.757941704831586, 0.10822566835756797, 0.3994773637025294, 0.4514312013965569, 0.8064721209683582, 0.0968846711983603, 0.6556879871242103, 0.9975511737992669, 0.8222263054819534, 0.7956397691386494, 0.7906090273013825, 0.7274839693085154, 0.331129911314626, 0.9412712161586065, 0.6379222545981054, 0.2579631931967177, 0.28853876141213053, 0.7312367622988465, 0.813142534730378, 0.7891449993422347, 0.26579644666182345, 0.4373899014691902, 0.210072557222571, 0.5936872075502084, 0.7836627679057899, 0.42988195262501094, 0.637368380759557, 0.9015623539212465, 0.2764077200445668, 0.9253654577061289, 0.7181713562901854, 0.856538209086299, 0.05024930967556673, 0.6613117407929626, 0.8837693791303389, 0.061080433416377033, 0.38594597029211475, 0.7422004917333395, 0.406508680164707, 0.7235011539643911, 0.6277370915132627, 0.10353498691462149, 0.937684511701168, 0.5744989195598216, 0.6994581366154118, 0.
 39608141567304955, 0.9428978276016312, 0.5298045921557242, 0.9935394493524786, 0.6054263374696867, 0.760371764115012, 0.5414688984828472, 0.9897310854550142, 0.39089180736200635, 0.7469806144003263, 0.202397169030328, 0.27777822206377556, 0.9530424900085874, 0.12225106253948892, 0.447057466520716, 0.8819048666330238, 0.22046474478697753, 0.6723533990104551, 0.12585813974539717, 0.9148465879461967, 0.2841931635002648, 0.43410872777890674, 0.32151194683978157, 0.6253051701704809, 0.30210485605562676, 0.8432951711441653, 0.02277314556197385, 0.8450816070055694, 0.5457974432377064, 0.6020445836607321, 0.1742552360971883, 0.690776263165084, 0.8689119775735473, 0.6968789613684981, 0.18060675826185468, 0.12982782478290955, 0.6385572011195664, 0.5819429646324217, 0.659143163318965, 0.6817957196517188, 0.09043706512598926, 0.5312478482789149, 0.6071027363178011, 0.09317332702385717, 0.14347850608995216, 0.5592488414275167, 0.17801546216333208, 0.2976520514773111, 0.9412020046659028, 0.110836
 5753592333, 0.9462433425267771, 0.3536393997867915, 0.7747726137960035, 0.9298305258037727, 0.8795713952172032, 0.33696666487119364, 0.5866144666957338, 0.12932049371312782, 0.23679906164006703, 0.3591050427195225, 0.9536673980912791, 0.5077099273430641, 0.2606013111177369, 0.9798412879954054, 0.3058088773498887, 0.5810725521998422, 0.8454999261389642, 0.2882475422562708, 0.1476608208951573, 0.6354417413594516, 0.3266053445102727, 0.8369120184846316, 0.22094507474839742, 0.3419099692028832, 0.15685946831491748, 0.43178052382818655, 0.8697475489054266, 0.12292295068708214, 0.518830815305988, 0.5035915758444576, 0.3524956715746288, 0.5099117769509192, 0.8516830804975344, 0.4167704588711213, 0.4821497846830488, 0.19041347650192564, 0.8647590529628723, 0.6077806171450012, 0.7700534316772653, 0.3070882244191421, 0.8562914365108764, 0.8873132617907411, 0.1990840837079454, 0.8415498303356004, 0.05907334381438756, 0.58932960402105, 0.85818991261739, 0.5926835352041221, 0.2944905848380911, 0
 .5048517217006946, 0.9330946065265994, 0.3421324906765759, 0.4321390326432136, 0.6995323848895715, 0.9668966000045076, 0.2888934459846655, 0.024319955170687302, 0.3814155009514749, 0.8794850022734163, 0.7304684167327546, 0.013027111445998774, 0.958932823933532, 0.9661399122022204, 0.2697781502437776, 0.5916092853254659, 0.3090646762336845, 0.9833792758290912, 0.5350501901605212, 0.14411245324638622, 0.061725558806502656, 0.4740611149828664, 0.5141526912624306, 0.6823760454815403, 0.39881818919489465, 0.6839435890287636, 0.1253526683430377, 0.3542780909574599, 0.4650324197726664, 0.7783525619647461, 0.5732353173015282, 0.40609429966523525, 0.9923483388160008, 0.5802997728663019, 0.3715286027699719, 0.44775658739217183, 0.15133537616986836, 0.7203415015769173, 0.65383561201477, 0.9918664994789098, 0.9597226303322013, 0.9136620764244053, 0.53878441157524, 0.01563999328044341, 0.8023476248667066, 0.06131242680093274, 0.8678633674942218, 0.7295293511671701, 0.7464743588136589, 0.01022833
 9258798402, 0.5922174778285663, 0.8270894617432436, 0.372603857964662, 0.10059981193152567, 0.44893942439542966, 0.23772493064155176, 0.7832829665194936, 0.07935662867846538, 0.3378960061752455, 0.8072757404108536, 0.3010510487447813, 0.5649818177248475, 0.9868899457327824, 0.390968507932103, 0.12154544038145498, 0.011559057641423398, 0.07596286157397814, 0.55908194491175, 0.8489209346709438, 0.03890656330710485, 0.9809051625274465, 0.4877440396201117, 0.408120228315177, 0.07084347453069428, 0.6915445493870325, 0.6355998870733782, 0.3021655438998059, 0.45508235832548094, 0.2621439312387074, 0.4916589620837608, 0.5625246799450461, 0.6259347032770686, 0.35924667861621684, 0.020450862372348944, 0.02673687705422878, 0.4149627561067136, 0.08656531252439459, 0.4276891174510852, 0.7698673786818722, 0.3149063454682902, 0.2870170749894978, 0.5229626542630686, 0.6149921428643469, 0.35865785855811594, 0.9487685038106446, 0.2655453184321437, 0.6527965570039118, 0.28775466644906045, 0.1751577590
 3187636, 0.0452335034601431, 0.6115057562906321, 0.8538033370734399, 0.4176223724695821, 0.8721928384101966, 0.735745865022884, 0.4214452268742024, 0.31626720169451883, 0.9281148221553078, 0.5565692369627887, 0.5229180711871482, 0.544465571907885, 0.8655548784783862, 0.5647588555488808, 0.6519644001175747, 0.5118939609170441, 0.8331433192878384, 0.33566043261567124, 0.37878178728310585, 0.6446590312571969, 0.45143133505461286, 0.09617316516317143, 0.8179061464264324, 0.23168297013593342, 0.8674542348928053, 0.07316312339967601, 0.5873640982901596, 0.5032714060885815, 0.5756643593453586, 0.21744597057963821, 0.6601259234535015, 0.6907957236749968, 0.9087068637361913, 0.05861855840666508, 0.9278359466910031, 0.3056793627983697, 0.9735814138929638, 0.5134017370567479, 0.8719925287395595, 0.7713858162953737, 0.3370532988514311, 0.44378971951855506, 0.4691948741285401, 0.8438635001544307, 0.9487477719625231, 0.6411835177658276, 0.3447695973165983, 0.9639106422567512, 0.6311820743752927, 
 0.6988200054806613, 0.23303523169485174, 0.7991813410659612, 0.9630590532441978, 0.9271892604618073, 0.0026574978502948987, 0.7540610166410795, 0.5354677649575315, 0.06576745168189291, 0.8619948063410434, 0.15239869725472632, 0.3702055702637572, 0.0492740301849095, 0.8720272536343183, 0.13108953174104665, 0.7552576623486889, 0.49779542603525084, 0.5634088657343707, 0.02176054931820226, 0.8745392801428223, 0.9174998949120897, 0.2793775969336495, 0.14331919519776037, 0.387326981321241, 0.8223016242458814, 0.8477029256375637, 0.47988264800500524, 0.5473750490463997, 0.11671404761058268, 0.32254640300983106, 0.6766135128671812, 0.5425338570894994, 0.11626608879329858, 0.8310556519186834, 0.15024531446918588, 0.4827570910144018, 0.5242371276791956, 0.009707738985193726, 0.8815891384166643, 0.8677051020957297, 0.532099829045869, 0.12186298519959138, 0.6340703696522895, 0.42472201284326305, 0.965942441889573, 0.7106858707365237, 0.9253525581049581, 0.9237843643075019, 0.06227508720913899, 
 0.5605347437891749, 0.06836284192311637, 0.9480897662986004, 0.02400759332765734, 0.8151173661176715, 0.9935418805929509, 0.6272736921466411, 0.9810186859650643, 0.33215503877856745, 0.3658981670572451, 0.02481363385122448, 0.35961985174692246, 0.7419926004486619, 0.7184551182821308, 0.21199963410472988, 0.9513495397566267, 0.22153776439508766, 0.9379367755554112, 0.05792046726106259, 0.16391620013889585, 0.6951079143295154, 0.45904602257746563, 0.41823558597068977, 0.035741758124318124, 0.03020817711717194, 0.9504467143275817, 0.71507267027975, 0.6004396069993865, 0.8089596775622757, 0.10856909053966868, 0.8662111784492639, 0.09869429638910276, 0.42484225133011555, 0.41461598167497726, 0.1437828840261296, 0.3523926155581253, 0.5796114421842303, 0.3556543657393023, 0.8210400381163203, 0.33948887266098493, 0.7366944897487726, 0.6624355429718513, 0.4003127992501907, 0.6379696429880417, 0.7250656097591708, 0.5745940754607776, 0.8060974520972758, 0.9434167860928265, 0.48154613715735106,
  0.00813450481359912, 0.1742873446123292, 0.3070489691031767, 0.8696705947640021, 0.7706216153099937, 0.771878904336109, 0.08975655652421644, 0.6197043341314188, 0.6462485565940382, 0.14105532501617435, 0.8825259342213243, 0.6970949762727773, 0.2559647073068393, 0.7773707083674612, 0.8047830725891008, 0.8255838389172941, 0.08710288160695356, 0.5238280645956681, 0.6595686162730452, 0.4663997613364953, 0.9824535330889405, 0.39096436756096586, 0.3980897884523671, 0.2864027854565113, 0.6804910266143543, 0.6976270017347691, 0.2260768586326335, 0.03574012560086759, 0.7064654373398804, 0.26794526935211305, 0.47050522994379584, 0.7995231148033123, 0.028330611601574684, 0.917071678961763, 0.017626462645062957, 0.7882276129993611, 0.4119872340362746, 0.6115877244084031, 0.7318849255714782, 0.1879674682978245, 0.3370776294319848, 0.5000392762622246, 0.770179155554173, 0.10448437613056138, 0.6675764796589261, 0.4748869199921938, 0.11706001573253433, 0.3036436193271288, 0.01067994144996065, 0.04
 922700399274249, 0.9838403042683139, 0.27642056581703245, 0.8430028130176718, 0.10995947809402495, 0.9382239759781952, 0.13780749442939833, 0.0348519050093965, 0.5436486156981643, 0.7609448739100015, 0.006766116178589354, 0.36135095680314266, 0.8221436867867288, 0.062038580991212955, 0.8082964789715039, 0.9534508358339543, 0.3820961260464698, 0.6317535047282545, 0.8888879815630976, 0.4123660856342757, 0.035830670691015865, 0.34974010554840806, 0.4430636930768148, 0.4877368612067249, 0.7105982102508717, 0.23103964590441872, 0.5114245708566156, 0.8028233866828006, 0.5290577999708409, 0.8485276714372791, 0.2295451043969463, 0.09929730647974921, 0.6159065266554419, 0.9388489418273173, 0.08226810023986586, 0.869702674105732, 0.04389549199053089, 0.4725142672129077, 0.14939014105243997, 0.04741686542727119, 0.017040835356833095, 0.8487123383597535, 0.7164264534506586, 0.631949384178187, 0.7296588466130116, 0.16152202938911842, 0.16808742546696942, 0.7416748384942012, 0.3369248027389101, 0
 .676594067419303, 0.7822956356977048, 0.6066033860522694, 0.1259750979077604, 0.7560032581738608, 0.8990168661147616, 0.4405507267165638, 0.286143029516729, 0.47385568682277857, 0.9508741346139585, 0.9650765848570484, 0.6427383157972408, 0.07636785482252284, 0.01145910950763862, 0.5968192343974121, 0.3408512741230666, 0.2575174752783338, 0.8424504010547953, 0.9553607595159557, 0.743982660785413, 0.9449989442917842, 0.9020938800140137, 0.40051618438181835, 0.7468704462868441, 0.9210984861344618, 0.41529484508555503, 0.7659468429646279, 0.29270521551514217, 0.13923773534617068, 0.3995182917926301, 0.3202565256742943, 0.13199614147096705, 0.03931646291230584, 0.49088726939111027, 0.2572368412763558, 0.7636698229579694, 0.5136187335994031, 0.52131641456422, 0.6936854827568332, 0.0585545964927322, 0.4299288233729901, 0.06021135151882828, 0.23622969623571766, 0.28122314124758263, 0.7796368141281211, 0.5532114084718693, 0.667844361627054, 0.09823665072980914, 0.5859959971027751, 0.26564728
 3452101, 0.20026231021929963, 0.6891541586959358, 0.6813272616894925, 0.9698313979945565, 0.040232218376834106, 0.26124064538900915, 0.3418485394345818, 0.20299365093458854, 0.7255870602758207, 0.10962838645393536, 0.5549400244099694, 0.966620348010587, 0.8013444061420096, 0.8617557036798204, 0.3860388373021143, 0.9381501453300912, 0.9265668340000516, 0.6982280692131909, 0.5109894305377316, 0.6224252386352619, 0.4294695415196552, 0.1518166149425626, 0.4845591502200861, 0.3829920637783235, 0.6612691599620051, 0.5521294898497155, 0.7269607556871112, 0.962290749775438, 0.2385260925480398, 0.30556156440746707, 0.5276069733501375, 0.9249679296611849, 0.4092405025386886, 0.9784488797694282, 0.5889546144960564, 0.43999274457473814, 0.32971955565106315, 0.5859238198758235, 0.763178847447759, 0.954951584020573, 0.9972417798198367, 0.5797878841964734, 0.6705507205871191, 0.6098416956368423, 0.947801788720719, 0.7757889373744672, 0.8307742955538014, 0.6878799085789794, 0.5987268077342649, 0.86
 40101463182734, 0.7871721005117369, 0.007287454959327255, 0.29271381791667683, 0.20564841081023133, 0.45849070575208606, 0.9207073894428829, 0.6828445726060688, 0.3352799222754269, 0.6142310298968094, 0.5078565040149067, 0.08739284344092269, 0.959781215916931, 0.9367905492840505, 0.6222088535035335, 0.8387466106042177, 0.2733942983609484, 0.6073379041332094, 0.6839530476596343, 0.45252538026473854, 0.092994419390662, 0.7322223568389723, 0.499161653670369, 0.6509809401871495, 0.6584400192745117, 0.21285451299090585, 0.9290549765462137, 0.8890272516153802, 0.93995404273566, 0.8543764027264512, 0.9334383890824955, 0.051753550334432075, 0.9478063870374817, 0.6070726260265361, 0.36883621818417034, 0.4190319018131582, 0.8491377465972763, 0.8393566464095615, 0.9345416226979897, 0.8258388683153597, 0.6625851723591712, 0.4522994103221657, 0.8287465981825777, 0.1861683348621449, 0.9234400110142518, 0.4630499481605812, 0.0021116749558744985, 0.19392048545802965, 0.9903892439292712, 0.408130803
 4879114, 0.3775804561456083, 0.06590986971373358, 0.5994617126429763, 0.5474738177450669, 0.042938888855299284, 0.6501431399856727, 0.8616206562612089, 0.38619416230149106, 0.21656527931763647, 0.0867748025394336, 0.14751478183649136, 0.4734843014920461, 0.7370114078938734, 0.479657496154106, 0.20621984940732285, 0.7866413322691169, 0.9335015164917753, 0.7131605022082537, 0.4243736245640881, 0.26312466315907035, 0.9971300186718774, 0.631938800713467, 0.347926961940929, 0.20467354043718122, 0.3914851705546999, 0.2905919920521576, 0.4608042826269031, 0.34489663531325176, 0.41662092824094676, 0.8756607845647423, 0.7430867838662943, 0.34537915988160517, 0.7717438180915818, 0.20416637275158644, 0.7159026669780729, 0.365235172308511, 0.8496782291696278, 0.4109058384801888, 0.5210280602789527, 0.1351659344688133, 0.5401178555896596, 0.5509743256890491, 0.08027324738478447, 0.659468176106353, 0.5188666467329203, 0.36449309152847975, 0.5330491865274379, 0.2680683621379175, 0.3567180900508504
 , 0.3772797461289715, 0.9600791085576885, 0.24554226758143083, 0.2943165670673641, 0.8442051603033515, 0.8670407896320325, 0.6752120860683933, 0.1982905869328343, 0.7812391702548969, 0.023995076359978107, 0.45379579888773103, 0.039141843841237534, 0.477695264651811, 0.03477685673199504, 0.6557162527531516, 0.21282055633069086, 0.43194181445423285, 0.44509468266056806, 0.5560608277271538, 0.16561221091807177, 0.6118557238698392, 0.2889056578255087, 0.6573148444256848, 0.07590604875748841, 0.3055517791229265, 0.7197027245977416, 0.007374526390003133, 0.7966113871589151, 0.5984290447739097, 0.7766714902744589, 0.9449789357884073, 0.8684330551713896, 0.6893334365459391, 0.39524419426828106, 0.16358206678412401, 0.17691191853058852, 0.8416430701016762, 0.4873987531086046, 0.6458202342903571, 0.7450830880814792, 0.7543415476753232, 0.2891211036404505, 0.5881115660809152, 0.22879819384265543, 0.5336511825833102, 0.20076934134835112, 0.9085580252647334, 0.913204547807233, 0.2475569559984176
 5, 0.03064846499358498, 0.3112029628315963, 0.39656894005066745, 0.25355898098277796, 0.8929249205347496, 0.16582994822966257, 0.691578973075578, 0.10424955978015626, 0.008080512098399328, 0.6548714305337017, 0.1744503577769272, 0.8558186991220208, 0.30742503443302904, 0.7376286288051186, 0.6686138749112315, 0.03134938852026603, 0.052758474594836846, 0.48688714634711605, 0.9624510316229753, 0.3237723893065555, 0.6480923602291274, 0.26414480861622724, 0.22949267930956507, 0.36260735824947765, 0.4424114154646629, 0.017052327413427104, 0.14469011120117092, 0.514125506243909, 0.600776652781394, 0.7062985878539497, 0.9718137831214863, 0.7363267475723625, 0.05564079585511217, 0.4990568681901737, 0.5347297857008627, 0.9681316521030818, 0.1410681483545455, 0.902239728337176, 0.7792494945087407, 0.3964468573222113, 0.7596206124269231, 0.45119387123337296, 0.5824435404659399, 0.21527024546632645, 0.8349920960579198, 0.9180988329129032, 0.5727561396928129, 0.1677597392733734, 0.363044136561515
 9, 0.2592163377846801, 0.9869511513440351, 0.24519792697549836, 0.5897153736796229, 0.6812697455414986, 0.008866934370273194, 0.40604173244795694, 0.7163483033885227, 0.27461602002392305, 0.610151897833586, 0.07375529306091055, 0.0612966549729379, 0.6345053231798493, 0.644598372526525, 0.47388061050592, 0.7912473536631359, 0.7716062303345849, 0.835387241361674, 0.578809912115984, 0.17651124154878461, 0.39910736339365593, 0.3385018278783518, 0.5429798775897776, 0.8216067541027384, 0.37608915495756134, 0.7657639952334278, 0.031717147450397776, 0.4757818401660816, 0.6854918627856109, 0.9073913218428905, 0.7737213066855988, 0.8376885894303493, 0.8298895195348316, 0.9066694635539098, 1.7979353909314266E-4, 0.2342465699371561, 0.9394112469874897, 0.3082787583826596, 0.17276710140932394, 0.1758000403302168, 0.20353553389797807, 0.2270098215324331, 0.5482879679434537, 0.5679954727894702, 0.4621143971256064, 0.8650481384178149, 0.6794827827306607, 0.7953755966707081, 0.9208684644142947, 0.74
 37027152813265, 0.909520406983043, 0.9825065360293272, 0.9927260017489535, 0.30939955496636085, 0.2819622038953261, 0.4855250150806, 0.20876202201573968, 0.023955614289415617, 0.058488406368547685, 0.0703130144388906, 0.9911264391090877, 0.6555946083633589, 0.21343577270615532, 0.29307743926591934, 0.835232966377537, 0.24986103624326994, 0.8572037262376607, 0.31493259136559737, 0.10961327919682584, 0.17853428858902176, 0.529843291874423, 0.7541536996040207, 0.24922392810716687, 0.7398399315073252, 0.7016516794528104, 0.20809023329052612, 0.5916258711951341, 0.014700613879448543, 0.6193525934594455, 0.7612974413700475, 0.48534002077390426, 0.07146436928766953, 0.8444505512391178, 0.1372656871167356, 0.5858141579073395, 0.15549331851904524, 0.44460516204097433, 0.14819673305401426, 0.4468201775844194, 0.2001910735609257, 0.8193788744952913, 0.7505969158705862, 0.6137328922474019, 0.8524928119331646, 0.5684701983139289, 0.3240272288067503, 0.43002462807240704, 0.21853660061573055, 0.64
 35514331843579, 0.7561707573331896, 0.6794427326275037, 0.7041806498259426, 0.26408811658430487, 0.5394481266910107, 0.0037197848838858016, 0.6502258776515963, 0.723025555389845, 0.3627320342397725, 0.7592991911640184, 0.9436487469324477, 0.2599847713936114, 0.6162821979416707, 0.6064417101569732, 0.42384491663439705, 0.6376188590169559, 0.818842378212358, 0.6602109636911812, 0.6968286391853447, 0.9210379953465436, 0.222911300274662, 0.4752194539295225, 0.9205294750073286, 0.23071954233777192, 0.7575200482176146, 0.48464853354713344, 0.23077203231577037, 0.9160771547733739, 0.027418195963151315, 0.7689394713263257, 0.7402203209943944, 0.6798707772407991, 0.30339181945076255, 0.19068480908966434, 0.46151375842410836, 0.6728974319616392, 0.3105064527990701, 0.9996325227179813, 0.08101670373191039, 0.473910051514732, 0.14938854381235112, 0.9853483963336798, 0.24104878732229118, 0.17146984606241333, 0.8702293306148812, 0.13496397303048757, 0.8761594235725599, 0.4724452191924271, 0.06001
 518220731639, 0.5395714976357163, 0.19295379578513028, 0.876128326524969, 0.3005589471959509, 0.9852091551679771, 0.08317427578304792, 0.4734479190204465, 0.5082270066906805, 0.2630129707623088, 0.11618553010816646, 0.857515082133117, 0.975827185051961, 0.7379947956840452, 0.7697075833548483, 0.964746246740031, 0.402942869546848, 0.1270382784972065, 0.22658618943806175, 0.4050656167499309, 0.3230691932934707, 0.7540877131786689, 0.8905847188089656, 0.9851972741896399, 0.11422699145770332, 0.7741374368189528, 0.755628013118959, 0.6808143144691777, 0.21430122175354593, 0.9875122271404226, 0.9006231295485292, 0.2713212189585309, 0.2531703670121386, 0.9020621032079555, 0.6759276049946354, 0.21530234016093597, 0.5607636043476636, 0.6432774355569268, 0.5977310572127816, 0.1151782524822863, 0.9135033216686046, 0.16096336759998175, 0.5730615893140106, 0.2963686961533354, 0.44116039784380956, 0.9359954786382682, 0.8408229514962217, 0.6092691617138946, 0.3319701023564746, 0.051810903257462404
 , 0.9332148985896338, 0.5516058308606087, 0.34592690281791005, 0.06267235675825822, 0.037083238273039165, 0.20424120243857902, 0.17449687936562308, 0.564375509462118, 0.36619262495504, 0.27526107968364855, 0.36955610357993796, 0.7529522038914365, 0.5431093009295711, 0.7551805158281779, 0.9051847197128426, 0.5430557256520365, 0.5333271600536946, 0.30036522223454964, 0.4330156636277682, 0.010096517595456267, 0.37446534219549743, 0.48227193221320463, 0.9451582316316665, 0.018829781496909948, 0.7998112793339308, 0.4668691456386841, 0.8682862675449605, 0.9059793734766287, 0.8681778438154732, 0.5216080477584527, 0.5783015670340904, 0.9956145832641013, 0.8809891118494385, 0.9865702928465646, 0.9470530427570949, 0.5433802667492977, 0.1927044329129497, 0.991928211755762, 0.8109128615703729, 0.42931242727454655, 0.00213550987867972, 0.25560972924684944, 0.185580785007611, 0.5488552288472085, 0.8499069543442634, 0.7530690643506496, 0.6423083446499888, 0.3878271333505794, 0.3412557520122048, 0.
 9153213798528742, 0.5078048178253245, 0.7094680628632191, 0.5635603865313963, 0.2819077318029989, 0.45340744491907625, 0.6303305498250054, 0.7744362653846008, 0.21221447151973916, 0.5986681874928312, 0.6166625916446775, 0.19892834791741942, 0.3415788795142556, 0.4703301161068726, 0.09406084734203202, 0.1932781442378526, 0.9578706654091135, 0.02973673321532644, 0.43055729293423584, 0.6079371370068273, 0.36974709499977465, 0.510037492669639, 0.02693382869086136, 0.9139440685364623, 0.6395367725977532, 0.9891234567053707, 0.12794346125258393, 0.5444787774632961, 0.7617154806637989, 0.46504746522302154, 0.754248351527588, 0.9473506307891058, 0.4821947392125697, 0.6977529038686814, 0.8472684720785508, 0.044375033578810785, 0.4741521595237136, 0.23425608402567089, 0.31776745630465775, 0.1500366375442681, 0.5128898461423109, 0.8439973329837698, 0.19388478883415483, 0.9145331432581796, 0.6252154917846707, 0.05689608993242501, 0.4754560354932085, 0.5252391478454784, 0.9280983105197906, 0.618
 7156875343087, 0.6188466831108234, 0.2258246452370125, 0.01685960855367885, 0.4566640845769061, 0.800156509757678, 0.19370803118042235, 0.314916075940337, 0.1827265594911811, 0.23453540550191143, 0.9542188150869839, 0.7551013511668968, 0.25304728902454743, 0.8654838910603857, 0.8037908295612946, 0.5268491962994317, 0.648746630589674, 0.6803942037830326, 0.16470152793291026, 0.39402978283497037, 0.34688135050183755, 0.7572211991665715, 0.4003187137212495, 0.2649878162820498, 0.9098255911342523, 0.9020544995000385, 0.38251117419417324, 0.2793397857632455, 0.23717019859686206, 0.39849663387933965, 0.021824967887027036, 0.665344637674408, 0.8963088674141859, 0.9571344334416452, 0.04622257293488985, 0.017170724187181796, 0.4761994924933155, 0.4270238047511207, 0.5456272475980248, 0.44313909221967085, 0.09335952234091072, 0.21099100910015645, 0.18721039737664824, 0.24403151511644627, 0.5199747425894515, 0.6126860704238087, 0.5503049928778307, 0.39480875838883456, 0.401070489027046, 0.5357
 070669490231, 0.7300232881262128, 0.12359349700994315, 0.6810154617996298, 0.05374655011450802, 0.2151028612488779, 0.3085288330064032, 0.6691982262464675, 0.3755944751028818, 0.058199580076147006, 0.24613213270145629, 0.14053631813121548, 0.6399646582333721, 0.5257919577770064, 0.5840213664654325, 0.4885494397830932, 0.7924945233196521, 0.7400566008417222, 0.8563988250394505, 0.2270469493082975, 0.5209008513103376, 0.3065195552732397, 0.11243519913142552, 0.07105169783746401, 0.8541570772736905, 0.5589839227996771, 0.5857043807496093, 0.41840135888091723, 0.5676904009643334, 0.8471791041913032, 0.2170779276900454, 0.3005394727400673, 0.9789795620159646, 0.2427727128567092, 0.11399485709126933, 0.7743283981290775, 0.06656780013538155, 0.14583129240341752, 0.7057812935235448, 0.28628675751544885, 0.10038693666947485, 0.5365114502545568, 0.43770158758665223, 0.7564743202554538, 0.1324228919846514, 0.03570776812837906, 0.3356808971877838, 0.0013174840722576242, 0.7641342582671122, 0.51
 35803130017437, 0.6691462331527637, 0.6429888739231724, 0.5806698890986298, 0.7163285171624585, 0.241191981252025, 0.83752500332198, 0.5218126317017259, 0.26927462989871453, 0.54328231963873, 0.9946860045889617, 0.24571075238207762, 0.45826442573074433, 0.12596208871582815, 0.15963198066137474, 0.2338401511461723, 0.3915890356447622, 0.05755442847341141, 0.8798106977432715, 0.9842739855172264, 0.5010759083507524, 0.2695820337914293, 0.19002181065917456, 0.14521896518924693, 0.8536936126125644, 0.9206563300627073, 0.8144428462431478, 0.8519102026151796, 0.4139591841799922, 0.6498365808617941, 0.806472882030324, 0.8473396165598518, 0.22263306818286355, 0.735969308597778, 0.6255696683988787, 0.1909255919921925, 0.4386313122073109, 0.7312755903998958, 0.08482687869470529, 0.48911407925794503, 0.32878040258846264, 0.09546184125373014, 0.6938577899681978, 0.6180348497636368, 0.5839852483847385, 0.720887660958758, 0.6686225448611702, 0.8507186067117215, 0.7905761385118111, 0.00983200709985
 9208, 0.2831563304545439, 0.1500142666040677, 0.4920460723125596, 0.34165664603820267, 0.2977258391635075, 0.14322771692202407, 0.03908450033012845, 0.3506738557562217, 0.9137621441160703, 0.9556368915434851, 0.34475990734465467, 0.42478306748726324, 0.07798804735019726, 0.6474205250709809, 0.7368977111098887, 0.23044890995195477, 0.9864785635324779, 0.47217041889530753, 0.33114645075883886, 0.871351065018523, 0.32396306912722417, 0.5550407123029666, 0.7359196834859021, 0.49604402456953545, 0.11449881848811139, 0.8747194316135046, 0.4443466304086605, 0.7842225682172124, 0.05671472594948179, 0.6257913486054628, 0.9685603395820448, 0.7616970520018781, 0.8701454407654886, 0.13669319106679134, 0.6280964460624178, 0.820939665897453, 0.48874599826403686, 0.5233458212450126, 0.916871977669904, 0.657071379944271, 0.9034968499594415, 0.16968003283262278, 0.5421584000871479, 0.06841661331932147, 0.07808085273627163, 0.9436561388399095, 0.5541584513358087, 0.2162606200846967, 0.639932378751979
 5, 0.781009819269215, 0.4662417023432167, 0.9902025787789902, 0.9835303518953332, 0.8988398103707498, 0.9035891376460626, 0.5835047447825351, 0.013627029615597697, 0.4372856351282006, 0.1119423782757819, 0.24709645832691107, 0.0031738333897805004, 0.769926651166877, 0.6083482142116879, 0.4613988831439694, 0.9440272884403665, 0.07216600560278918, 0.36223481524604373, 0.4418949569838464, 0.32808694814740436, 0.9917466630157824, 0.15785105027914093, 0.97737605123235, 0.5123388921717555, 0.42667866661185094, 0.208045704725205, 0.08905533226493734, 0.6933473222783696, 0.48324024357594264, 0.6645586862337268, 0.8942451246793872, 0.04505836024226495, 0.6603476659036925, 0.40144180363534443, 0.36005378231477747, 0.7583494755534634, 0.18647213301047916, 0.33592161804630827, 0.3405650825662663, 0.5615789708169804, 0.221414340492205, 0.5949626885269622, 0.4052210160247144, 0.7984598283518222, 0.327630077350638, 0.03598436023893159, 0.4866370767097157, 0.2929028351739339, 0.21443397136089404, 0
 .9866298439210152, 0.30283606939173113, 0.04004367577908163, 0.2632896537593792, 0.5811817878331308, 0.8418953775738632, 0.22849017592247745, 0.7535292495433399, 0.7401631914499751, 0.9362595851151668, 0.22089450636490715, 0.025226512875554996, 0.6648587383071143, 0.09947844546910845, 0.8982770376974609, 0.039581497698829016, 0.6999363340705036, 0.3498881486818495, 0.2080978346134914, 0.927057177935162, 0.5550438665750105, 0.7037615135349709, 0.8468926328899674, 0.14964132518408835, 0.17020210627001342, 0.3847103838458391, 0.7553601110804897, 0.845989737806533, 0.9600093159707671, 0.5666317449036274, 0.5071077820203952, 0.7768397426962494, 0.7781190116357156, 0.6442464678085809, 0.7318342906541163, 0.04296564069117892, 0.7061471013923318, 0.6082874622998106, 0.4295454948524494, 0.4090808556944302, 0.6901104292810663, 0.09165228309572504, 0.4044733007167034, 0.6344737989672772, 0.12269376013812705, 0.048342356011583654, 0.07568495068815362, 0.6847978406778955, 0.7010875206754985, 0.8
 287692629393884, 0.6059424161697415, 0.07931841309319221, 0.2778343975594465, 0.6230163398510212, 0.6311903039220567, 0.9140528392404446, 0.9660672042822939, 0.8579012887231424, 0.8701222225829589, 0.2936216132893158, 0.46071728818001434, 0.1541938492806023, 0.5465953125244242, 0.0032625708365356942, 0.39371622640802606, 0.8770998974641798, 0.8672328093720062, 0.25734212574993576, 0.5539104797812996, 0.14613131837956872, 0.7623752974879757, 0.27183058975143803, 0.9713530011422346, 0.4097391504393658, 0.7206112095414356, 0.22559148678651786, 0.19083146424727981, 0.33559459257315283, 0.8254375724689882, 0.8989675476266668, 0.5409380615262479, 0.8191611005546571, 0.03717255653418394, 0.9080287061803012, 0.8426811913038926, 0.3712078871390583, 0.3540613077649438, 0.988980647803312, 0.7955844893560092, 0.009173485236078971, 0.7335868331632163, 0.22961480628804565, 0.22941397323460733, 0.0477955028995134, 0.5583187799728287, 0.49331370498858196, 0.5161198437606569, 0.8611806643981302, 0.9
 90202048244993, 0.5360086224665721, 0.6992739681114885, 0.7714972460116083, 0.773008531886003, 0.47761881024365516, 0.3172303077612023, 0.708729050288195, 0.9147452316226755, 0.35688482197335347, 0.4714299907888603, 0.18143921774965133, 0.21013195438949606, 0.4721410669522278, 0.4227188337213782, 0.4002483840389579, 0.747832716498083, 0.09045553435938014, 0.8862397191785535, 0.10374097334632004, 0.3879448654435095, 0.8936975244277198, 0.23940819315951423, 0.7507701131779835, 0.6923204962632896, 0.22247694539232865, 0.05767442665565403, 0.2636403060044099, 0.9436147559993756, 0.7775316630598172, 0.7492203467550638, 0.4334040237607787, 0.6985263259165503, 0.943021956953613, 0.5724358222037965, 0.4315170351255665, 0.7865153747953021, 0.3073244906030902, 0.988963345492444, 0.6113543739259194, 0.787354616023763, 0.27430013828747957, 0.13029574916931153, 0.9285969431090945, 0.7991897316580807, 0.881643789772876, 0.1373568733020082, 0.8612568837042369, 0.7433553199235815, 0.072806391420688
 9, 0.270190510497785, 0.6681170087992314, 0.5430129155866124, 0.7051124947947846, 0.9788280370215833, 0.1886178502323853, 0.2597483613262841, 0.5190795039768806, 0.5766008022903351, 0.15773364072752527, 0.6755856060777475, 0.1425775359073349, 0.7636082234657493, 0.18613634832201542, 0.8674444230249019, 0.9583515875210731, 0.010748375352656825, 0.1389544249293505, 0.03689017315919607, 0.32456052434485483, 0.7336625325687461, 0.6356912400202913, 0.23009368112161688, 0.20537026756238752, 0.2611410994781317, 0.7893753959550078, 0.6112398348080706, 0.7422713858321534, 0.8339830757747612, 0.10477375194730099, 0.6371936109617361, 0.6766740645569039, 0.25799715525728195, 0.6635045321189587, 0.8220528491762066, 0.8204661437547424, 0.37877713455142104, 0.3381473365955151, 0.7772871139021265, 0.9636029735997707, 0.24608016971589985, 0.7979529993198972, 0.8664824561699808, 0.7064303400157302, 0.6920817289705653, 0.49551558382742533, 0.12328223266744631, 0.7093249131164723, 0.060475893914259116,
  0.656775254257146, 0.046934150925000306, 0.7633690488740027, 0.9923355986827025, 0.5359798615036149, 0.5119246522669959, 0.9031597894720574, 0.4998423058649649, 0.8427395239254996, 0.30650124959254577, 0.506485023928878, 0.04044004937167944, 0.30006777185235833, 0.7494543957865538, 0.9554818850611343, 0.9416640572768163, 0.7503093773332243, 0.42535044173684466, 0.9700259707987543, 0.3208397191700295, 0.0901134543124904, 0.3254777108623441, 0.16648232424801546, 0.5940424247759188, 0.5861182714037946, 0.5850879693850978, 0.7435822844932616, 0.7751547404732722, 0.7997714547599905, 0.5670856679102112, 0.45207258549014173, 0.028552302509609828, 0.43271792856710156, 0.2581382834915258, 0.16591671341301228, 0.35789526967724217, 0.893444916178544, 0.8152504435654062, 0.44021038933107115, 0.8886972999268878, 0.2813817648990419, 0.6291659743195493, 0.4146067897875628, 0.9337898677273011, 0.7864097287281383, 0.4555309072636078, 0.30092911686067914, 0.9184941612659424, 0.20974497882386334, 0.0
 05572550831891121, 0.47978160024439054, 0.2862590094773161, 0.4264156848445966, 0.8851182534505836, 0.9233522745933701, 0.7057179803244092, 0.13006209455508022, 0.9240990816595475, 0.260234116299108, 0.2565645714479057, 0.12754588244333875, 0.7482534687144695, 0.3029738101846584, 0.7843391066679789, 0.7838635479375948, 0.9105660417714211, 0.27019004006152847, 0.2831671496765443, 0.9571157124329137, 0.022047532790265056, 0.25248611531188503, 0.990683462722147, 0.9087924888464464, 0.09978166798820598, 0.6950149139390285, 0.8608220050949611, 0.22639272576565705, 0.7847355988381378, 0.7994362742702683, 0.21784690713241106, 0.7534028141101199, 0.17265002590128353, 0.7841402931696285, 0.6889449591352023, 0.8859183511525698, 0.7483967299479697, 0.8839925808745691, 0.9628956246080232, 0.3657001192038958, 0.317509949779191, 0.8194904244924329, 0.5287797150026274, 0.8762132131551871, 0.8881700125886465, 0.22010171085421593, 0.3113831254805406, 0.3312064560213035, 0.17273631879274043, 0.147690
 71407558787, 0.03889757713992026, 0.7346466716042457, 0.7499986314050993, 0.4333187436955239, 0.25371545521267047, 0.8863302347918719, 0.9576346299756751, 0.014339167239079575, 0.10856100706748084, 0.009252428915467403, 0.2689329272472797, 0.8385903838659501, 0.47789593469613345, 0.5828225672351723, 0.24866045791181057, 0.18862201603730577, 0.4584821892899571, 0.2802709678132972, 0.18002386637188328, 0.4712733295048588, 0.0331803833735278, 0.9291013815687655, 0.10799467960916076, 0.6165549482067387, 0.5285077548816394, 0.8402005691753653, 0.353838099163194, 0.2346685333965114, 0.5091742577816274, 0.6842243605908914, 0.5215061365083828, 0.5911309225756182, 0.9217613356293745, 0.33313750114516116, 0.6386257010377409, 0.2773530584405962, 0.6249030673330307, 0.7302920707084123, 0.35313248542617204, 0.17765609619969858, 0.9247744183428891, 0.9041874496006502, 0.041172243265714115, 0.9635902947877515, 0.3723869709423696, 0.699062549197688, 0.27271099233732565, 0.882554945178537, 0.8731376
 266965969, 0.6597285501583245, 0.32836599580118964, 0.050100322811786, 0.1092512366889612, 0.7785585462055403, 0.3642901325548161, 0.8039258883495115, 0.38220316098309826, 0.4893080992510015, 0.38151175339191756, 0.9324710922285021, 0.2449034606715308, 0.37544869622856425, 0.3719534606389511, 0.7521285926715994, 0.7123695322152279, 0.5499535801303388, 0.8854225862496222, 0.07449216427992245, 0.4148089664765693, 0.5589838677320074, 0.7901870084757293, 0.07868435145249608, 0.7884804906980148, 0.9636796268469665, 0.184320522263831, 0.9836920523722743, 0.28739555708743414, 0.4622699432947348, 0.25582114586424964, 0.20177066207703798, 0.6428685329987668, 0.5289719122364935, 0.3865777228105014, 0.3463603492929892, 0.3981190183040155, 0.06665825658975932, 0.948578720846827, 0.06659120812599462, 0.30157496939303186, 0.4970294116009315, 0.05032910698439774, 0.1853359669677127, 0.6552204337413688, 0.04503496093859294, 0.9572601235490278, 0.02890281212744983, 0.3832774349035498, 0.713546288884
 1189, 0.03198790435010124, 0.15384286314709916, 0.4117065838172631, 0.5897906377052893, 0.818597023708054, 0.8999800795420976, 0.9304561182047882, 0.6768144025452689, 0.329695187898153, 0.3286541712024529, 0.2480849730847876, 0.4012511584981112, 0.6858057162293326, 0.1412995013903493, 0.24233007579791455, 0.31292802810651277, 0.003734874751794015, 0.11586648739959993, 0.8603841697240969, 0.26235145962198014, 0.8338713257608387, 0.4472239968928975, 0.7290947939700906, 0.5188717871279822, 0.5059451494532529, 0.13256492635624928, 0.6979913815290494, 0.8753330713862888, 0.4585944826278423, 0.20396906003560655, 0.05699217967842696, 0.774273223124109, 1.5789423259426894E-4, 0.17766082571209596, 0.24583785534526903, 0.8862468297432353, 0.9764311474229035, 0.5631321098285631, 0.10374453775325287, 0.3726009620206223, 0.5056970140328546, 0.6551829267837634, 0.09546392743783949, 0.27304212695132124, 0.282157412660642, 0.11806594739983634, 0.49388338028491385, 0.07259026274436953, 0.94596727794
 73772, 0.06330339814377595, 0.1994946023178047, 0.5230237189572382, 0.6713900407484911, 0.4213776887257832, 0.7335044989288835, 0.22067653046699398, 0.434164572293658, 0.41238010259036517, 0.9540570181191029, 0.27545755794949567, 0.3849943655172885, 0.8318345306238636, 0.213980689885782, 0.645218605545174, 0.3781391497267089, 0.6852960795665282, 0.003671070935885812, 0.35510666032643323, 0.982410592234395, 0.4289573686325143, 0.604163212113803, 0.8732927657569071, 0.23644280686915287, 0.7033776956492072, 0.9131444891566204, 0.3724785902906792, 9.298029682265341E-4, 0.9835837808662378, 0.40060312266098086, 0.4708387705888044, 0.0075915227069958124, 0.011172740640812195, 0.10100344394041061, 0.31656547245515143, 0.6079509033925687, 0.22268150822916333, 0.6553240539625368, 0.5569392502523293, 0.7845745423823492, 0.643369452497378, 0.6789901880376152, 0.8158579572104679, 0.20324979554781664, 0.8766934905898232, 0.15067746523202952, 0.7930135968205893, 0.9566414907861178, 0.8871381733086
 732, 0.09215869734148618, 0.9934144024161534, 0.1782920200980892, 0.24769610339045844, 0.5408115975154211, 0.39770771231768265, 0.8903983143174082, 0.4031762991895841, 0.7040997850211647, 0.09176647909867175, 0.28962011534845866, 0.28665154253489, 0.2292854998129098, 0.9882607488004646, 0.1722438214771872, 0.3187383231556413, 0.7218269233930805, 0.29344241256460457, 0.35334686950601923, 0.4180356369498216, 0.0023593632882223936, 0.2991302027153149, 0.7441775536077232, 0.15593196911813823, 0.10485380379683451, 0.4128891336672329, 0.21778584010890445, 0.6001225507536689, 0.6874923149033791, 0.07865652092151654, 0.5150107191924342, 0.7085615976735176, 0.35299514530409226, 0.3855747229981953, 0.22125032455755944, 0.3592073191516566, 0.35657895965268926, 0.714548468905114, 0.2790109900829014, 0.23080021709860543, 0.3174219240876027, 0.7300521470709977, 0.5920519694078614, 0.7860477321453403, 0.8434559791961826, 0.38897076875215264, 0.02401282601244692, 0.5509715707990103, 0.4295021244969
 339, 0.4087657363232733, 0.4977305227353601, 0.732806636835167, 0.5952100835168144, 0.8140146241058145, 0.9610242524229055, 0.9737230013229154, 0.9919585636476513, 0.7633815455655985, 0.7005454478436567, 0.6987759861397144, 0.12681830302668462, 0.8483332431733817, 0.857361497193666, 0.0838709719695161, 0.4268746447013715, 0.45042297964854006, 0.026261461419732957, 0.14316395057938325, 0.5986522381215751, 0.5457229099523176, 0.5053428210815473, 0.8152000429618768, 0.6250797024021244, 0.5529927579205115, 0.10325831952848141, 0.213226219782507, 0.2863771120480121, 0.43643108144204457, 0.4085724035681998, 0.3974967585303667, 0.7914448482005235, 0.2943625331928895, 0.07899687729634475, 0.12602266880062107, 0.9627773558408685, 0.09197296253096177, 0.01784616281598006, 0.04509962185605876, 0.4306208518713036, 0.029211603863418922, 0.27087221506288994, 0.6920810299809184, 0.46103945943700453, 0.2742375222352209, 0.09871014046099391, 0.5002465322638636, 0.36774901910979285, 0.438295550626616
 07, 0.9584852682543622, 0.7192482909972147, 0.9390238102723785, 0.21966420181312774, 0.06270312877857565, 0.7945006455479464, 0.948676490254168, 0.10232651209368204, 0.7743641039353296, 0.28501148213919525, 0.6140563652268377, 0.8592719713356806, 0.34431446152254497, 0.8415362925895201, 0.8142073890056368, 0.8808460425093462, 0.4001662589407222, 0.11556893853971362, 0.9877717058606662, 0.025321322135442026, 0.15568860872060564, 0.724565879896658, 0.8929205086859203, 0.7718719306120216, 0.5253932005879912, 0.5599332459544969, 0.830538628280904, 0.9453912018915441, 0.20110738496851732, 0.7916892656651894, 0.5070854640930748, 0.17810009418804862, 0.27564822801414224, 0.8973471867420296, 0.44499933329377583, 0.5746513531131515, 0.18328141895598615, 0.9224465871443851, 0.7362080486490924, 0.7554430411251164, 0.5674691955123351, 0.9662939617131937, 0.9782539269747884, 0.5671119798227955, 0.18348180552561477, 0.5788561297640441, 0.4327363449207118, 0.26968597426273666, 0.09676707043900046,
  0.3533828927397278, 0.8022208726297413, 0.9366316278850002, 0.24754347623914796, 0.5042181765537848, 0.5344586187795173, 0.1370417104698921, 0.9179017592202274, 0.6303917998346977, 0.23170524583924057, 0.7084646449880674, 0.7597712470052572, 0.1829663612559268, 0.5681244484252116, 0.5743859159308287, 0.8204263704905678, 0.3777475034632012, 0.09970550061518646, 0.11212933669243885, 0.43678014579832614, 0.42484739660754367, 0.48882027500033487, 0.5349536865080836, 0.9605311775296558, 0.9827253552238541, 0.9594904717019531, 0.11637831121337494, 0.02330089706609384, 0.6867109917657119, 0.24963046094186347, 0.8552788187015238, 0.4542186203931423, 0.45064952333076413, 0.9329775222164091, 0.3570520324938192, 0.021981994492264345, 0.9951905248777341, 0.7461153482717665, 0.28915703104048773, 0.8892403789353457, 0.7173747348708638, 0.47494322638336917, 0.75859697737764, 0.039219827307805044, 0.5835564263313942, 0.030137470027180036, 0.3544143249656104, 0.962827968878825, 0.2013312683215327, 
 0.5725979063425582, 0.40002955800015005, 0.5790475691107264, 0.6060229206434072, 0.9375670866573094, 0.1682617971914897, 0.702248108195117, 0.5834139355562845, 0.15005506260639134, 0.4261271083485495, 0.3671146445653527, 0.5427897025415117, 0.5243984315557189, 0.6970181745631603, 0.9752139110710698, 0.0529967797682237, 0.6972271140021967, 0.05462719296673724, 0.536787624501907, 0.15909984339887928, 0.9558078152033788, 0.09598107792373001, 0.24718341749533523, 0.5083543701000781, 0.3926787325473472, 0.9802164100972685, 0.2594041189182621, 0.13159484047461645, 0.7426848454312565, 0.06522880009620358, 0.792576463585391, 0.44977111393670843, 0.2399688836171211, 0.7260900291912906, 0.6507105109540988, 0.7052485149009664, 0.4701681740481436, 0.3971330904067888, 0.816941378337909, 0.5536198327273493, 0.9870128941066615, 0.7438637468379411, 0.03551530250499835, 0.9941605442111137, 0.41271669206637995, 0.14136506612883837, 0.19710247466994113, 0.16569080622664623, 0.8479957409032417, 0.31261
 86104713998, 0.9025888027880039, 0.7894913427987832, 0.680247755134519, 0.06317464641717208, 0.796319976895931, 0.2020525384723244, 0.29736794942543043, 0.5378605503692692, 0.13184080386630703, 0.9878190984552762, 0.789873592377593, 0.32072541304509616, 0.5614932109469892, 0.19572554724609947, 0.455455242391302, 0.0011758002684753022, 0.32580722031375164, 0.003200521912941401, 0.8077668618558361, 0.5372658672085715, 0.8594384983117677, 0.4774504377862483, 0.8352131423077678, 0.861363177725784, 0.3274772401692655, 0.776216512421499, 0.45568724345, 0.6939182043988844, 0.7592731984035581, 0.7288312719258069, 0.4696578988111745, 0.45782491433105865, 0.2564917929985595, 0.24328379709731152, 0.8266995331952197, 0.5188662099076208, 0.21793675173708404, 0.39613591810790616, 0.6895049518018261, 0.716996761031195, 0.6457450982800385, 0.37951403764466485, 0.6046151522233104, 0.29872318089846217, 0.3758802952794803, 0.7030342667923438, 0.07054610033689768, 0.9125991555304048, 0.5104201818886519
 , 0.17840151434321794, 0.9316879143150584, 0.2878614701590234, 0.419955963129252, 0.25055560182412995, 0.320296744686591, 0.5804902142776897, 0.4314896992115369, 0.18162067973500728, 0.31909548093464934, 0.14148440100505622, 0.7205370937104177, 0.4425466902809406, 0.12344433426509793, 0.6389021944461434, 0.9857134518580773, 0.422214863268519, 0.24473775752973326, 0.04822242412172362, 0.19990068760362778, 0.2249114119836393, 0.21562953040225297, 0.014842434284084938, 0.10416354161746189, 0.8944572330686985, 0.33288330979197656, 0.48803425836368597, 0.1389033893555266, 0.87691339432925, 0.20509074728086052, 0.32744006706537254, 0.19130886900133237, 0.9136655121572498, 0.2601952995307626, 0.7911944467144493, 0.8789820021120222, 0.8428230026217683, 0.9117042133049328, 0.4290198522950259, 0.08668971885892618, 0.9792514286322237, 0.7826679816019892, 0.9241487772318742, 0.12651012879169954, 0.8421897268770978, 0.24294983624421196, 0.8546843076388009, 0.20121410314940202, 0.0042863449771815
 , 0.5280052093905581, 0.1780183966832669, 0.8165419709967803, 0.7470213227280779, 0.8763349175626627, 0.48118523936368174, 0.4101068045244295, 0.02486429187809447, 0.4479718934309572, 0.6855342827637888, 0.7429744993982571, 0.9641509336704079, 0.2678064388605681, 0.9458663390456756, 0.5345426194423784, 0.7525434866398159, 0.0752500320278876, 0.9421829547556677, 0.4121594067722143, 0.31107347031904575, 0.7556835411548172, 0.6264576090557468, 0.49399482642357284, 0.5831478670540906, 0.9793303255165569, 0.7573450959725314, 0.8269961095780269, 0.6235324962676264, 0.7796410316857281, 0.23250314301924258, 0.46133032601388435, 0.9252096090522305, 0.06597065774846655, 0.3112079305934419, 0.15853675477003537, 0.5831387993629328, 0.4842756805360259, 0.3092035527508744, 0.6026825451213526, 0.9613341068403553, 0.6147526874103149, 0.2432327525594682, 0.5121348181083518, 0.5033019799433511, 0.22678613429973826, 0.20445408357286432, 0.2341334554355602, 0.20576849560903654, 0.19926806372983707, 0.9
 479590526344056, 0.24516837732026564, 0.06175628700740465, 0.7582978142058575, 0.2998597914747062, 0.47723596995780626, 0.4714550539542143, 0.8844649205803417, 0.5039261719159012, 0.5433693558052619, 0.21796192210259135, 0.09836419962923626, 0.5565090698913151, 0.7454077618779978, 0.10985079706922074, 0.3456865733196951, 0.9114938906456905, 0.21278997669731126, 0.8036630940546842, 0.37217554401860087, 0.6762867430958188, 0.6432281498059154, 0.6367942625686542, 0.8039331990353842, 0.5653433201894608, 0.8944375665152543, 0.5248840815806197, 0.13000641672356938, 0.4295079320273787, 0.11066886262477282, 0.10770110547774359, 0.9218779551448312, 0.1374390227473058, 0.1586092983922256, 0.3636444900475885, 0.9184332593115897, 0.9496538275939085, 0.894290216216021, 0.855369407390396, 0.05516880131032942, 0.2962340023367712, 0.1859576341353304, 0.6176749668720326, 0.12285313378984486, 0.045991076458977265, 0.677433910204084, 0.4536762045708618, 0.5035470996381639, 0.9378985288623584, 0.992318
 1519640675, 0.06641485239380662, 0.3508350971922044, 0.8348554300650259, 0.4514436118915295, 0.020446281563239244, 0.6563144294846395, 0.8553540141303887, 0.40433073524554697, 0.0015725166291474801, 0.8758352383852362, 0.2403813544516047, 0.9599445578851037, 0.222774653127367, 0.17667877379496566, 0.020893624329418437, 0.2428855483046778, 0.11053016421151263, 0.910806908410904, 0.16545861669920825, 0.29821363115577626, 0.5980838928240354, 0.6195114462424972, 0.9081703039427208, 0.4214248714849197, 0.4508740590896084, 0.7312369492519667, 0.3836972102812899, 0.9996709726698721, 0.05419754480354977, 0.1270457149286025, 0.5293337391508313, 0.9161322289883558, 0.004331467003930056, 0.9171397280091905, 0.4531817916172246, 0.1915242798606288, 0.6617109186816091, 0.09704875268097934, 0.1130976732981539, 0.8150399338247446, 0.5483956025872616, 0.2335397590246432, 0.2027970124605426, 0.6515974098708037, 0.10194057111175703, 0.9679978920624334, 0.5995185138857569, 0.9030863567384375, 0.8339198
 120552312, 0.9189743856640564, 0.5919047788541734, 0.8483741343618643, 0.38373340885992, 0.5157925692587775, 0.37726659151029096, 0.4901540806957235, 0.30696227088178996, 0.40154778858026685, 0.9941548138364408, 0.29628865636980484, 0.6924172906849964, 0.30653310632122266, 0.9391714497954137, 0.43912041916484046, 0.8631623454693232, 0.7120673674876231, 0.20620997931264262, 0.8082937732560264, 0.48417360834559353, 0.19989770848133603, 0.381822206360688, 0.6009512437043784, 0.3729300546704558, 0.11379414324898418, 0.7321670860754439, 0.4777381695734523, 0.1891137016851352, 0.04764878876103573, 0.9250044607318865, 0.11386956183941044, 0.9657296535113885, 0.7183379069302285, 0.14445499756620583, 0.4715841783817336, 0.13634014690898222, 0.9817999891451169, 0.8767547072812814, 0.27363006641255916, 0.3117387675232468, 0.10722878497564081, 0.5719909128226707, 0.9669969807416556, 0.5642739906948241, 0.9369750179893579, 0.7198138706628845, 0.9319309260259793, 0.38916430119695733, 0.2286963420
 3986866, 0.6163762989289986, 0.8133366340766617, 0.8295295429446919, 0.301489844295057, 0.24526088457831396, 0.6299939409039189, 0.475496273296648, 0.08918739144091581, 0.24691345527112574, 0.7515842940123804, 0.1776176145553361, 0.8416888991364687, 0.9091076556667699, 0.9243444604517081, 0.3509808397999925, 0.0091538359355392, 0.475224321474468, 0.2732336702047826, 0.688758760861252, 0.2940673538632148, 0.8376369517529844, 0.6118223135007088, 0.9525939924766214, 0.4626417460344203, 0.8602524591728781, 0.4506790733172733, 0.2620875735892926, 0.26971001360542823, 0.1572638335403116, 0.8108548122389465, 0.9667967840910865, 0.7702131252183498, 0.14442964812252557, 0.3211313395265055, 0.7515292318018629, 0.36557841141596603, 0.5747026532705615, 0.1060233564884413, 0.09935363529672869, 0.16473114458969795, 0.5717882130741224, 0.6560928481561894, 0.6166299134611771, 0.6271773493826087, 0.38322017238251693, 0.21295460247141074, 0.010494954186543337, 0.9995407061023464, 0.6888005144665841, 
 0.1984227623564916, 0.7016218832108055, 0.757132062262647, 0.32075653111809543, 0.8705677450781013, 0.15370509457617276, 0.11857510945377192, 0.6787266672014818, 0.699023999000941, 0.006004641940152489, 0.7142072723245152, 0.9556951212969943, 0.9177088405255228, 0.006417189458924999, 0.6579201974947178, 0.5302517044945433, 0.3351829310585649, 0.5599236615332491, 0.5972993333006158, 0.526062506612915, 0.7612037407715797, 0.4308958335538825, 0.798367463045302, 0.5911938098570977, 0.3425931274474979, 0.7676275762774425, 0.9589328587958623, 0.4079400302680706, 0.6923606336291015, 0.1535891290352588, 0.23848694808061366, 0.30806434991598575, 0.9430946477178466, 0.8322851829700096, 0.9427929958031837, 0.6388677700113338, 0.7589535587785896, 0.6012311652216156, 0.40775155973567645, 0.20226132367425076, 0.9001596228753468, 0.12544379920111282, 0.8898244629990352, 0.1720226815624072, 0.14971819909798667, 0.5819762749497424, 0.6892438799619931, 0.38367735580190043, 0.2841559153650799, 0.61440
 96538482906, 0.30742889444976784, 0.786951761850725, 0.2983909822321603, 0.5498461607635556, 0.17147671330593872, 0.1525034546782864, 0.27941967148316993, 0.7108321498335443, 0.3462914966431947, 0.8326417675776641, 0.16432303768956524, 0.44077144970965476, 0.6988586657080509, 0.4900098341387439, 0.11516837297121307, 0.15377294884969595, 0.1793683136070442, 0.22870652169644956, 0.3967000965428408, 0.3185085389018937, 0.24033602114523056, 0.48908511650458586, 0.452285327721233, 0.7905409954007595, 0.9275655029094122, 0.40047309744917436, 0.3393994245353916, 0.3187133428006709, 0.8457420823550293, 0.025314925276451183, 0.8080070945872039, 0.3685751525693205, 0.026641656532581104, 0.0737483074360874, 0.08351546426657586, 0.6294529012268423, 0.709385309519263, 0.21938137133461622, 0.8505449535942865, 0.7016552613818594, 0.17986450236009244, 0.10504655638857241, 0.024585046289548718, 0.49643634729725317, 0.11040754980599654, 0.4915647017965902, 0.6456716557715012, 0.8237231546239848, 0.29
 095337044572955, 0.7730603733622646, 0.6315434797159377, 0.4912462166535341, 0.24944338617333606, 0.2591279522739305, 0.1873048544716257, 0.9935947871537972, 0.021838454330492252, 0.20715479378043256, 0.384443075065393, 0.5308516642263811, 0.2511121591043951, 0.5773665198791101, 0.7941183406453303, 0.6638446037834189, 0.8697681696198335, 0.8786670486051076, 0.0879648958641277, 0.3866845475552262, 0.8518735630169991, 0.27510320715173, 0.20421223806418176, 0.7771437761634884, 0.908296109473817, 0.3307926491096994, 0.01775535878571688, 0.6715597268386798, 0.542778587084333, 0.38541052759503946, 0.4498737205437823, 0.0033329901563965425, 0.5866784298358899, 0.572839048859098, 0.6066186800039455, 0.5427210101222274, 0.18664635203280833, 0.9223412178728273, 0.8444592059959983, 0.7805116610022261, 0.23221315660234665, 0.8146283364494517, 0.6275967502980727, 0.6092937138189531, 0.37919869460974076, 0.3044483524768764, 0.6634294030970315, 0.2734241907079459, 0.1500528510552076, 0.46683010336
 071584, 0.0809671013041744, 0.7824250956064195, 0.07368349087137072, 0.9758762219818818, 0.35156838146712344, 0.612425107710916, 0.4650213732519488, 0.5830055604945534, 0.24219099993834892, 0.056394427107598455, 0.2385179515856264, 0.025803199334669902, 0.3384180784015499, 0.02630399381990023, 0.0999664851686104, 0.16914466614887158, 0.5778420875162874, 0.4589730320764076, 0.5674831863739681, 0.8570181501753924, 0.79395934599347, 0.7322513753829706, 0.5351792940600397, 0.0364768696149963, 0.4865476229467349, 0.6593164672776828, 0.5120013816454225, 0.2044110528439863, 0.9065707505226107, 0.6962471616883303, 0.24341180844722143, 0.12333715119238386, 0.0719728284797323, 0.9716695368605864, 0.1550266132756617, 0.05415965974105652, 0.5152966526334458, 0.6271526957887353, 0.945023927570415, 0.461868521600416, 0.40488015151800383, 0.7431654904008391, 0.6960807210406231, 0.26439317117027683, 0.17845532569143385, 0.817787471764316, 0.07779115906979261, 0.21534210278863297, 0.9812944997659246
 , 0.3640008330182526, 0.5669656010336538, 0.9381141349332752, 0.010618991018260582, 0.6962722467161523, 0.12590396104920654, 0.4618762585987539, 0.3338055554192788, 0.3882668369746284, 0.4476827395786437, 0.8491599227837112, 0.8699963560312989, 0.2009192215257951, 0.702690532275999, 0.6503358153234549, 0.0030385460452050372, 0.5037189223881833, 0.702091754650066, 0.47198510877021926, 0.7282495265376498, 0.34195108572831046, 0.5315938796172586, 0.9041926692518983, 0.32632494293399095, 0.33876543605381804, 0.0028388353676152, 0.4160457205953154, 0.3047760505713317, 0.5633297867985563, 0.7199344127707322, 0.5991292012265959, 0.750603831272764, 0.3449568991478431, 0.16508654010255486, 0.31556694979026334, 0.297487599945532, 0.7679764690209776, 0.9699896478994348, 0.8901655196301158, 0.6265856389895746, 0.24325684302454686, 0.45857600022565925, 0.4754753290858976, 0.7921165460973875, 0.11966368811179307, 0.6219219200508768, 0.11409184006750639, 0.69120933369637, 0.7723758626767633, 0.917
 5529229786347, 0.8242767169102428, 0.6820815415164386, 0.9848614138190812, 0.6730854997054412, 0.24298747764836692, 0.6954242455881324, 0.7274496419893745, 0.4180887906761944, 0.9746613581632468, 0.18094347509292985, 0.6278176509981577, 0.16824437841327478, 0.8146072609455339, 0.8843279461016447, 0.5273427666224223, 0.5423280015201595, 0.442617249440756, 0.759482979732328, 0.5221554223557527, 0.6052319773031952, 0.015417074502313666, 0.28669121353128824, 0.7988590264986991, 0.4991928633228202, 0.3258329810005065, 0.08943074881352253, 0.3922670104156021, 0.2505839018619497, 0.8420595252297446, 0.02090158246056062, 0.71613976296217, 0.6210770736622231, 0.9921715742148991, 0.8258820559039151, 0.3477866580065787, 0.5096752128341602, 0.5170309254757595, 0.07435650075613709, 0.08353161078261007, 0.20605945620558663, 0.5789960621202394, 0.31042647353854413, 0.7244234518404261, 0.7458932844872302, 0.584351132000112, 0.26333749412561636, 0.40020452697113573, 0.7251688176915079, 0.35412863728
 101585, 0.527024222729352, 0.4944350296866734, 0.19337751892587085, 0.36100714306429826, 0.3303965482502891, 0.3948016726488497, 0.6642313600310363, 0.23922424123876607, 0.8926603650785785, 0.3600238580635353, 0.8578572931743235, 0.6699519854197541, 0.3862859073283945, 0.7684021665931493, 0.5198318770967285, 0.04634000998217225, 0.7148080679967145, 0.8929369519203771, 0.5501191916441033, 0.030439979668074213, 0.15346253629310447, 0.13407829905566182, 0.7189199545929209, 0.22848163597245053, 0.39773220705306034, 0.3335505522625487, 0.44663504226636996, 0.731479744413183, 0.7181987510145069, 0.3913280084095154, 0.48159249943388105, 0.19490818782054586, 0.7296644987064214, 0.9127209684702927, 0.8218816077410734, 0.4641859308216506, 0.7483297645609274, 0.023664201769330595, 0.8965836605608846, 0.49197644069961544, 0.8328815988073396, 0.8096826318759559, 0.8468836960511423, 0.4388114279906986, 0.3836698847968584, 0.05764201271428537, 0.7289936240870691, 0.4003058935108641, 0.608717099349
 0716, 0.10927490262733475, 0.5985114150339909, 0.16224075506860514, 0.25612838706431784, 0.6716054058942826, 0.7297874266574995, 0.7490460250751146, 0.7595390302220656, 0.3717786540496414, 0.6740633128068716, 0.3017262240648606, 0.34655632783832946, 0.6924341098908663, 0.4665950240441359, 0.9980266896966941, 0.7197238208008914, 0.6786440647752137, 0.05566619589357524, 0.31064804378020183, 0.6353554944666993, 0.46857244457510583, 0.9453438110172021, 0.888573182020276, 0.07765817165618116, 0.2673906354028741, 0.11902319548564966, 0.7420795539131033, 0.7538809282481337, 0.8576210019823539, 0.5422549561725184, 0.2137669385824883, 0.32079990297924466, 0.11896089720896441, 0.9310412536652506, 0.42593888643272293, 0.09366810420016791, 0.3167727096931955, 0.015734859515002975, 0.508933577342277, 0.9858293119683041, 0.11080904573879746, 0.23086914553738103, 0.9355083699347524, 0.08051609301223761, 0.3343741537841044, 0.6176927164982957, 0.7131388924145485, 0.5459219493018189, 0.5990056166232
 66, 0.7388776188805624, 0.019102772904492582, 0.5704578555905884, 0.6908939660086089, 0.5738156676840643, 0.10247463195095319, 0.6171742351891368, 0.5426394588010912, 0.6881122829759565, 0.9226677835599907, 0.574649032943262, 0.2875789362056447, 0.45233276359010355, 0.045915056371938934, 0.3162133690163248, 0.5337984130640945, 0.7227514019186893, 0.7790898594524628, 0.7702065382740432, 0.5310793789244156, 0.190798888614663, 0.3845254268911068, 0.8510221707664553, 0.34640995855655454, 0.5580988589929421, 0.4899988370462578, 0.5040279191528347, 0.512325167771079, 0.8004299257601769, 0.4848038909225739, 0.677011030938516, 0.8672226465501673, 0.04664436545817885, 0.7758443674003616, 0.24662316170979603, 0.4679333927718715, 0.017520611517636064, 0.26693795156527855, 0.8003671377576445, 0.5222102621285061, 0.5447215603076837, 0.32002762032776, 0.70298964395039, 0.26242889779580014, 0.4214759081493593, 0.7587566077753214, 0.06071322545607871, 0.5616793766403034, 0.7242956794660044, 0.18286
 933060524235, 0.037562156007833525, 0.8795592866308624, 0.7889875886005645, 0.9761597174987287, 0.4751063923971357, 0.8173618068723619, 0.22441182096608947, 0.9111955615547368, 0.920268523066743, 0.16098524497499078, 0.7038870028385277, 0.18080542450657033, 0.9754086591269061, 0.7947732323972809, 0.7868350300872958, 0.019322445833274582, 0.35072278602737317, 0.2806276551769755, 0.6922054153759251, 0.8158374321728183, 0.1210602547102273, 0.2141191368695171, 0.7704136109746659, 0.7672210030811933, 0.4675864830948213, 0.6929527309681688, 0.4019730988439846, 0.6704327935735236, 0.6192870257641446, 0.3630293768790328, 0.18869685146774773, 0.41164339112322357, 0.8001299203073687, 0.5434763137441017, 0.5972883573052052, 0.07395202349943852, 0.734773142809937, 0.8744026260924345, 0.6653383068890373, 0.3559919809447112, 0.6690242796621383, 0.7797687439013284, 0.9861879696450729, 0.1704721943100841, 0.9208304760051743, 0.7926418567298239, 0.9518159859825075, 0.874712172553366, 0.6933959536633
 37, 0.4512413966345671, 0.15185004046801875, 0.5744849329800966, 0.25409577762398095, 0.04156233707526458, 0.9020008306214633, 0.3379904388063366, 0.6111742604250688, 0.9359962435453864, 0.10666068178182186, 0.958539899309466, 0.5047867527743277, 0.6572530761010865, 0.04810439289193369, 0.11253889429261132, 0.869764039912697, 0.08301967102853802, 0.23867566935789464, 0.3736155657909541, 0.924548818822202, 0.39917326095915606, 0.17253722783070746, 0.5642473920628077, 0.6940740097832543, 0.508001730522673, 0.6700084150952675, 0.35299815477778684, 0.08392309111909302, 0.6027646705734001, 0.5202092705026745, 0.838484661363187, 0.5889325378710776, 0.5310622080537286, 0.29667540672629045, 0.100450143662974, 0.34146752216251275, 0.38686166324077476, 0.5458032165076802, 0.343829586023504, 0.4829216129216162, 0.4946645465966969, 0.029430448741288084, 0.3134496183964479, 0.7765617988412156, 0.5791794905069685, 0.34055233106374416, 0.11501381628056306, 0.9706607525759751, 0.4499991949485759, 0
 .005118893123088442, 0.44228161037988967, 0.667927872740771, 0.9731627661920299, 0.13305410063762713, 0.480857253271346, 0.3605718313748213, 0.45011724527485886, 0.7880833055273724, 0.43576285352762734, 0.8498968399198641, 0.7794748651839504, 0.5086503848597909, 0.7129773084049392, 0.48515588978137825, 0.9380960149787511, 0.831151271212023, 0.2743809400712841, 0.5284537843626547, 0.3854097157969434, 0.24190610109292776, 0.5810533054837724, 0.48081380851238886, 0.9685173829014049, 0.7157374047496122, 0.841629378951784, 0.38909171129950393, 0.785507259501351, 0.8867170919405243, 0.9904274865769396, 0.9942412460751007, 0.25054064726107106, 0.9405901411663348, 0.9343196769374251, 0.0785056556101783, 0.2966536767434689, 0.518578649271087, 0.1278235052283786, 0.5224298951625054, 0.25085052907494243, 0.27871472434101463, 0.46265170044246906, 0.39215863539421936, 0.4209478985313817, 0.2735460826519791, 0.7909639946625474, 0.8433056867728854, 0.8447622747361028, 0.9720040208393559, 0.1408236
 3935063147, 0.7497501410418217, 0.9696152864382608, 0.5784181426668197, 0.8278681765267607, 0.7440356475286127, 0.2537644442364545, 0.2831077480026203, 0.2189627041580532, 0.790618956549776, 0.8112499932486378, 0.21667521600161643, 0.8616109860741181, 0.5040343445340554, 0.25778109283887773, 0.17407199033506138, 0.05715794479536185, 0.17111115987801084, 0.8576673115821983, 0.6247028122009419, 0.7263248804918596, 0.8397013028725729, 0.9204345063841997, 0.13127979417497282, 0.592376329893984, 0.8959447638151657, 0.1506468894212598, 0.06283219192409872, 0.29022553060760126, 0.2679206504574202, 0.3155894455275885, 0.406224468824523, 0.9990433183941252, 0.765602209227177, 0.1628485743618926, 0.5557289918787093, 0.5914426446346717, 0.09535424867667186, 0.9934606029823332, 0.19164646607961922, 0.09406941243686529, 0.7717818368811814, 0.6761816537561409, 0.7466613013809039, 0.46243738495932274, 0.490559436887117, 0.1745641769115578, 0.5025654141809082, 0.2871035285570175, 0.5229398551925248
 , 0.8509091110042714, 0.06017844060031685, 0.06878720194794297, 0.3459628950663939, 0.9097949441708687, 0.7817383941939957, 0.12679486238370963, 0.031953728921505165, 0.34876788123134117, 0.8177314856571287, 0.03811986615200946, 0.8867214770068355, 0.6949101761267372, 0.4990756548603804, 0.7438441956240472, 0.4479385285193771, 0.6233865850459849, 0.20076714080665314, 0.10497751543832967, 0.23350177913061343, 0.2777013709716829, 0.26454524880795593, 0.7416260150630134, 0.5642472322054733, 0.7412817310662908, 0.20582571230606506, 0.44739210354153547, 0.3762091706497832, 0.6461353613278975, 0.22409754029341744, 0.3351627872357331, 0.18468651302439099, 0.3467139329658908, 0.9861751194748926, 0.14095368496457028, 0.2656979532932692, 0.43574583811560075, 0.7581410152113619, 0.7892322891186839, 0.6799789955145156, 0.6336851838792013, 0.2245114218723886, 0.4061524533375549, 0.9804935476877459, 0.24455666321325287, 0.7521007384231915, 0.45638985258603815, 0.3165033130711562, 0.43835836884048
 707, 0.43125773104818066, 0.5747760352159968, 0.17555842676852051, 0.5938180938423008, 0.958590375050545, 0.9656860429981097, 0.18934342002214088, 0.815180631790393, 0.312992410472407, 0.9511432171990515, 0.27013240273472605, 0.5228373880611316, 0.2207294262601739, 0.3628616222175107, 0.8416780499148914, 0.7099914127461673, 0.6055380223198037, 0.9939553177761559, 0.08167915928828762, 0.05618469823407268, 0.5761365887344077, 0.2528007262019222, 0.902541841088051, 0.5933966599105088, 0.016241932164534623, 0.49454655314609597, 0.8629698070343218, 0.03155590812462716, 0.5111306722872846, 0.24502362318134563, 0.3249893265004905, 0.16769041229487636, 0.16998041023983768, 0.6773144370534758, 0.9493350483086297, 0.02657328322587993, 0.8846703797012678, 0.2530002731413845, 0.056692899108831574, 0.5106050266641926, 0.703416849746635, 0.3981117352500039, 0.47311685596981246, 0.3728054484162334, 0.29156395856152606, 0.4480658385535843, 0.2250062403389559, 0.2691711922247152, 0.9020456949960223,
  0.8663811114831105, 0.8523513194768945, 0.4952879127306953, 0.8844802601306992, 0.9187197172516658, 0.16852421118393235, 0.7669690590160729, 0.5245533480702964, 0.5668135460350576, 0.010713540438243796, 0.06582518594929898, 0.9242921479859026, 0.3360888525139135, 0.8971706562209252, 0.13805188419956116, 0.7090271177962141, 0.3063120883981503, 0.5357560893916405, 0.7083559675632027, 0.5782839479230808, 0.17633710731738683, 0.8128452318382258, 0.850823082619438, 0.10554365107849373, 0.5620419878563053, 0.2324720624634775, 0.2446626092822375, 0.7357535763766626, 0.30080143547247584, 0.7726606717120681, 0.2351482571926684, 0.6296679029366747, 0.5563483984801392, 0.7792142124140405, 0.7187059963683234, 0.8605411953218199, 0.9446365993505675, 0.8268014592633378, 0.8759833264718593, 0.4813941076450857, 0.7343232312970526, 0.4430012150685939, 0.6878671607172365, 0.15001536618340838, 0.41474924886972764, 0.21303550639835134, 0.9867999747595417, 0.3240598445719399, 0.5698019254603792, 0.0612
 42374277864386, 0.5193769870399788, 0.5200477386056075, 0.04780286558200786, 0.4399097859120732, 0.12608981779002548, 0.7677645195562787, 0.8755718378389475, 0.7469121318814547, 0.680215019592432, 0.18550810342310786, 0.2290748708150202, 0.5406614200003986, 0.8702283230924698, 0.5518233628693611, 0.04357322401679753, 0.024205395245136763, 0.2587588872667397, 0.8558104806094124, 0.5229467684120374, 0.8298751797104521, 0.3059798706008122, 0.07078862967766908, 0.890874159947114, 0.35170525001123, 0.20646531724019335, 0.8150417511681354, 0.2158091202121627, 0.7449190393365587, 0.45592519378987717, 0.13856856016174457, 0.0372006231192793, 0.24009388436850232, 0.037864277346812414, 0.4062041370422149, 0.8221621428584405, 0.6041885990298087, 9.244534227337287E-5, 0.13517603107718612, 0.9898030750325235, 0.4214385085476344, 0.9771473291261004, 0.6891026431261518, 0.354693448403968, 0.3641919227713246, 0.9070730903725, 0.9306376868800198, 0.36144054776824075, 0.7770829989911384, 0.7896313815
 36878, 0.291408832511417, 0.40911402277768594, 0.7075060743637271, 0.8379039978727406, 0.23059508732200207, 0.8972639117305748, 0.9811768497380956, 0.7863792476656438, 0.7224319670937531, 0.3645338081177145, 0.043636235489237785, 0.43599671733992706, 0.2381288757049047, 0.236379066577022, 0.06687260005685791, 0.7402582465714277, 0.669129145442831, 0.25015800538565447, 0.2548091727766527, 0.8663955720188732, 0.776337407841793, 0.5838716802545911, 0.3722228961829859, 0.7017179024192951, 0.04388528379847578, 0.0790855750284295, 0.26873184106807757, 0.21764550356699663, 0.2662010321788608, 0.719074687549844, 0.5728085573612428, 0.9525682885182765, 0.803594415631729, 0.6999908658365739, 0.7903466588596609, 0.6295510388051089, 0.20426324505899918, 0.015859480164364226, 0.7581288711041426, 0.5860399715583313, 0.8095735058122479, 0.46746988701844416, 0.7746145783833402, 0.2601076635370766, 0.03466000843820671, 0.4007325138140354, 0.7587853314036797, 0.4514070068045716, 0.2478401281604411, 0
 .2818324293107428, 0.4723573298416469, 0.45488668353019435, 0.5600284167831159, 0.42451819730845375, 0.1814899479511909, 0.4514542152148372, 0.3792849565566798, 0.7991211254617367, 0.1311897759980709, 0.7321551780140371, 0.28582605261481075, 0.8734094836350498, 0.8589298243897145, 0.43106988776504385, 0.6392707387609504, 0.8621387974685903, 0.6835741962192838, 0.44701187992047053, 0.3425922546801097, 0.595167482489662, 0.6715185599590932, 0.1452452150036222, 0.3561887105658682, 0.4532655906871298, 0.004026996587467635, 0.24782041685923628, 0.8806824966544584, 0.46625102999039836, 0.9480664631308262, 0.044761743581658076, 0.6085036649510017, 0.7198465641926225, 0.5675099110482069, 0.6882212760430113, 0.08200707800586604, 0.6756579447984591, 0.902400198648146, 0.8678481056259928, 0.9898192522596165, 0.8729316621929836, 0.5910361529592892, 0.6080036216825542, 0.18330564529899318, 0.7512587797228897, 0.8129635510945189, 0.27608710663505287, 0.30149442059198384, 0.01014865391660591, 0.99
 05234106425176, 0.14611805239435627, 0.15548851460082902, 0.004404567064937415, 0.9024303659902982, 0.022854753040795428, 0.44234237127068654, 0.1634909593256464, 0.7349246156904565, 0.4991119561326317, 0.9335760226944207, 0.3185897467585901, 0.12436085235183358, 0.3925720443462196, 0.014943303650517548, 0.12280103003334919, 0.4296677592320277, 0.7882683490223884, 0.34176752596260496, 0.4579514551880063, 0.31393276219817723, 0.8990511035312476, 0.41810222727085455, 0.8357308675001134, 0.21143596058035774, 0.6487295799339329, 0.10817077545930454, 0.3213865649040908, 0.409778385070534, 0.48121696982532625, 0.44498297406207576, 0.4349609215815927, 0.28631936894059806, 0.4038872118586432, 0.613723354085566, 0.8180676754726663, 0.7039587910932086, 0.6844857558216231, 0.6564497556591257, 0.19345834800650397, 0.6933695736098979, 0.9383191415513996, 0.26983729926867295, 0.2901391402026914, 0.3271065661150001, 0.5660810965281973, 0.04914751029250364, 0.9585562180347209, 0.12558836411757213, 
 0.6325555441709875, 0.6692978594540305, 0.6583746493619407, 0.42576822659063907, 0.31256979349669245, 0.7878522278475931, 0.8656348004343625, 0.8903817020021569, 0.9729543840088963, 0.9049482603167969, 0.7165774173756946, 0.7389775651917974, 0.8078977695106498, 0.02162263912363449, 0.38218603964187414, 0.341510246484498, 0.6999683686104746, 0.7006910341221856, 0.25050436795598463, 0.5310912265659747, 0.6402079894826282, 0.05328424181642244, 0.9830425663106469, 0.36456780982362524, 0.6080840650271198, 0.24933773598735143, 0.9070954466133277, 0.08791778658453453, 0.28572120227190667, 0.3817772802240307, 0.0075526695859793325, 0.7480187791407514, 0.36071452651224367, 0.9018378910464242, 0.989789283135142, 0.23317144823367697, 0.4579610324120881, 0.3868996884525452, 0.499482172431155, 0.5218260071580214, 0.9333271542280708, 0.10241867589664055, 0.6057067283635494, 0.572281813754138, 0.8606044925272168, 0.6759762192403386, 0.08042469149650333, 0.5957001564832858, 0.548087270558081, 0.214
 16642419629572, 0.8404831227783082, 0.4205906748200481, 0.8322392442600514, 0.361985237564446, 0.15750131971376713, 0.27900193283229757, 0.21329928505434714, 0.12958143744072415, 0.8865308286681607, 0.622967481528415, 0.6213088498544588, 0.7926741403772513, 0.016131339767511244, 0.5143283929852835, 0.23092641390231616, 0.810636989621356, 0.30051317396898336, 0.7218743351321173, 0.2795273496661549, 0.5001949188899485, 0.598654716053255, 0.9068146362094198, 0.28547030494106407, 0.15287401424463654, 0.8158110936828658, 0.18091212803497003, 0.41204408736878884, 0.39079087690505177, 0.019588144637416205, 0.8811819829877734, 0.9774916110779583, 0.8180628318024356, 0.38576366911056637, 0.620878638728845, 0.2726405875161728, 0.4440659694603538, 0.11829543118048935, 0.5485640019403066, 0.9150532590165298, 0.9299628771795103, 0.3834638284597338, 0.2753703318512791, 0.9422402507654671, 0.4413678423418316, 0.5998538645798396, 0.11514176444898716, 0.7263668824417828, 0.7777118968483573, 0.133890
 52082004904, 0.12295867849413278, 0.5790121722332175, 0.49281596967267394, 0.56489743824992, 0.9146191886837004, 0.6069245912648383, 0.9213756721945658, 0.2041585451112885, 0.4029726053806997, 0.9783507498093079, 0.38250590308070564, 0.4792185793457123, 0.3965701431136499, 0.2643964194112697, 0.3441898654828778, 0.12766726403908046, 0.47191794019589584, 0.9310117247990545, 0.0014212457182173344, 0.8839092452977815, 0.6526883723666194, 0.37457551099631603, 0.546024739333262, 0.4879295858300293, 0.20756555854447611, 0.11768647818069411, 0.387584237976608, 0.39682302561300187, 0.31297140143072066, 0.9116516183393316, 0.7226044877782395, 0.5378622575164639, 0.4177389880631971, 0.5282952770025584, 0.14297642660881826, 0.8061390263529559, 0.8938698908429338, 0.9660946810880138, 0.42501211603528266, 0.7803189344988724, 0.9608331975136162, 0.6442813016577672, 0.24676062447692348, 0.30680934660742154, 0.9304334423968714, 0.9134293653785709, 0.34401116748126936, 0.1856471074091841, 0.53509546
 79424774, 0.8721862454182808, 0.534335356966387, 0.18067084241856002, 0.47448442487162024, 0.04718210133437517, 0.11371573415171488, 0.9720502392442489, 0.3436996043497963, 0.9667888624807269, 0.36175300973492186, 0.4710028757145096, 0.7908506181484176, 0.03928157978564828, 0.27913969907447556, 0.12453636515603006, 0.09264525666952428, 0.43208791900751553, 0.22503319421242485, 0.4523120789841286, 0.3336885208884963, 0.8411943135679503, 0.822156788321677, 0.0782342023940572, 0.028512647378007516, 0.1384104540290817, 0.35390215063916153, 0.9461253340626886, 0.33492790407201933, 0.889076052918665, 0.8904023736389809, 0.24185847727206988, 0.9731050035263428, 0.6439269611529589, 0.5494898019162149, 0.7835723757398738, 0.5868694481208564, 0.7262207869891241, 0.6236484768760734, 0.28710512177362857, 0.562009117171716, 0.9588543308902996, 0.785515628232662, 0.965539166405824, 0.2509663066754457, 0.4060883668607215, 0.22218322255129297, 0.3797580541192036, 0.14410580301260434, 0.740628291205
 4164, 0.34249435734530176, 0.3211734151936715, 0.7748211292746872, 0.28919165506821376, 0.6517258693993154, 0.7782886801972562, 0.049061146589503335, 0.15832613394489337, 0.4944931545654627, 0.7155232313721192, 0.24942951266898206, 0.09712530245158568, 0.6294087044877167, 0.47572921178363525, 0.17983012559154632, 0.9146212486850073, 0.017971180668839515, 0.10709044190870276, 0.7887014651694283, 0.3370164836449676, 0.7853571915216523, 0.8106011396174785, 0.1372036242905469, 0.16355063163361583, 0.0422330006172128, 0.00607964491366908, 0.7632917718069001, 0.45967938655018203, 0.14439936634983053, 0.2226803074352287, 0.3334300883667066, 0.31237679426770637, 0.011026942276001761, 0.13412447673105032, 0.9707708521546027, 0.3647546141178508, 0.3224361171601786, 0.5793794854467017, 0.801943235028288, 0.9458247699561849, 0.7748197911589512, 0.33627191851770677, 0.07416708054421584, 0.11611458597848312, 0.08200300710225039, 0.7485299266077162, 0.660226909227303, 0.3609763063507696, 0.0832789
 2783448809, 0.04600636594607754, 0.8909119742647029, 0.9808164186214441, 0.1415555822985809, 0.3433431245481895, 0.4328484040159717, 0.45110175553005727, 0.9974934057837563, 0.8540943698277979, 0.36421953984886535, 0.1548833791148011, 0.38300889003726235, 0.8153340106410506, 0.6916915370116741, 0.9267112951942074, 0.9914647708710587, 0.9820315403259233, 0.2989308030571074, 0.789196130293629, 0.8520626509190438, 0.32098671221332464, 0.7204206594541195, 0.11210665599698744, 0.7623993809870385, 0.7108726939162763, 0.19385504363149075, 0.17877513808125411, 0.19584339834709896, 0.037944370155062024, 0.16020677806374994, 0.805714548728368, 0.01933319838868175, 0.06622647499532985, 0.7526388572870917, 0.22873557062972427, 0.43239469385663776, 0.856504085773403, 0.8437472865600634, 0.9424831228966031, 0.7696202396017667, 0.3101959261759193, 0.905194171132967, 0.5601955756562486, 0.19654675402424548, 0.6221983237840961, 0.27461031568552297, 0.046622538664277235, 0.8728162291045484, 0.0410460
 6780484199, 0.42171677457399903, 0.8694195264322148, 0.8566968135209638, 0.3552438040348581, 0.45893505703270543, 0.7129712444575744, 0.8814883510856601, 0.9513795956651847, 0.45132629072024855, 0.40347751768229734, 0.14703116516822756, 0.12263969314063394, 0.04651933029067512, 0.6052688708398962, 0.43848257843979577, 0.5456024503014384, 0.5673590683986991, 0.665701964242788, 0.2097205964414397, 0.4886955400292763, 0.9883254735367569, 0.709616982791797, 0.7501798532737649, 0.3674599273459799, 0.15134504645389169, 0.7142230233209155, 0.1916200217192695, 0.7197709826445398, 0.1674361316634957, 0.8314093385958428, 0.9052721436010674, 0.5384250003285972, 0.19770013748756055, 0.6668638049629947, 0.7900039328659085, 0.169007930575232, 0.7104909108082692, 0.5740124429829447, 0.058589941070223195, 0.6577197563156355, 0.8711649368546068, 0.27567666689007975, 0.39186853449959336, 0.5168368112731606, 0.9366351535003763, 0.2874778862765812, 0.7495120381404897, 0.3647516084098589, 0.399021564835
 84206, 0.6844018637827637, 0.7013322982933404, 0.7930377857943935, 0.15101234628544924, 0.7255677414791978, 0.8744893507158779, 0.11474516310586469, 0.1301192615403256, 0.7321299887348316, 0.9996655716932014, 0.3561386334851824, 0.7398311239693697, 0.39937402674946265, 0.08321355759587368, 0.9683511606629845, 0.8427406244141119, 0.4164489479661886, 0.43248114676333993, 0.2559301891953232, 0.2740381959770234, 0.8658139000954356, 0.617962091785779, 0.4460137613712818, 0.6333392047657812, 0.37935739080250563, 0.9686165512031083, 0.6548806822630164, 0.5786286134534869, 0.43928139256665255, 0.8656506499058487, 0.7478772903987734, 0.9681800477117638, 0.5292252838579422, 0.92523955746349, 0.23423028553709535, 0.9746243226233736, 0.06473717540683166, 0.1421244343671727, 0.8900139748658082, 0.7299143166349262, 0.16208315654761807, 0.20370797748189284, 0.760547156625458, 0.15075262420160696, 0.3405176517379408, 0.4782194808179955, 0.5577534356574417, 0.34111911717285226, 0.1955212209562518, 0
 .6307152315120795, 0.2385259741617689, 0.3718554024149614, 0.9291444137519146, 0.42712518867087357, 0.7268957879947036, 0.29683411631470247, 0.37847964732048045, 0.04925278091995833, 0.3258106461600362, 0.9810606763724152, 0.8444155720476006, 

<TRUNCATED>

[08/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/data/compression/incompressible_object.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/data/compression/incompressible_object.adm b/asterixdb/asterix-app/data/compression/incompressible_object.adm
new file mode 100644
index 0000000..e03e6c0
--- /dev/null
+++ b/asterixdb/asterix-app/data/compression/incompressible_object.adm
@@ -0,0 +1,2 @@
+{"id":0, "array_doubles":[0.8353859890622932, 0.22500827673705914, 0.22157022791391168, 0.32035245758794395, 0.34939474725656017, 0.5554287293317939, 0.06613512953798861, 0.9681347508140643, 0.6623415178580739, 0.5919439929535106, 0.05212040031639453, 0.3892855164316662, 0.8844497593574515, 0.6074932969350857, 0.7711957613447981, 0.8631583218045213, 0.1770457827094719, 0.4525687775376275, 0.8036709263952709, 0.5019105296004952, 0.40277490848861697, 0.3190257275538467, 0.7110984656234003, 0.47492940777795445, 0.6588714514705232, 0.8422160078737491, 0.6587109939654401, 0.7413350665701102, 0.8347485080603378, 0.01498504222097563, 0.6551497676477125, 0.8641987866071527, 0.9416184262349903, 0.5638318471854301, 0.4565123230471907, 0.23499879702437088, 0.7245505917442545, 0.02053900526300123, 0.8975670677530877, 0.6979679511672306, 0.9863184718241791, 0.7118370974009355, 0.7850529576017674, 0.13137044593192237, 0.12758276331945362, 0.28517618646209486, 0.5163029978250586, 0.016377992191920
 93, 0.6394033369931067, 0.19043485821420558, 0.27100002466638606, 0.40409840155892873, 0.6717675614750906, 0.745404515677119, 0.964694454966989, 0.9774422135729012, 0.3457680486842587, 0.027145395602065636, 0.14978284412603105, 0.09689613474086856, 0.1772583959126579, 0.851817571134352, 0.7029676527431258, 0.8179046088632121, 0.836260348127726, 0.5323490307372913, 0.4477877164800176, 0.8724437851767138, 0.47909147244416905, 0.9363096377075704, 0.826630340403748, 0.5185049307619617, 0.0591278224570877, 0.6056187878344298, 0.3587053620587216, 0.9785870656178437, 0.3473107559787938, 0.34522114147657645, 0.898782433631359, 0.9119317000304087, 0.8749535739799605, 0.7396238130336652, 0.6460626445222623, 0.995114802462112, 0.6490973667424694, 0.5141953106193369, 0.4433764232814532, 0.2217446083229403, 0.28461222664174113, 0.5223847738684045, 0.15396944033783244, 0.9743962859771625, 0.14743461508521505, 0.4105277061797251, 0.8064587833050655, 0.2705472366999413, 0.20977679144649508, 0.68866
 25866980012, 0.7581122349133979, 0.839159905125601, 0.580840032967774, 0.018789554722417168, 0.342423334544698, 0.8456916113612959, 0.0955084362426809, 0.021261763607677175, 0.3390254486454911, 0.15069384533534935, 0.9186953318108272, 0.795506176239649, 0.1323084784948293, 0.9607958089401623, 0.8544597784952086, 0.6530294504855796, 0.6096719468989508, 0.6107507786732062, 0.9367059800745291, 0.2686721024230544, 0.691168728727381, 0.0013555592010559003, 0.92222336647191, 0.5703316191423874, 0.8793637744292353, 0.9104370043899721, 0.5070212780975464, 0.20788364967407158, 0.3715525360297284, 0.9927316460777484, 0.44335624313740585, 0.3687874777328216, 0.0860545310122598, 0.7893680959550368, 0.24754639259535172, 0.04087043020742853, 0.4201169616155035, 0.47879385372934047, 0.8576121464237345, 0.5620434058822834, 0.6790345554522537, 0.4183716243074923, 0.424977533820756, 0.4422071989205446, 0.8944112317086809, 0.7495028636692472, 0.4059125754468552, 0.9943679332633895, 0.18406028257852403
 , 0.4690186185609001, 0.25609149191727487, 0.2699868860840089, 0.896859601201321, 0.40571157235176947, 0.32558764303904175, 0.15784836022047277, 0.5329487541371086, 0.4832648479788375, 0.0018208421281796783, 0.2762365786044213, 0.8329754628279173, 0.49946678078557394, 0.7544183688493702, 0.46497106817803957, 0.4896688642550392, 0.4064833256880278, 0.4986460577435724, 0.2668571583537266, 0.6508375244374913, 0.7380925988988857, 0.6130114153387053, 0.7773661705853161, 0.7549604023395863, 0.397168991286312, 0.02183574359078644, 0.6368683230634229, 0.26885938730935377, 0.4903368813041652, 0.9850376750522303, 0.7414979635370078, 0.5815120643082735, 0.2765971441756202, 0.6250662364010331, 0.45673994414693575, 0.6488840558988345, 0.5828619403295261, 0.7648657934523091, 0.7320825197263998, 0.9817300836078994, 0.2317167114370684, 0.35095198602327304, 0.5355910622654452, 0.221634123363029, 0.12536063689782695, 0.5921219388371786, 0.4540135793971187, 0.8601280196190044, 0.6345596166045365, 0.95
 70176434432645, 0.13703794438703332, 0.6215494538255102, 0.8525793260886928, 0.3427072240032266, 0.5035778211793611, 0.6523698093894362, 0.09144750451720662, 0.051643351535156756, 0.273472138075992, 0.9380429939713437, 0.7215287654149819, 0.6958038006037093, 0.4397412103692946, 0.3029965532382284, 0.08527703645335194, 0.784514589186673, 0.7545500585208672, 0.940206646690063, 0.7521641124716618, 0.5464009638064378, 0.1662871871624697, 0.6638689241066048, 0.8241652118915335, 0.48610740162357824, 0.4156753346107207, 0.9819977723027192, 0.3498157234856416, 0.28287253407244817, 0.6755881763331116, 0.645355917066487, 0.15973561072652742, 0.9007407177517643, 0.20225483116999843, 0.5449381532331123, 0.5976513057298175, 0.19913686110876494, 0.4547917694659974, 0.7023015382906737, 0.6883270942425279, 0.9077267587060027, 0.03551005794922535, 0.6253562782319632, 0.2574514408036823, 0.5609303964423067, 0.1052025918462095, 0.48680063418280517, 0.39840611944126914, 0.8751507839382795, 0.6356250389
 441056, 0.42320435733103856, 0.46697899449085467, 0.09572652984533137, 0.9756412274652144, 0.7498148862765809, 0.47746389955380575, 0.04081154411321142, 0.7280115852304494, 0.03948127381327704, 0.37222428863484525, 0.9258304497949561, 0.860561549524682, 0.8236606121865943, 0.14167966219114514, 0.5648536647917183, 0.30847382719846694, 0.13929286202418534, 0.1828068860195442, 0.21418171574656708, 0.06370743501572396, 0.2673991970545776, 0.6016313199188558, 0.14862302469664457, 0.8930526588012075, 0.7875804714413934, 0.38881533470452445, 0.04239548866817067, 0.3780905013276109, 0.5071616691135529, 0.5273346929521358, 0.3642767474222771, 0.9283317783373297, 0.9145493473886707, 0.12736622199054393, 0.2905738915486633, 0.7830696124956467, 0.5534386982990959, 0.6258926509105323, 0.08967856490764692, 0.7879863136591094, 0.11782791547116012, 0.9371637280031505, 0.538418409074168, 0.9093139644056136, 0.1273670323413092, 0.19503533568053544, 0.6931114174047233, 0.03135539595090464, 0.758389373
 4582627, 0.3893408428052567, 0.2571751334615425, 0.5724305040150481, 0.7780438951358732, 0.7658327056653681, 0.8623377874910653, 0.8498199456404653, 0.557266580833382, 0.3328635934887805, 0.8119435197740464, 0.9260372758419227, 0.822900339476027, 0.19786083914167718, 0.22253987872720027, 0.8767667887191496, 0.15009177639683136, 0.7427795420137674, 0.04388027144375972, 0.03506383076060504, 0.19316507149326823, 0.8401029613309663, 0.13767003965074642, 0.7869206129025924, 0.3064921811963376, 0.4095918182887037, 0.45182947381115246, 0.9200598060679294, 0.5164497612403659, 0.6208665658502489, 0.339743282937226, 0.3608246360387287, 0.7713729352150592, 0.9580396884901152, 0.24291021642679111, 0.3732931918388648, 0.018487125796476822, 0.17999801983285701, 0.2806029933235842, 0.9402516526569102, 0.9351182984316809, 0.9335674223040599, 0.7715930975970625, 0.09741934038054767, 0.0760533730226648, 0.406675762613563, 0.3506909067145595, 0.45032636831792083, 0.789742661540051, 0.9447620791080007,
  0.1367966060497391, 0.9675020177373276, 0.5031055547401347, 0.996063226498628, 0.35935274464056044, 0.41414047159179135, 0.8780226024167288, 0.10004199912740752, 0.9565347806518504, 0.4671677254189486, 0.90216590174722, 0.8881030887038365, 0.9212108323563561, 0.635641628994703, 0.19264100463268452, 0.22687698466402306, 0.5896476124422604, 0.5996747328655427, 0.6629836968980145, 0.7017336477050021, 0.643134779275433, 0.4765776217438785, 0.23237464546087383, 0.611821398636598, 0.3241448672050964, 0.8246776986442755, 0.4405064531848849, 0.5278090150403909, 0.2563416468868692, 0.9032769512227845, 0.6723566049520555, 0.24964160511528533, 0.30169518772720305, 0.8214694326500231, 0.19419128832388055, 0.6940041169752231, 0.08815660089319388, 0.7290471245649632, 0.3554445086498037, 0.8398187779248226, 0.19779141473608708, 0.12948105728453518, 0.12068786790450436, 0.36498569787880086, 0.5179656142771164, 0.7279717758854606, 0.5617260523746306, 0.018708613756401427, 0.9271233374610145, 0.9668
 215163894344, 0.06192427798944056, 0.09113081891332186, 0.31189125528849304, 0.5458880655361705, 0.7631929930115235, 0.5656387895988371, 0.8936892672991418, 0.777638054985954, 0.22062693845538173, 0.6690739470273791, 0.9807228662890946, 0.5843255054634809, 0.32061468865236553, 0.5712259766004212, 0.5551777125845686, 0.30907871545431376, 0.2653058062018421, 0.927363822379401, 0.5556323537110611, 0.2831873475707364, 0.8296429010187638, 0.8061930341052319, 0.32115018286374475, 0.26863803362138083, 0.15673619558593543, 0.30051156426248304, 0.42620702172647407, 0.47795607804467377, 0.07317845922901622, 0.7789681438212199, 0.05455658693963339, 0.8626674348749306, 0.27796847266058755, 0.3047110322925114, 0.13533027500316142, 0.7572038639757042, 0.7870316840992571, 0.220346843139974, 0.6961144095502548, 0.7668312828051563, 0.6998967494084836, 0.2785128656781246, 0.27544203292609903, 0.9961896516933901, 0.6959072390977359, 0.9834983375858704, 0.781082870173694, 0.3292901624481307, 0.66441793
 14067685, 0.0017333886977698443, 0.8807095092961701, 0.4923237556964084, 0.5267452400870454, 0.597597237679028, 0.8679634052873876, 0.057392209070833755, 0.6172124015769943, 0.05587952953091713, 0.9596401179292597, 0.37828148113615734, 0.9564492378191886, 0.9633621523438356, 0.14865758071622392, 0.6886542806392905, 0.3721775220919771, 0.039371260943652064, 0.06266737423428237, 0.23854695400942505, 0.7010329872746882, 0.9287821425864429, 0.759613842452502, 0.3968136731507218, 0.23954512533534977, 8.425237111984174E-4, 0.029241217916994078, 0.10516820283037687, 0.20321668486585276, 0.733312460122401, 0.5634943392421479, 0.9337320652125436, 0.19971448145927762, 0.703802074222364, 0.30895399196663575, 0.258505120422696, 0.6255042523192753, 0.5588208652504856, 0.4208928155383046, 0.7579326667206011, 0.4548316973692854, 0.5859354630176191, 0.5012333913135502, 0.5087511931399413, 0.10577286234774252, 0.6755654792673146, 0.22163532220675486, 0.45055196744682746, 0.9401470868357267, 0.526329
 3100343238, 0.008133415149142786, 0.8840105590997338, 0.48240581554005024, 0.9214858456984, 0.19519437850584187, 0.9694872168509081, 0.627181503813748, 0.7838992248718403, 0.9447548076596834, 0.9007113199927351, 0.9460002690527081, 0.5030880885850223, 0.26974839978057585, 0.5425210532778811, 0.7007331712206972, 0.7203026049052722, 0.701826088220545, 0.8421366904037744, 0.3246807849709996, 0.35620564093689133, 0.4994172702627482, 0.25020442130683584, 0.941755870592959, 0.17061756270302275, 0.8049766036483089, 0.7208549166566659, 0.9600501318681526, 0.5549165582498513, 0.829205450733648, 0.5992788750376741, 0.8981400621240695, 0.5750691634228464, 0.9652325321939834, 0.8508295416694933, 0.38080859800273126, 0.5068990764631086, 0.7171838953219764, 0.6374199648746411, 0.6754282698926716, 0.23508432049913364, 0.46526738363347153, 0.20499089567783968, 0.23107301871500596, 0.923506228089565, 0.38344198092903414, 0.41008238091049687, 0.4294954002619078, 0.3069564781420412, 0.2759382157111784
 , 0.9840505308737686, 0.16042651569209077, 0.8028408853210908, 0.7053406117950739, 0.810228647327581, 0.893226095668618, 0.39640158542339843, 0.5476386121039944, 0.0723392935535232, 0.6327309031712448, 0.9141127516474774, 0.02859803797501681, 0.12172221228299929, 0.5192195459147763, 0.8672862296996823, 0.3709998582283819, 0.3064506847665206, 0.860732065847133, 0.521558216305488, 0.4137391851589741, 0.8071663477145705, 0.07055356860689055, 0.22780941179524872, 0.9514715463223165, 0.7324708363579607, 0.3234903902123122, 0.3362044068322294, 0.08744028047493746, 0.3409702176248248, 0.7416390141551527, 0.20741935968733538, 0.5923127406510057, 0.27757579227272944, 0.5736669340763476, 0.933758838327004, 0.11829951955501028, 0.1384230747942764, 0.14119574818451508, 0.3877463192236421, 0.1305251950384586, 0.5117780702134279, 0.29169703601321595, 0.12690567432089994, 0.5865654030403541, 0.8815582052515809, 0.7193813785281924, 0.5008326664599363, 0.7202594481000216, 0.4256030966626583, 0.79388
 20718341421, 0.03029016922162886, 0.7673537038730573, 0.7925874537919323, 0.4711041510191172, 0.757941704831586, 0.10822566835756797, 0.3994773637025294, 0.4514312013965569, 0.8064721209683582, 0.0968846711983603, 0.6556879871242103, 0.9975511737992669, 0.8222263054819534, 0.7956397691386494, 0.7906090273013825, 0.7274839693085154, 0.331129911314626, 0.9412712161586065, 0.6379222545981054, 0.2579631931967177, 0.28853876141213053, 0.7312367622988465, 0.813142534730378, 0.7891449993422347, 0.26579644666182345, 0.4373899014691902, 0.210072557222571, 0.5936872075502084, 0.7836627679057899, 0.42988195262501094, 0.637368380759557, 0.9015623539212465, 0.2764077200445668, 0.9253654577061289, 0.7181713562901854, 0.856538209086299, 0.05024930967556673, 0.6613117407929626, 0.8837693791303389, 0.061080433416377033, 0.38594597029211475, 0.7422004917333395, 0.406508680164707, 0.7235011539643911, 0.6277370915132627, 0.10353498691462149, 0.937684511701168, 0.5744989195598216, 0.6994581366154118, 0.
 39608141567304955, 0.9428978276016312, 0.5298045921557242, 0.9935394493524786, 0.6054263374696867, 0.760371764115012, 0.5414688984828472, 0.9897310854550142, 0.39089180736200635, 0.7469806144003263, 0.202397169030328, 0.27777822206377556, 0.9530424900085874, 0.12225106253948892, 0.447057466520716, 0.8819048666330238, 0.22046474478697753, 0.6723533990104551, 0.12585813974539717, 0.9148465879461967, 0.2841931635002648, 0.43410872777890674, 0.32151194683978157, 0.6253051701704809, 0.30210485605562676, 0.8432951711441653, 0.02277314556197385, 0.8450816070055694, 0.5457974432377064, 0.6020445836607321, 0.1742552360971883, 0.690776263165084, 0.8689119775735473, 0.6968789613684981, 0.18060675826185468, 0.12982782478290955, 0.6385572011195664, 0.5819429646324217, 0.659143163318965, 0.6817957196517188, 0.09043706512598926, 0.5312478482789149, 0.6071027363178011, 0.09317332702385717, 0.14347850608995216, 0.5592488414275167, 0.17801546216333208, 0.2976520514773111, 0.9412020046659028, 0.110836
 5753592333, 0.9462433425267771, 0.3536393997867915, 0.7747726137960035, 0.9298305258037727, 0.8795713952172032, 0.33696666487119364, 0.5866144666957338, 0.12932049371312782, 0.23679906164006703, 0.3591050427195225, 0.9536673980912791, 0.5077099273430641, 0.2606013111177369, 0.9798412879954054, 0.3058088773498887, 0.5810725521998422, 0.8454999261389642, 0.2882475422562708, 0.1476608208951573, 0.6354417413594516, 0.3266053445102727, 0.8369120184846316, 0.22094507474839742, 0.3419099692028832, 0.15685946831491748, 0.43178052382818655, 0.8697475489054266, 0.12292295068708214, 0.518830815305988, 0.5035915758444576, 0.3524956715746288, 0.5099117769509192, 0.8516830804975344, 0.4167704588711213, 0.4821497846830488, 0.19041347650192564, 0.8647590529628723, 0.6077806171450012, 0.7700534316772653, 0.3070882244191421, 0.8562914365108764, 0.8873132617907411, 0.1990840837079454, 0.8415498303356004, 0.05907334381438756, 0.58932960402105, 0.85818991261739, 0.5926835352041221, 0.2944905848380911, 0
 .5048517217006946, 0.9330946065265994, 0.3421324906765759, 0.4321390326432136, 0.6995323848895715, 0.9668966000045076, 0.2888934459846655, 0.024319955170687302, 0.3814155009514749, 0.8794850022734163, 0.7304684167327546, 0.013027111445998774, 0.958932823933532, 0.9661399122022204, 0.2697781502437776, 0.5916092853254659, 0.3090646762336845, 0.9833792758290912, 0.5350501901605212, 0.14411245324638622, 0.061725558806502656, 0.4740611149828664, 0.5141526912624306, 0.6823760454815403, 0.39881818919489465, 0.6839435890287636, 0.1253526683430377, 0.3542780909574599, 0.4650324197726664, 0.7783525619647461, 0.5732353173015282, 0.40609429966523525, 0.9923483388160008, 0.5802997728663019, 0.3715286027699719, 0.44775658739217183, 0.15133537616986836, 0.7203415015769173, 0.65383561201477, 0.9918664994789098, 0.9597226303322013, 0.9136620764244053, 0.53878441157524, 0.01563999328044341, 0.8023476248667066, 0.06131242680093274, 0.8678633674942218, 0.7295293511671701, 0.7464743588136589, 0.01022833
 9258798402, 0.5922174778285663, 0.8270894617432436, 0.372603857964662, 0.10059981193152567, 0.44893942439542966, 0.23772493064155176, 0.7832829665194936, 0.07935662867846538, 0.3378960061752455, 0.8072757404108536, 0.3010510487447813, 0.5649818177248475, 0.9868899457327824, 0.390968507932103, 0.12154544038145498, 0.011559057641423398, 0.07596286157397814, 0.55908194491175, 0.8489209346709438, 0.03890656330710485, 0.9809051625274465, 0.4877440396201117, 0.408120228315177, 0.07084347453069428, 0.6915445493870325, 0.6355998870733782, 0.3021655438998059, 0.45508235832548094, 0.2621439312387074, 0.4916589620837608, 0.5625246799450461, 0.6259347032770686, 0.35924667861621684, 0.020450862372348944, 0.02673687705422878, 0.4149627561067136, 0.08656531252439459, 0.4276891174510852, 0.7698673786818722, 0.3149063454682902, 0.2870170749894978, 0.5229626542630686, 0.6149921428643469, 0.35865785855811594, 0.9487685038106446, 0.2655453184321437, 0.6527965570039118, 0.28775466644906045, 0.1751577590
 3187636, 0.0452335034601431, 0.6115057562906321, 0.8538033370734399, 0.4176223724695821, 0.8721928384101966, 0.735745865022884, 0.4214452268742024, 0.31626720169451883, 0.9281148221553078, 0.5565692369627887, 0.5229180711871482, 0.544465571907885, 0.8655548784783862, 0.5647588555488808, 0.6519644001175747, 0.5118939609170441, 0.8331433192878384, 0.33566043261567124, 0.37878178728310585, 0.6446590312571969, 0.45143133505461286, 0.09617316516317143, 0.8179061464264324, 0.23168297013593342, 0.8674542348928053, 0.07316312339967601, 0.5873640982901596, 0.5032714060885815, 0.5756643593453586, 0.21744597057963821, 0.6601259234535015, 0.6907957236749968, 0.9087068637361913, 0.05861855840666508, 0.9278359466910031, 0.3056793627983697, 0.9735814138929638, 0.5134017370567479, 0.8719925287395595, 0.7713858162953737, 0.3370532988514311, 0.44378971951855506, 0.4691948741285401, 0.8438635001544307, 0.9487477719625231, 0.6411835177658276, 0.3447695973165983, 0.9639106422567512, 0.6311820743752927, 
 0.6988200054806613, 0.23303523169485174, 0.7991813410659612, 0.9630590532441978, 0.9271892604618073, 0.0026574978502948987, 0.7540610166410795, 0.5354677649575315, 0.06576745168189291, 0.8619948063410434, 0.15239869725472632, 0.3702055702637572, 0.0492740301849095, 0.8720272536343183, 0.13108953174104665, 0.7552576623486889, 0.49779542603525084, 0.5634088657343707, 0.02176054931820226, 0.8745392801428223, 0.9174998949120897, 0.2793775969336495, 0.14331919519776037, 0.387326981321241, 0.8223016242458814, 0.8477029256375637, 0.47988264800500524, 0.5473750490463997, 0.11671404761058268, 0.32254640300983106, 0.6766135128671812, 0.5425338570894994, 0.11626608879329858, 0.8310556519186834, 0.15024531446918588, 0.4827570910144018, 0.5242371276791956, 0.009707738985193726, 0.8815891384166643, 0.8677051020957297, 0.532099829045869, 0.12186298519959138, 0.6340703696522895, 0.42472201284326305, 0.965942441889573, 0.7106858707365237, 0.9253525581049581, 0.9237843643075019, 0.06227508720913899, 
 0.5605347437891749, 0.06836284192311637, 0.9480897662986004, 0.02400759332765734, 0.8151173661176715, 0.9935418805929509, 0.6272736921466411, 0.9810186859650643, 0.33215503877856745, 0.3658981670572451, 0.02481363385122448, 0.35961985174692246, 0.7419926004486619, 0.7184551182821308, 0.21199963410472988, 0.9513495397566267, 0.22153776439508766, 0.9379367755554112, 0.05792046726106259, 0.16391620013889585, 0.6951079143295154, 0.45904602257746563, 0.41823558597068977, 0.035741758124318124, 0.03020817711717194, 0.9504467143275817, 0.71507267027975, 0.6004396069993865, 0.8089596775622757, 0.10856909053966868, 0.8662111784492639, 0.09869429638910276, 0.42484225133011555, 0.41461598167497726, 0.1437828840261296, 0.3523926155581253, 0.5796114421842303, 0.3556543657393023, 0.8210400381163203, 0.33948887266098493, 0.7366944897487726, 0.6624355429718513, 0.4003127992501907, 0.6379696429880417, 0.7250656097591708, 0.5745940754607776, 0.8060974520972758, 0.9434167860928265, 0.48154613715735106,
  0.00813450481359912, 0.1742873446123292, 0.3070489691031767, 0.8696705947640021, 0.7706216153099937, 0.771878904336109, 0.08975655652421644, 0.6197043341314188, 0.6462485565940382, 0.14105532501617435, 0.8825259342213243, 0.6970949762727773, 0.2559647073068393, 0.7773707083674612, 0.8047830725891008, 0.8255838389172941, 0.08710288160695356, 0.5238280645956681, 0.6595686162730452, 0.4663997613364953, 0.9824535330889405, 0.39096436756096586, 0.3980897884523671, 0.2864027854565113, 0.6804910266143543, 0.6976270017347691, 0.2260768586326335, 0.03574012560086759, 0.7064654373398804, 0.26794526935211305, 0.47050522994379584, 0.7995231148033123, 0.028330611601574684, 0.917071678961763, 0.017626462645062957, 0.7882276129993611, 0.4119872340362746, 0.6115877244084031, 0.7318849255714782, 0.1879674682978245, 0.3370776294319848, 0.5000392762622246, 0.770179155554173, 0.10448437613056138, 0.6675764796589261, 0.4748869199921938, 0.11706001573253433, 0.3036436193271288, 0.01067994144996065, 0.04
 922700399274249, 0.9838403042683139, 0.27642056581703245, 0.8430028130176718, 0.10995947809402495, 0.9382239759781952, 0.13780749442939833, 0.0348519050093965, 0.5436486156981643, 0.7609448739100015, 0.006766116178589354, 0.36135095680314266, 0.8221436867867288, 0.062038580991212955, 0.8082964789715039, 0.9534508358339543, 0.3820961260464698, 0.6317535047282545, 0.8888879815630976, 0.4123660856342757, 0.035830670691015865, 0.34974010554840806, 0.4430636930768148, 0.4877368612067249, 0.7105982102508717, 0.23103964590441872, 0.5114245708566156, 0.8028233866828006, 0.5290577999708409, 0.8485276714372791, 0.2295451043969463, 0.09929730647974921, 0.6159065266554419, 0.9388489418273173, 0.08226810023986586, 0.869702674105732, 0.04389549199053089, 0.4725142672129077, 0.14939014105243997, 0.04741686542727119, 0.017040835356833095, 0.8487123383597535, 0.7164264534506586, 0.631949384178187, 0.7296588466130116, 0.16152202938911842, 0.16808742546696942, 0.7416748384942012, 0.3369248027389101, 0
 .676594067419303, 0.7822956356977048, 0.6066033860522694, 0.1259750979077604, 0.7560032581738608, 0.8990168661147616, 0.4405507267165638, 0.286143029516729, 0.47385568682277857, 0.9508741346139585, 0.9650765848570484, 0.6427383157972408, 0.07636785482252284, 0.01145910950763862, 0.5968192343974121, 0.3408512741230666, 0.2575174752783338, 0.8424504010547953, 0.9553607595159557, 0.743982660785413, 0.9449989442917842, 0.9020938800140137, 0.40051618438181835, 0.7468704462868441, 0.9210984861344618, 0.41529484508555503, 0.7659468429646279, 0.29270521551514217, 0.13923773534617068, 0.3995182917926301, 0.3202565256742943, 0.13199614147096705, 0.03931646291230584, 0.49088726939111027, 0.2572368412763558, 0.7636698229579694, 0.5136187335994031, 0.52131641456422, 0.6936854827568332, 0.0585545964927322, 0.4299288233729901, 0.06021135151882828, 0.23622969623571766, 0.28122314124758263, 0.7796368141281211, 0.5532114084718693, 0.667844361627054, 0.09823665072980914, 0.5859959971027751, 0.26564728
 3452101, 0.20026231021929963, 0.6891541586959358, 0.6813272616894925, 0.9698313979945565, 0.040232218376834106, 0.26124064538900915, 0.3418485394345818, 0.20299365093458854, 0.7255870602758207, 0.10962838645393536, 0.5549400244099694, 0.966620348010587, 0.8013444061420096, 0.8617557036798204, 0.3860388373021143, 0.9381501453300912, 0.9265668340000516, 0.6982280692131909, 0.5109894305377316, 0.6224252386352619, 0.4294695415196552, 0.1518166149425626, 0.4845591502200861, 0.3829920637783235, 0.6612691599620051, 0.5521294898497155, 0.7269607556871112, 0.962290749775438, 0.2385260925480398, 0.30556156440746707, 0.5276069733501375, 0.9249679296611849, 0.4092405025386886, 0.9784488797694282, 0.5889546144960564, 0.43999274457473814, 0.32971955565106315, 0.5859238198758235, 0.763178847447759, 0.954951584020573, 0.9972417798198367, 0.5797878841964734, 0.6705507205871191, 0.6098416956368423, 0.947801788720719, 0.7757889373744672, 0.8307742955538014, 0.6878799085789794, 0.5987268077342649, 0.86
 40101463182734, 0.7871721005117369, 0.007287454959327255, 0.29271381791667683, 0.20564841081023133, 0.45849070575208606, 0.9207073894428829, 0.6828445726060688, 0.3352799222754269, 0.6142310298968094, 0.5078565040149067, 0.08739284344092269, 0.959781215916931, 0.9367905492840505, 0.6222088535035335, 0.8387466106042177, 0.2733942983609484, 0.6073379041332094, 0.6839530476596343, 0.45252538026473854, 0.092994419390662, 0.7322223568389723, 0.499161653670369, 0.6509809401871495, 0.6584400192745117, 0.21285451299090585, 0.9290549765462137, 0.8890272516153802, 0.93995404273566, 0.8543764027264512, 0.9334383890824955, 0.051753550334432075, 0.9478063870374817, 0.6070726260265361, 0.36883621818417034, 0.4190319018131582, 0.8491377465972763, 0.8393566464095615, 0.9345416226979897, 0.8258388683153597, 0.6625851723591712, 0.4522994103221657, 0.8287465981825777, 0.1861683348621449, 0.9234400110142518, 0.4630499481605812, 0.0021116749558744985, 0.19392048545802965, 0.9903892439292712, 0.408130803
 4879114, 0.3775804561456083, 0.06590986971373358, 0.5994617126429763, 0.5474738177450669, 0.042938888855299284, 0.6501431399856727, 0.8616206562612089, 0.38619416230149106, 0.21656527931763647, 0.0867748025394336, 0.14751478183649136, 0.4734843014920461, 0.7370114078938734, 0.479657496154106, 0.20621984940732285, 0.7866413322691169, 0.9335015164917753, 0.7131605022082537, 0.4243736245640881, 0.26312466315907035, 0.9971300186718774, 0.631938800713467, 0.347926961940929, 0.20467354043718122, 0.3914851705546999, 0.2905919920521576, 0.4608042826269031, 0.34489663531325176, 0.41662092824094676, 0.8756607845647423, 0.7430867838662943, 0.34537915988160517, 0.7717438180915818, 0.20416637275158644, 0.7159026669780729, 0.365235172308511, 0.8496782291696278, 0.4109058384801888, 0.5210280602789527, 0.1351659344688133, 0.5401178555896596, 0.5509743256890491, 0.08027324738478447, 0.659468176106353, 0.5188666467329203, 0.36449309152847975, 0.5330491865274379, 0.2680683621379175, 0.3567180900508504
 , 0.3772797461289715, 0.9600791085576885, 0.24554226758143083, 0.2943165670673641, 0.8442051603033515, 0.8670407896320325, 0.6752120860683933, 0.1982905869328343, 0.7812391702548969, 0.023995076359978107, 0.45379579888773103, 0.039141843841237534, 0.477695264651811, 0.03477685673199504, 0.6557162527531516, 0.21282055633069086, 0.43194181445423285, 0.44509468266056806, 0.5560608277271538, 0.16561221091807177, 0.6118557238698392, 0.2889056578255087, 0.6573148444256848, 0.07590604875748841, 0.3055517791229265, 0.7197027245977416, 0.007374526390003133, 0.7966113871589151, 0.5984290447739097, 0.7766714902744589, 0.9449789357884073, 0.8684330551713896, 0.6893334365459391, 0.39524419426828106, 0.16358206678412401, 0.17691191853058852, 0.8416430701016762, 0.4873987531086046, 0.6458202342903571, 0.7450830880814792, 0.7543415476753232, 0.2891211036404505, 0.5881115660809152, 0.22879819384265543, 0.5336511825833102, 0.20076934134835112, 0.9085580252647334, 0.913204547807233, 0.2475569559984176
 5, 0.03064846499358498, 0.3112029628315963, 0.39656894005066745, 0.25355898098277796, 0.8929249205347496, 0.16582994822966257, 0.691578973075578, 0.10424955978015626, 0.008080512098399328, 0.6548714305337017, 0.1744503577769272, 0.8558186991220208, 0.30742503443302904, 0.7376286288051186, 0.6686138749112315, 0.03134938852026603, 0.052758474594836846, 0.48688714634711605, 0.9624510316229753, 0.3237723893065555, 0.6480923602291274, 0.26414480861622724, 0.22949267930956507, 0.36260735824947765, 0.4424114154646629, 0.017052327413427104, 0.14469011120117092, 0.514125506243909, 0.600776652781394, 0.7062985878539497, 0.9718137831214863, 0.7363267475723625, 0.05564079585511217, 0.4990568681901737, 0.5347297857008627, 0.9681316521030818, 0.1410681483545455, 0.902239728337176, 0.7792494945087407, 0.3964468573222113, 0.7596206124269231, 0.45119387123337296, 0.5824435404659399, 0.21527024546632645, 0.8349920960579198, 0.9180988329129032, 0.5727561396928129, 0.1677597392733734, 0.363044136561515
 9, 0.2592163377846801, 0.9869511513440351, 0.24519792697549836, 0.5897153736796229, 0.6812697455414986, 0.008866934370273194, 0.40604173244795694, 0.7163483033885227, 0.27461602002392305, 0.610151897833586, 0.07375529306091055, 0.0612966549729379, 0.6345053231798493, 0.644598372526525, 0.47388061050592, 0.7912473536631359, 0.7716062303345849, 0.835387241361674, 0.578809912115984, 0.17651124154878461, 0.39910736339365593, 0.3385018278783518, 0.5429798775897776, 0.8216067541027384, 0.37608915495756134, 0.7657639952334278, 0.031717147450397776, 0.4757818401660816, 0.6854918627856109, 0.9073913218428905, 0.7737213066855988, 0.8376885894303493, 0.8298895195348316, 0.9066694635539098, 1.7979353909314266E-4, 0.2342465699371561, 0.9394112469874897, 0.3082787583826596, 0.17276710140932394, 0.1758000403302168, 0.20353553389797807, 0.2270098215324331, 0.5482879679434537, 0.5679954727894702, 0.4621143971256064, 0.8650481384178149, 0.6794827827306607, 0.7953755966707081, 0.9208684644142947, 0.74
 37027152813265, 0.909520406983043, 0.9825065360293272, 0.9927260017489535, 0.30939955496636085, 0.2819622038953261, 0.4855250150806, 0.20876202201573968, 0.023955614289415617, 0.058488406368547685, 0.0703130144388906, 0.9911264391090877, 0.6555946083633589, 0.21343577270615532, 0.29307743926591934, 0.835232966377537, 0.24986103624326994, 0.8572037262376607, 0.31493259136559737, 0.10961327919682584, 0.17853428858902176, 0.529843291874423, 0.7541536996040207, 0.24922392810716687, 0.7398399315073252, 0.7016516794528104, 0.20809023329052612, 0.5916258711951341, 0.014700613879448543, 0.6193525934594455, 0.7612974413700475, 0.48534002077390426, 0.07146436928766953, 0.8444505512391178, 0.1372656871167356, 0.5858141579073395, 0.15549331851904524, 0.44460516204097433, 0.14819673305401426, 0.4468201775844194, 0.2001910735609257, 0.8193788744952913, 0.7505969158705862, 0.6137328922474019, 0.8524928119331646, 0.5684701983139289, 0.3240272288067503, 0.43002462807240704, 0.21853660061573055, 0.64
 35514331843579, 0.7561707573331896, 0.6794427326275037, 0.7041806498259426, 0.26408811658430487, 0.5394481266910107, 0.0037197848838858016, 0.6502258776515963, 0.723025555389845, 0.3627320342397725, 0.7592991911640184, 0.9436487469324477, 0.2599847713936114, 0.6162821979416707, 0.6064417101569732, 0.42384491663439705, 0.6376188590169559, 0.818842378212358, 0.6602109636911812, 0.6968286391853447, 0.9210379953465436, 0.222911300274662, 0.4752194539295225, 0.9205294750073286, 0.23071954233777192, 0.7575200482176146, 0.48464853354713344, 0.23077203231577037, 0.9160771547733739, 0.027418195963151315, 0.7689394713263257, 0.7402203209943944, 0.6798707772407991, 0.30339181945076255, 0.19068480908966434, 0.46151375842410836, 0.6728974319616392, 0.3105064527990701, 0.9996325227179813, 0.08101670373191039, 0.473910051514732, 0.14938854381235112, 0.9853483963336798, 0.24104878732229118, 0.17146984606241333, 0.8702293306148812, 0.13496397303048757, 0.8761594235725599, 0.4724452191924271, 0.06001
 518220731639, 0.5395714976357163, 0.19295379578513028, 0.876128326524969, 0.3005589471959509, 0.9852091551679771, 0.08317427578304792, 0.4734479190204465, 0.5082270066906805, 0.2630129707623088, 0.11618553010816646, 0.857515082133117, 0.975827185051961, 0.7379947956840452, 0.7697075833548483, 0.964746246740031, 0.402942869546848, 0.1270382784972065, 0.22658618943806175, 0.4050656167499309, 0.3230691932934707, 0.7540877131786689, 0.8905847188089656, 0.9851972741896399, 0.11422699145770332, 0.7741374368189528, 0.755628013118959, 0.6808143144691777, 0.21430122175354593, 0.9875122271404226, 0.9006231295485292, 0.2713212189585309, 0.2531703670121386, 0.9020621032079555, 0.6759276049946354, 0.21530234016093597, 0.5607636043476636, 0.6432774355569268, 0.5977310572127816, 0.1151782524822863, 0.9135033216686046, 0.16096336759998175, 0.5730615893140106, 0.2963686961533354, 0.44116039784380956, 0.9359954786382682, 0.8408229514962217, 0.6092691617138946, 0.3319701023564746, 0.051810903257462404
 , 0.9332148985896338, 0.5516058308606087, 0.34592690281791005, 0.06267235675825822, 0.037083238273039165, 0.20424120243857902, 0.17449687936562308, 0.564375509462118, 0.36619262495504, 0.27526107968364855, 0.36955610357993796, 0.7529522038914365, 0.5431093009295711, 0.7551805158281779, 0.9051847197128426, 0.5430557256520365, 0.5333271600536946, 0.30036522223454964, 0.4330156636277682, 0.010096517595456267, 0.37446534219549743, 0.48227193221320463, 0.9451582316316665, 0.018829781496909948, 0.7998112793339308, 0.4668691456386841, 0.8682862675449605, 0.9059793734766287, 0.8681778438154732, 0.5216080477584527, 0.5783015670340904, 0.9956145832641013, 0.8809891118494385, 0.9865702928465646, 0.9470530427570949, 0.5433802667492977, 0.1927044329129497, 0.991928211755762, 0.8109128615703729, 0.42931242727454655, 0.00213550987867972, 0.25560972924684944, 0.185580785007611, 0.5488552288472085, 0.8499069543442634, 0.7530690643506496, 0.6423083446499888, 0.3878271333505794, 0.3412557520122048, 0.
 9153213798528742, 0.5078048178253245, 0.7094680628632191, 0.5635603865313963, 0.2819077318029989, 0.45340744491907625, 0.6303305498250054, 0.7744362653846008, 0.21221447151973916, 0.5986681874928312, 0.6166625916446775, 0.19892834791741942, 0.3415788795142556, 0.4703301161068726, 0.09406084734203202, 0.1932781442378526, 0.9578706654091135, 0.02973673321532644, 0.43055729293423584, 0.6079371370068273, 0.36974709499977465, 0.510037492669639, 0.02693382869086136, 0.9139440685364623, 0.6395367725977532, 0.9891234567053707, 0.12794346125258393, 0.5444787774632961, 0.7617154806637989, 0.46504746522302154, 0.754248351527588, 0.9473506307891058, 0.4821947392125697, 0.6977529038686814, 0.8472684720785508, 0.044375033578810785, 0.4741521595237136, 0.23425608402567089, 0.31776745630465775, 0.1500366375442681, 0.5128898461423109, 0.8439973329837698, 0.19388478883415483, 0.9145331432581796, 0.6252154917846707, 0.05689608993242501, 0.4754560354932085, 0.5252391478454784, 0.9280983105197906, 0.618
 7156875343087, 0.6188466831108234, 0.2258246452370125, 0.01685960855367885, 0.4566640845769061, 0.800156509757678, 0.19370803118042235, 0.314916075940337, 0.1827265594911811, 0.23453540550191143, 0.9542188150869839, 0.7551013511668968, 0.25304728902454743, 0.8654838910603857, 0.8037908295612946, 0.5268491962994317, 0.648746630589674, 0.6803942037830326, 0.16470152793291026, 0.39402978283497037, 0.34688135050183755, 0.7572211991665715, 0.4003187137212495, 0.2649878162820498, 0.9098255911342523, 0.9020544995000385, 0.38251117419417324, 0.2793397857632455, 0.23717019859686206, 0.39849663387933965, 0.021824967887027036, 0.665344637674408, 0.8963088674141859, 0.9571344334416452, 0.04622257293488985, 0.017170724187181796, 0.4761994924933155, 0.4270238047511207, 0.5456272475980248, 0.44313909221967085, 0.09335952234091072, 0.21099100910015645, 0.18721039737664824, 0.24403151511644627, 0.5199747425894515, 0.6126860704238087, 0.5503049928778307, 0.39480875838883456, 0.401070489027046, 0.5357
 070669490231, 0.7300232881262128, 0.12359349700994315, 0.6810154617996298, 0.05374655011450802, 0.2151028612488779, 0.3085288330064032, 0.6691982262464675, 0.3755944751028818, 0.058199580076147006, 0.24613213270145629, 0.14053631813121548, 0.6399646582333721, 0.5257919577770064, 0.5840213664654325, 0.4885494397830932, 0.7924945233196521, 0.7400566008417222, 0.8563988250394505, 0.2270469493082975, 0.5209008513103376, 0.3065195552732397, 0.11243519913142552, 0.07105169783746401, 0.8541570772736905, 0.5589839227996771, 0.5857043807496093, 0.41840135888091723, 0.5676904009643334, 0.8471791041913032, 0.2170779276900454, 0.3005394727400673, 0.9789795620159646, 0.2427727128567092, 0.11399485709126933, 0.7743283981290775, 0.06656780013538155, 0.14583129240341752, 0.7057812935235448, 0.28628675751544885, 0.10038693666947485, 0.5365114502545568, 0.43770158758665223, 0.7564743202554538, 0.1324228919846514, 0.03570776812837906, 0.3356808971877838, 0.0013174840722576242, 0.7641342582671122, 0.51
 35803130017437, 0.6691462331527637, 0.6429888739231724, 0.5806698890986298, 0.7163285171624585, 0.241191981252025, 0.83752500332198, 0.5218126317017259, 0.26927462989871453, 0.54328231963873, 0.9946860045889617, 0.24571075238207762, 0.45826442573074433, 0.12596208871582815, 0.15963198066137474, 0.2338401511461723, 0.3915890356447622, 0.05755442847341141, 0.8798106977432715, 0.9842739855172264, 0.5010759083507524, 0.2695820337914293, 0.19002181065917456, 0.14521896518924693, 0.8536936126125644, 0.9206563300627073, 0.8144428462431478, 0.8519102026151796, 0.4139591841799922, 0.6498365808617941, 0.806472882030324, 0.8473396165598518, 0.22263306818286355, 0.735969308597778, 0.6255696683988787, 0.1909255919921925, 0.4386313122073109, 0.7312755903998958, 0.08482687869470529, 0.48911407925794503, 0.32878040258846264, 0.09546184125373014, 0.6938577899681978, 0.6180348497636368, 0.5839852483847385, 0.720887660958758, 0.6686225448611702, 0.8507186067117215, 0.7905761385118111, 0.00983200709985
 9208, 0.2831563304545439, 0.1500142666040677, 0.4920460723125596, 0.34165664603820267, 0.2977258391635075, 0.14322771692202407, 0.03908450033012845, 0.3506738557562217, 0.9137621441160703, 0.9556368915434851, 0.34475990734465467, 0.42478306748726324, 0.07798804735019726, 0.6474205250709809, 0.7368977111098887, 0.23044890995195477, 0.9864785635324779, 0.47217041889530753, 0.33114645075883886, 0.871351065018523, 0.32396306912722417, 0.5550407123029666, 0.7359196834859021, 0.49604402456953545, 0.11449881848811139, 0.8747194316135046, 0.4443466304086605, 0.7842225682172124, 0.05671472594948179, 0.6257913486054628, 0.9685603395820448, 0.7616970520018781, 0.8701454407654886, 0.13669319106679134, 0.6280964460624178, 0.820939665897453, 0.48874599826403686, 0.5233458212450126, 0.916871977669904, 0.657071379944271, 0.9034968499594415, 0.16968003283262278, 0.5421584000871479, 0.06841661331932147, 0.07808085273627163, 0.9436561388399095, 0.5541584513358087, 0.2162606200846967, 0.639932378751979
 5, 0.781009819269215, 0.4662417023432167, 0.9902025787789902, 0.9835303518953332, 0.8988398103707498, 0.9035891376460626, 0.5835047447825351, 0.013627029615597697, 0.4372856351282006, 0.1119423782757819, 0.24709645832691107, 0.0031738333897805004, 0.769926651166877, 0.6083482142116879, 0.4613988831439694, 0.9440272884403665, 0.07216600560278918, 0.36223481524604373, 0.4418949569838464, 0.32808694814740436, 0.9917466630157824, 0.15785105027914093, 0.97737605123235, 0.5123388921717555, 0.42667866661185094, 0.208045704725205, 0.08905533226493734, 0.6933473222783696, 0.48324024357594264, 0.6645586862337268, 0.8942451246793872, 0.04505836024226495, 0.6603476659036925, 0.40144180363534443, 0.36005378231477747, 0.7583494755534634, 0.18647213301047916, 0.33592161804630827, 0.3405650825662663, 0.5615789708169804, 0.221414340492205, 0.5949626885269622, 0.4052210160247144, 0.7984598283518222, 0.327630077350638, 0.03598436023893159, 0.4866370767097157, 0.2929028351739339, 0.21443397136089404, 0
 .9866298439210152, 0.30283606939173113, 0.04004367577908163, 0.2632896537593792, 0.5811817878331308, 0.8418953775738632, 0.22849017592247745, 0.7535292495433399, 0.7401631914499751, 0.9362595851151668, 0.22089450636490715, 0.025226512875554996, 0.6648587383071143, 0.09947844546910845, 0.8982770376974609, 0.039581497698829016, 0.6999363340705036, 0.3498881486818495, 0.2080978346134914, 0.927057177935162, 0.5550438665750105, 0.7037615135349709, 0.8468926328899674, 0.14964132518408835, 0.17020210627001342, 0.3847103838458391, 0.7553601110804897, 0.845989737806533, 0.9600093159707671, 0.5666317449036274, 0.5071077820203952, 0.7768397426962494, 0.7781190116357156, 0.6442464678085809, 0.7318342906541163, 0.04296564069117892, 0.7061471013923318, 0.6082874622998106, 0.4295454948524494, 0.4090808556944302, 0.6901104292810663, 0.09165228309572504, 0.4044733007167034, 0.6344737989672772, 0.12269376013812705, 0.048342356011583654, 0.07568495068815362, 0.6847978406778955, 0.7010875206754985, 0.8
 287692629393884, 0.6059424161697415, 0.07931841309319221, 0.2778343975594465, 0.6230163398510212, 0.6311903039220567, 0.9140528392404446, 0.9660672042822939, 0.8579012887231424, 0.8701222225829589, 0.2936216132893158, 0.46071728818001434, 0.1541938492806023, 0.5465953125244242, 0.0032625708365356942, 0.39371622640802606, 0.8770998974641798, 0.8672328093720062, 0.25734212574993576, 0.5539104797812996, 0.14613131837956872, 0.7623752974879757, 0.27183058975143803, 0.9713530011422346, 0.4097391504393658, 0.7206112095414356, 0.22559148678651786, 0.19083146424727981, 0.33559459257315283, 0.8254375724689882, 0.8989675476266668, 0.5409380615262479, 0.8191611005546571, 0.03717255653418394, 0.9080287061803012, 0.8426811913038926, 0.3712078871390583, 0.3540613077649438, 0.988980647803312, 0.7955844893560092, 0.009173485236078971, 0.7335868331632163, 0.22961480628804565, 0.22941397323460733, 0.0477955028995134, 0.5583187799728287, 0.49331370498858196, 0.5161198437606569, 0.8611806643981302, 0.9
 90202048244993, 0.5360086224665721, 0.6992739681114885, 0.7714972460116083, 0.773008531886003, 0.47761881024365516, 0.3172303077612023, 0.708729050288195, 0.9147452316226755, 0.35688482197335347, 0.4714299907888603, 0.18143921774965133, 0.21013195438949606, 0.4721410669522278, 0.4227188337213782, 0.4002483840389579, 0.747832716498083, 0.09045553435938014, 0.8862397191785535, 0.10374097334632004, 0.3879448654435095, 0.8936975244277198, 0.23940819315951423, 0.7507701131779835, 0.6923204962632896, 0.22247694539232865, 0.05767442665565403, 0.2636403060044099, 0.9436147559993756, 0.7775316630598172, 0.7492203467550638, 0.4334040237607787, 0.6985263259165503, 0.943021956953613, 0.5724358222037965, 0.4315170351255665, 0.7865153747953021, 0.3073244906030902, 0.988963345492444, 0.6113543739259194, 0.787354616023763, 0.27430013828747957, 0.13029574916931153, 0.9285969431090945, 0.7991897316580807, 0.881643789772876, 0.1373568733020082, 0.8612568837042369, 0.7433553199235815, 0.072806391420688
 9, 0.270190510497785, 0.6681170087992314, 0.5430129155866124, 0.7051124947947846, 0.9788280370215833, 0.1886178502323853, 0.2597483613262841, 0.5190795039768806, 0.5766008022903351, 0.15773364072752527, 0.6755856060777475, 0.1425775359073349, 0.7636082234657493, 0.18613634832201542, 0.8674444230249019, 0.9583515875210731, 0.010748375352656825, 0.1389544249293505, 0.03689017315919607, 0.32456052434485483, 0.7336625325687461, 0.6356912400202913, 0.23009368112161688, 0.20537026756238752, 0.2611410994781317, 0.7893753959550078, 0.6112398348080706, 0.7422713858321534, 0.8339830757747612, 0.10477375194730099, 0.6371936109617361, 0.6766740645569039, 0.25799715525728195, 0.6635045321189587, 0.8220528491762066, 0.8204661437547424, 0.37877713455142104, 0.3381473365955151, 0.7772871139021265, 0.9636029735997707, 0.24608016971589985, 0.7979529993198972, 0.8664824561699808, 0.7064303400157302, 0.6920817289705653, 0.49551558382742533, 0.12328223266744631, 0.7093249131164723, 0.060475893914259116,
  0.656775254257146, 0.046934150925000306, 0.7633690488740027, 0.9923355986827025, 0.5359798615036149, 0.5119246522669959, 0.9031597894720574, 0.4998423058649649, 0.8427395239254996, 0.30650124959254577, 0.506485023928878, 0.04044004937167944, 0.30006777185235833, 0.7494543957865538, 0.9554818850611343, 0.9416640572768163, 0.7503093773332243, 0.42535044173684466, 0.9700259707987543, 0.3208397191700295, 0.0901134543124904, 0.3254777108623441, 0.16648232424801546, 0.5940424247759188, 0.5861182714037946, 0.5850879693850978, 0.7435822844932616, 0.7751547404732722, 0.7997714547599905, 0.5670856679102112, 0.45207258549014173, 0.028552302509609828, 0.43271792856710156, 0.2581382834915258, 0.16591671341301228, 0.35789526967724217, 0.893444916178544, 0.8152504435654062, 0.44021038933107115, 0.8886972999268878, 0.2813817648990419, 0.6291659743195493, 0.4146067897875628, 0.9337898677273011, 0.7864097287281383, 0.4555309072636078, 0.30092911686067914, 0.9184941612659424, 0.20974497882386334, 0.0
 05572550831891121, 0.47978160024439054, 0.2862590094773161, 0.4264156848445966, 0.8851182534505836, 0.9233522745933701, 0.7057179803244092, 0.13006209455508022, 0.9240990816595475, 0.260234116299108, 0.2565645714479057, 0.12754588244333875, 0.7482534687144695, 0.3029738101846584, 0.7843391066679789, 0.7838635479375948, 0.9105660417714211, 0.27019004006152847, 0.2831671496765443, 0.9571157124329137, 0.022047532790265056, 0.25248611531188503, 0.990683462722147, 0.9087924888464464, 0.09978166798820598, 0.6950149139390285, 0.8608220050949611, 0.22639272576565705, 0.7847355988381378, 0.7994362742702683, 0.21784690713241106, 0.7534028141101199, 0.17265002590128353, 0.7841402931696285, 0.6889449591352023, 0.8859183511525698, 0.7483967299479697, 0.8839925808745691, 0.9628956246080232, 0.3657001192038958, 0.317509949779191, 0.8194904244924329, 0.5287797150026274, 0.8762132131551871, 0.8881700125886465, 0.22010171085421593, 0.3113831254805406, 0.3312064560213035, 0.17273631879274043, 0.147690
 71407558787, 0.03889757713992026, 0.7346466716042457, 0.7499986314050993, 0.4333187436955239, 0.25371545521267047, 0.8863302347918719, 0.9576346299756751, 0.014339167239079575, 0.10856100706748084, 0.009252428915467403, 0.2689329272472797, 0.8385903838659501, 0.47789593469613345, 0.5828225672351723, 0.24866045791181057, 0.18862201603730577, 0.4584821892899571, 0.2802709678132972, 0.18002386637188328, 0.4712733295048588, 0.0331803833735278, 0.9291013815687655, 0.10799467960916076, 0.6165549482067387, 0.5285077548816394, 0.8402005691753653, 0.353838099163194, 0.2346685333965114, 0.5091742577816274, 0.6842243605908914, 0.5215061365083828, 0.5911309225756182, 0.9217613356293745, 0.33313750114516116, 0.6386257010377409, 0.2773530584405962, 0.6249030673330307, 0.7302920707084123, 0.35313248542617204, 0.17765609619969858, 0.9247744183428891, 0.9041874496006502, 0.041172243265714115, 0.9635902947877515, 0.3723869709423696, 0.699062549197688, 0.27271099233732565, 0.882554945178537, 0.8731376
 266965969, 0.6597285501583245, 0.32836599580118964, 0.050100322811786, 0.1092512366889612, 0.7785585462055403, 0.3642901325548161, 0.8039258883495115, 0.38220316098309826, 0.4893080992510015, 0.38151175339191756, 0.9324710922285021, 0.2449034606715308, 0.37544869622856425, 0.3719534606389511, 0.7521285926715994, 0.7123695322152279, 0.5499535801303388, 0.8854225862496222, 0.07449216427992245, 0.4148089664765693, 0.5589838677320074, 0.7901870084757293, 0.07868435145249608, 0.7884804906980148, 0.9636796268469665, 0.184320522263831, 0.9836920523722743, 0.28739555708743414, 0.4622699432947348, 0.25582114586424964, 0.20177066207703798, 0.6428685329987668, 0.5289719122364935, 0.3865777228105014, 0.3463603492929892, 0.3981190183040155, 0.06665825658975932, 0.948578720846827, 0.06659120812599462, 0.30157496939303186, 0.4970294116009315, 0.05032910698439774, 0.1853359669677127, 0.6552204337413688, 0.04503496093859294, 0.9572601235490278, 0.02890281212744983, 0.3832774349035498, 0.713546288884
 1189, 0.03198790435010124, 0.15384286314709916, 0.4117065838172631, 0.5897906377052893, 0.818597023708054, 0.8999800795420976, 0.9304561182047882, 0.6768144025452689, 0.329695187898153, 0.3286541712024529, 0.2480849730847876, 0.4012511584981112, 0.6858057162293326, 0.1412995013903493, 0.24233007579791455, 0.31292802810651277, 0.003734874751794015, 0.11586648739959993, 0.8603841697240969, 0.26235145962198014, 0.8338713257608387, 0.4472239968928975, 0.7290947939700906, 0.5188717871279822, 0.5059451494532529, 0.13256492635624928, 0.6979913815290494, 0.8753330713862888, 0.4585944826278423, 0.20396906003560655, 0.05699217967842696, 0.774273223124109, 1.5789423259426894E-4, 0.17766082571209596, 0.24583785534526903, 0.8862468297432353, 0.9764311474229035, 0.5631321098285631, 0.10374453775325287, 0.3726009620206223, 0.5056970140328546, 0.6551829267837634, 0.09546392743783949, 0.27304212695132124, 0.282157412660642, 0.11806594739983634, 0.49388338028491385, 0.07259026274436953, 0.94596727794
 73772, 0.06330339814377595, 0.1994946023178047, 0.5230237189572382, 0.6713900407484911, 0.4213776887257832, 0.7335044989288835, 0.22067653046699398, 0.434164572293658, 0.41238010259036517, 0.9540570181191029, 0.27545755794949567, 0.3849943655172885, 0.8318345306238636, 0.213980689885782, 0.645218605545174, 0.3781391497267089, 0.6852960795665282, 0.003671070935885812, 0.35510666032643323, 0.982410592234395, 0.4289573686325143, 0.604163212113803, 0.8732927657569071, 0.23644280686915287, 0.7033776956492072, 0.9131444891566204, 0.3724785902906792, 9.298029682265341E-4, 0.9835837808662378, 0.40060312266098086, 0.4708387705888044, 0.0075915227069958124, 0.011172740640812195, 0.10100344394041061, 0.31656547245515143, 0.6079509033925687, 0.22268150822916333, 0.6553240539625368, 0.5569392502523293, 0.7845745423823492, 0.643369452497378, 0.6789901880376152, 0.8158579572104679, 0.20324979554781664, 0.8766934905898232, 0.15067746523202952, 0.7930135968205893, 0.9566414907861178, 0.8871381733086
 732, 0.09215869734148618, 0.9934144024161534, 0.1782920200980892, 0.24769610339045844, 0.5408115975154211, 0.39770771231768265, 0.8903983143174082, 0.4031762991895841, 0.7040997850211647, 0.09176647909867175, 0.28962011534845866, 0.28665154253489, 0.2292854998129098, 0.9882607488004646, 0.1722438214771872, 0.3187383231556413, 0.7218269233930805, 0.29344241256460457, 0.35334686950601923, 0.4180356369498216, 0.0023593632882223936, 0.2991302027153149, 0.7441775536077232, 0.15593196911813823, 0.10485380379683451, 0.4128891336672329, 0.21778584010890445, 0.6001225507536689, 0.6874923149033791, 0.07865652092151654, 0.5150107191924342, 0.7085615976735176, 0.35299514530409226, 0.3855747229981953, 0.22125032455755944, 0.3592073191516566, 0.35657895965268926, 0.714548468905114, 0.2790109900829014, 0.23080021709860543, 0.3174219240876027, 0.7300521470709977, 0.5920519694078614, 0.7860477321453403, 0.8434559791961826, 0.38897076875215264, 0.02401282601244692, 0.5509715707990103, 0.4295021244969
 339, 0.4087657363232733, 0.4977305227353601, 0.732806636835167, 0.5952100835168144, 0.8140146241058145, 0.9610242524229055, 0.9737230013229154, 0.9919585636476513, 0.7633815455655985, 0.7005454478436567, 0.6987759861397144, 0.12681830302668462, 0.8483332431733817, 0.857361497193666, 0.0838709719695161, 0.4268746447013715, 0.45042297964854006, 0.026261461419732957, 0.14316395057938325, 0.5986522381215751, 0.5457229099523176, 0.5053428210815473, 0.8152000429618768, 0.6250797024021244, 0.5529927579205115, 0.10325831952848141, 0.213226219782507, 0.2863771120480121, 0.43643108144204457, 0.4085724035681998, 0.3974967585303667, 0.7914448482005235, 0.2943625331928895, 0.07899687729634475, 0.12602266880062107, 0.9627773558408685, 0.09197296253096177, 0.01784616281598006, 0.04509962185605876, 0.4306208518713036, 0.029211603863418922, 0.27087221506288994, 0.6920810299809184, 0.46103945943700453, 0.2742375222352209, 0.09871014046099391, 0.5002465322638636, 0.36774901910979285, 0.438295550626616
 07, 0.9584852682543622, 0.7192482909972147, 0.9390238102723785, 0.21966420181312774, 0.06270312877857565, 0.7945006455479464, 0.948676490254168, 0.10232651209368204, 0.7743641039353296, 0.28501148213919525, 0.6140563652268377, 0.8592719713356806, 0.34431446152254497, 0.8415362925895201, 0.8142073890056368, 0.8808460425093462, 0.4001662589407222, 0.11556893853971362, 0.9877717058606662, 0.025321322135442026, 0.15568860872060564, 0.724565879896658, 0.8929205086859203, 0.7718719306120216, 0.5253932005879912, 0.5599332459544969, 0.830538628280904, 0.9453912018915441, 0.20110738496851732, 0.7916892656651894, 0.5070854640930748, 0.17810009418804862, 0.27564822801414224, 0.8973471867420296, 0.44499933329377583, 0.5746513531131515, 0.18328141895598615, 0.9224465871443851, 0.7362080486490924, 0.7554430411251164, 0.5674691955123351, 0.9662939617131937, 0.9782539269747884, 0.5671119798227955, 0.18348180552561477, 0.5788561297640441, 0.4327363449207118, 0.26968597426273666, 0.09676707043900046,
  0.3533828927397278, 0.8022208726297413, 0.9366316278850002, 0.24754347623914796, 0.5042181765537848, 0.5344586187795173, 0.1370417104698921, 0.9179017592202274, 0.6303917998346977, 0.23170524583924057, 0.7084646449880674, 0.7597712470052572, 0.1829663612559268, 0.5681244484252116, 0.5743859159308287, 0.8204263704905678, 0.3777475034632012, 0.09970550061518646, 0.11212933669243885, 0.43678014579832614, 0.42484739660754367, 0.48882027500033487, 0.5349536865080836, 0.9605311775296558, 0.9827253552238541, 0.9594904717019531, 0.11637831121337494, 0.02330089706609384, 0.6867109917657119, 0.24963046094186347, 0.8552788187015238, 0.4542186203931423, 0.45064952333076413, 0.9329775222164091, 0.3570520324938192, 0.021981994492264345, 0.9951905248777341, 0.7461153482717665, 0.28915703104048773, 0.8892403789353457, 0.7173747348708638, 0.47494322638336917, 0.75859697737764, 0.039219827307805044, 0.5835564263313942, 0.030137470027180036, 0.3544143249656104, 0.962827968878825, 0.2013312683215327, 
 0.5725979063425582, 0.40002955800015005, 0.5790475691107264, 0.6060229206434072, 0.9375670866573094, 0.1682617971914897, 0.702248108195117, 0.5834139355562845, 0.15005506260639134, 0.4261271083485495, 0.3671146445653527, 0.5427897025415117, 0.5243984315557189, 0.6970181745631603, 0.9752139110710698, 0.0529967797682237, 0.6972271140021967, 0.05462719296673724, 0.536787624501907, 0.15909984339887928, 0.9558078152033788, 0.09598107792373001, 0.24718341749533523, 0.5083543701000781, 0.3926787325473472, 0.9802164100972685, 0.2594041189182621, 0.13159484047461645, 0.7426848454312565, 0.06522880009620358, 0.792576463585391, 0.44977111393670843, 0.2399688836171211, 0.7260900291912906, 0.6507105109540988, 0.7052485149009664, 0.4701681740481436, 0.3971330904067888, 0.816941378337909, 0.5536198327273493, 0.9870128941066615, 0.7438637468379411, 0.03551530250499835, 0.9941605442111137, 0.41271669206637995, 0.14136506612883837, 0.19710247466994113, 0.16569080622664623, 0.8479957409032417, 0.31261
 86104713998, 0.9025888027880039, 0.7894913427987832, 0.680247755134519, 0.06317464641717208, 0.796319976895931, 0.2020525384723244, 0.29736794942543043, 0.5378605503692692, 0.13184080386630703, 0.9878190984552762, 0.789873592377593, 0.32072541304509616, 0.5614932109469892, 0.19572554724609947, 0.455455242391302, 0.0011758002684753022, 0.32580722031375164, 0.003200521912941401, 0.8077668618558361, 0.5372658672085715, 0.8594384983117677, 0.4774504377862483, 0.8352131423077678, 0.861363177725784, 0.3274772401692655, 0.776216512421499, 0.45568724345, 0.6939182043988844, 0.7592731984035581, 0.7288312719258069, 0.4696578988111745, 0.45782491433105865, 0.2564917929985595, 0.24328379709731152, 0.8266995331952197, 0.5188662099076208, 0.21793675173708404, 0.39613591810790616, 0.6895049518018261, 0.716996761031195, 0.6457450982800385, 0.37951403764466485, 0.6046151522233104, 0.29872318089846217, 0.3758802952794803, 0.7030342667923438, 0.07054610033689768, 0.9125991555304048, 0.5104201818886519
 , 0.17840151434321794, 0.9316879143150584, 0.2878614701590234, 0.419955963129252, 0.25055560182412995, 0.320296744686591, 0.5804902142776897, 0.4314896992115369, 0.18162067973500728, 0.31909548093464934, 0.14148440100505622, 0.7205370937104177, 0.4425466902809406, 0.12344433426509793, 0.6389021944461434, 0.9857134518580773, 0.422214863268519, 0.24473775752973326, 0.04822242412172362, 0.19990068760362778, 0.2249114119836393, 0.21562953040225297, 0.014842434284084938, 0.10416354161746189, 0.8944572330686985, 0.33288330979197656, 0.48803425836368597, 0.1389033893555266, 0.87691339432925, 0.20509074728086052, 0.32744006706537254, 0.19130886900133237, 0.9136655121572498, 0.2601952995307626, 0.7911944467144493, 0.8789820021120222, 0.8428230026217683, 0.9117042133049328, 0.4290198522950259, 0.08668971885892618, 0.9792514286322237, 0.7826679816019892, 0.9241487772318742, 0.12651012879169954, 0.8421897268770978, 0.24294983624421196, 0.8546843076388009, 0.20121410314940202, 0.0042863449771815
 , 0.5280052093905581, 0.1780183966832669, 0.8165419709967803, 0.7470213227280779, 0.8763349175626627, 0.48118523936368174, 0.4101068045244295, 0.02486429187809447, 0.4479718934309572, 0.6855342827637888, 0.7429744993982571, 0.9641509336704079, 0.2678064388605681, 0.9458663390456756, 0.5345426194423784, 0.7525434866398159, 0.0752500320278876, 0.9421829547556677, 0.4121594067722143, 0.31107347031904575, 0.7556835411548172, 0.6264576090557468, 0.49399482642357284, 0.5831478670540906, 0.9793303255165569, 0.7573450959725314, 0.8269961095780269, 0.6235324962676264, 0.7796410316857281, 0.23250314301924258, 0.46133032601388435, 0.9252096090522305, 0.06597065774846655, 0.3112079305934419, 0.15853675477003537, 0.5831387993629328, 0.4842756805360259, 0.3092035527508744, 0.6026825451213526, 0.9613341068403553, 0.6147526874103149, 0.2432327525594682, 0.5121348181083518, 0.5033019799433511, 0.22678613429973826, 0.20445408357286432, 0.2341334554355602, 0.20576849560903654, 0.19926806372983707, 0.9
 479590526344056, 0.24516837732026564, 0.06175628700740465, 0.7582978142058575, 0.2998597914747062, 0.47723596995780626, 0.4714550539542143, 0.8844649205803417, 0.5039261719159012, 0.5433693558052619, 0.21796192210259135, 0.09836419962923626, 0.5565090698913151, 0.7454077618779978, 0.10985079706922074, 0.3456865733196951, 0.9114938906456905, 0.21278997669731126, 0.8036630940546842, 0.37217554401860087, 0.6762867430958188, 0.6432281498059154, 0.6367942625686542, 0.8039331990353842, 0.5653433201894608, 0.8944375665152543, 0.5248840815806197, 0.13000641672356938, 0.4295079320273787, 0.11066886262477282, 0.10770110547774359, 0.9218779551448312, 0.1374390227473058, 0.1586092983922256, 0.3636444900475885, 0.9184332593115897, 0.9496538275939085, 0.894290216216021, 0.855369407390396, 0.05516880131032942, 0.2962340023367712, 0.1859576341353304, 0.6176749668720326, 0.12285313378984486, 0.045991076458977265, 0.677433910204084, 0.4536762045708618, 0.5035470996381639, 0.9378985288623584, 0.992318
 1519640675, 0.06641485239380662, 0.3508350971922044, 0.8348554300650259, 0.4514436118915295, 0.020446281563239244, 0.6563144294846395, 0.8553540141303887, 0.40433073524554697, 0.0015725166291474801, 0.8758352383852362, 0.2403813544516047, 0.9599445578851037, 0.222774653127367, 0.17667877379496566, 0.020893624329418437, 0.2428855483046778, 0.11053016421151263, 0.910806908410904, 0.16545861669920825, 0.29821363115577626, 0.5980838928240354, 0.6195114462424972, 0.9081703039427208, 0.4214248714849197, 0.4508740590896084, 0.7312369492519667, 0.3836972102812899, 0.9996709726698721, 0.05419754480354977, 0.1270457149286025, 0.5293337391508313, 0.9161322289883558, 0.004331467003930056, 0.9171397280091905, 0.4531817916172246, 0.1915242798606288, 0.6617109186816091, 0.09704875268097934, 0.1130976732981539, 0.8150399338247446, 0.5483956025872616, 0.2335397590246432, 0.2027970124605426, 0.6515974098708037, 0.10194057111175703, 0.9679978920624334, 0.5995185138857569, 0.9030863567384375, 0.8339198
 120552312, 0.9189743856640564, 0.5919047788541734, 0.8483741343618643, 0.38373340885992, 0.5157925692587775, 0.37726659151029096, 0.4901540806957235, 0.30696227088178996, 0.40154778858026685, 0.9941548138364408, 0.29628865636980484, 0.6924172906849964, 0.30653310632122266, 0.9391714497954137, 0.43912041916484046, 0.8631623454693232, 0.7120673674876231, 0.20620997931264262, 0.8082937732560264, 0.48417360834559353, 0.19989770848133603, 0.381822206360688, 0.6009512437043784, 0.3729300546704558, 0.11379414324898418, 0.7321670860754439, 0.4777381695734523, 0.1891137016851352, 0.04764878876103573, 0.9250044607318865, 0.11386956183941044, 0.9657296535113885, 0.7183379069302285, 0.14445499756620583, 0.4715841783817336, 0.13634014690898222, 0.9817999891451169, 0.8767547072812814, 0.27363006641255916, 0.3117387675232468, 0.10722878497564081, 0.5719909128226707, 0.9669969807416556, 0.5642739906948241, 0.9369750179893579, 0.7198138706628845, 0.9319309260259793, 0.38916430119695733, 0.2286963420
 3986866, 0.6163762989289986, 0.8133366340766617, 0.8295295429446919, 0.301489844295057, 0.24526088457831396, 0.6299939409039189, 0.475496273296648, 0.08918739144091581, 0.24691345527112574, 0.7515842940123804, 0.1776176145553361, 0.8416888991364687, 0.9091076556667699, 0.9243444604517081, 0.3509808397999925, 0.0091538359355392, 0.475224321474468, 0.2732336702047826, 0.688758760861252, 0.2940673538632148, 0.8376369517529844, 0.6118223135007088, 0.9525939924766214, 0.4626417460344203, 0.8602524591728781, 0.4506790733172733, 0.2620875735892926, 0.26971001360542823, 0.1572638335403116, 0.8108548122389465, 0.9667967840910865, 0.7702131252183498, 0.14442964812252557, 0.3211313395265055, 0.7515292318018629, 0.36557841141596603, 0.5747026532705615, 0.1060233564884413, 0.09935363529672869, 0.16473114458969795, 0.5717882130741224, 0.6560928481561894, 0.6166299134611771, 0.6271773493826087, 0.38322017238251693, 0.21295460247141074, 0.010494954186543337, 0.9995407061023464, 0.6888005144665841, 
 0.1984227623564916, 0.7016218832108055, 0.757132062262647, 0.32075653111809543, 0.8705677450781013, 0.15370509457617276, 0.11857510945377192, 0.6787266672014818, 0.699023999000941, 0.006004641940152489, 0.7142072723245152, 0.9556951212969943, 0.9177088405255228, 0.006417189458924999, 0.6579201974947178, 0.5302517044945433, 0.3351829310585649, 0.5599236615332491, 0.5972993333006158, 0.526062506612915, 0.7612037407715797, 0.4308958335538825, 0.798367463045302, 0.5911938098570977, 0.3425931274474979, 0.7676275762774425, 0.9589328587958623, 0.4079400302680706, 0.6923606336291015, 0.1535891290352588, 0.23848694808061366, 0.30806434991598575, 0.9430946477178466, 0.8322851829700096, 0.9427929958031837, 0.6388677700113338, 0.7589535587785896, 0.6012311652216156, 0.40775155973567645, 0.20226132367425076, 0.9001596228753468, 0.12544379920111282, 0.8898244629990352, 0.1720226815624072, 0.14971819909798667, 0.5819762749497424, 0.6892438799619931, 0.38367735580190043, 0.2841559153650799, 0.61440
 96538482906, 0.30742889444976784, 0.786951761850725, 0.2983909822321603, 0.5498461607635556, 0.17147671330593872, 0.1525034546782864, 0.27941967148316993, 0.7108321498335443, 0.3462914966431947, 0.8326417675776641, 0.16432303768956524, 0.44077144970965476, 0.6988586657080509, 0.4900098341387439, 0.11516837297121307, 0.15377294884969595, 0.1793683136070442, 0.22870652169644956, 0.3967000965428408, 0.3185085389018937, 0.24033602114523056, 0.48908511650458586, 0.452285327721233, 0.7905409954007595, 0.9275655029094122, 0.40047309744917436, 0.3393994245353916, 0.3187133428006709, 0.8457420823550293, 0.025314925276451183, 0.8080070945872039, 0.3685751525693205, 0.026641656532581104, 0.0737483074360874, 0.08351546426657586, 0.6294529012268423, 0.709385309519263, 0.21938137133461622, 0.8505449535942865, 0.7016552613818594, 0.17986450236009244, 0.10504655638857241, 0.024585046289548718, 0.49643634729725317, 0.11040754980599654, 0.4915647017965902, 0.6456716557715012, 0.8237231546239848, 0.29
 095337044572955, 0.7730603733622646, 0.6315434797159377, 0.4912462166535341, 0.24944338617333606, 0.2591279522739305, 0.1873048544716257, 0.9935947871537972, 0.021838454330492252, 0.20715479378043256, 0.384443075065393, 0.5308516642263811, 0.2511121591043951, 0.5773665198791101, 0.7941183406453303, 0.6638446037834189, 0.8697681696198335, 0.8786670486051076, 0.0879648958641277, 0.3866845475552262, 0.8518735630169991, 0.27510320715173, 0.20421223806418176, 0.7771437761634884, 0.908296109473817, 0.3307926491096994, 0.01775535878571688, 0.6715597268386798, 0.542778587084333, 0.38541052759503946, 0.4498737205437823, 0.0033329901563965425, 0.5866784298358899, 0.572839048859098, 0.6066186800039455, 0.5427210101222274, 0.18664635203280833, 0.9223412178728273, 0.8444592059959983, 0.7805116610022261, 0.23221315660234665, 0.8146283364494517, 0.6275967502980727, 0.6092937138189531, 0.37919869460974076, 0.3044483524768764, 0.6634294030970315, 0.2734241907079459, 0.1500528510552076, 0.46683010336
 071584, 0.0809671013041744, 0.7824250956064195, 0.07368349087137072, 0.9758762219818818, 0.35156838146712344, 0.612425107710916, 0.4650213732519488, 0.5830055604945534, 0.24219099993834892, 0.056394427107598455, 0.2385179515856264, 0.025803199334669902, 0.3384180784015499, 0.02630399381990023, 0.0999664851686104, 0.16914466614887158, 0.5778420875162874, 0.4589730320764076, 0.5674831863739681, 0.8570181501753924, 0.79395934599347, 0.7322513753829706, 0.5351792940600397, 0.0364768696149963, 0.4865476229467349, 0.6593164672776828, 0.5120013816454225, 0.2044110528439863, 0.9065707505226107, 0.6962471616883303, 0.24341180844722143, 0.12333715119238386, 0.0719728284797323, 0.9716695368605864, 0.1550266132756617, 0.05415965974105652, 0.5152966526334458, 0.6271526957887353, 0.945023927570415, 0.461868521600416, 0.40488015151800383, 0.7431654904008391, 0.6960807210406231, 0.26439317117027683, 0.17845532569143385, 0.817787471764316, 0.07779115906979261, 0.21534210278863297, 0.9812944997659246
 , 0.3640008330182526, 0.5669656010336538, 0.9381141349332752, 0.010618991018260582, 0.6962722467161523, 0.12590396104920654, 0.4618762585987539, 0.3338055554192788, 0.3882668369746284, 0.4476827395786437, 0.8491599227837112, 0.8699963560312989, 0.2009192215257951, 0.702690532275999, 0.6503358153234549, 0.0030385460452050372, 0.5037189223881833, 0.702091754650066, 0.47198510877021926, 0.7282495265376498, 0.34195108572831046, 0.5315938796172586, 0.9041926692518983, 0.32632494293399095, 0.33876543605381804, 0.0028388353676152, 0.4160457205953154, 0.3047760505713317, 0.5633297867985563, 0.7199344127707322, 0.5991292012265959, 0.750603831272764, 0.3449568991478431, 0.16508654010255486, 0.31556694979026334, 0.297487599945532, 0.7679764690209776, 0.9699896478994348, 0.8901655196301158, 0.6265856389895746, 0.24325684302454686, 0.45857600022565925, 0.4754753290858976, 0.7921165460973875, 0.11966368811179307, 0.6219219200508768, 0.11409184006750639, 0.69120933369637, 0.7723758626767633, 0.917
 5529229786347, 0.8242767169102428, 0.6820815415164386, 0.9848614138190812, 0.6730854997054412, 0.24298747764836692, 0.6954242455881324, 0.7274496419893745, 0.4180887906761944, 0.9746613581632468, 0.18094347509292985, 0.6278176509981577, 0.16824437841327478, 0.8146072609455339, 0.8843279461016447, 0.5273427666224223, 0.5423280015201595, 0.442617249440756, 0.759482979732328, 0.5221554223557527, 0.6052319773031952, 0.015417074502313666, 0.28669121353128824, 0.7988590264986991, 0.4991928633228202, 0.3258329810005065, 0.08943074881352253, 0.3922670104156021, 0.2505839018619497, 0.8420595252297446, 0.02090158246056062, 0.71613976296217, 0.6210770736622231, 0.9921715742148991, 0.8258820559039151, 0.3477866580065787, 0.5096752128341602, 0.5170309254757595, 0.07435650075613709, 0.08353161078261007, 0.20605945620558663, 0.5789960621202394, 0.31042647353854413, 0.7244234518404261, 0.7458932844872302, 0.584351132000112, 0.26333749412561636, 0.40020452697113573, 0.7251688176915079, 0.35412863728
 101585, 0.527024222729352, 0.4944350296866734, 0.19337751892587085, 0.36100714306429826, 0.3303965482502891, 0.3948016726488497, 0.6642313600310363, 0.23922424123876607, 0.8926603650785785, 0.3600238580635353, 0.8578572931743235, 0.6699519854197541, 0.3862859073283945, 0.7684021665931493, 0.5198318770967285, 0.04634000998217225, 0.7148080679967145, 0.8929369519203771, 0.5501191916441033, 0.030439979668074213, 0.15346253629310447, 0.13407829905566182, 0.7189199545929209, 0.22848163597245053, 0.39773220705306034, 0.3335505522625487, 0.44663504226636996, 0.731479744413183, 0.7181987510145069, 0.3913280084095154, 0.48159249943388105, 0.19490818782054586, 0.7296644987064214, 0.9127209684702927, 0.8218816077410734, 0.4641859308216506, 0.7483297645609274, 0.023664201769330595, 0.8965836605608846, 0.49197644069961544, 0.8328815988073396, 0.8096826318759559, 0.8468836960511423, 0.4388114279906986, 0.3836698847968584, 0.05764201271428537, 0.7289936240870691, 0.4003058935108641, 0.608717099349
 0716, 0.10927490262733475, 0.5985114150339909, 0.16224075506860514, 0.25612838706431784, 0.6716054058942826, 0.7297874266574995, 0.7490460250751146, 0.7595390302220656, 0.3717786540496414, 0.6740633128068716, 0.3017262240648606, 0.34655632783832946, 0.6924341098908663, 0.4665950240441359, 0.9980266896966941, 0.7197238208008914, 0.6786440647752137, 0.05566619589357524, 0.31064804378020183, 0.6353554944666993, 0.46857244457510583, 0.9453438110172021, 0.888573182020276, 0.07765817165618116, 0.2673906354028741, 0.11902319548564966, 0.7420795539131033, 0.7538809282481337, 0.8576210019823539, 0.5422549561725184, 0.2137669385824883, 0.32079990297924466, 0.11896089720896441, 0.9310412536652506, 0.42593888643272293, 0.09366810420016791, 0.3167727096931955, 0.015734859515002975, 0.508933577342277, 0.9858293119683041, 0.11080904573879746, 0.23086914553738103, 0.9355083699347524, 0.08051609301223761, 0.3343741537841044, 0.6176927164982957, 0.7131388924145485, 0.5459219493018189, 0.5990056166232
 66, 0.7388776188805624, 0.019102772904492582, 0.5704578555905884, 0.6908939660086089, 0.5738156676840643, 0.10247463195095319, 0.6171742351891368, 0.5426394588010912, 0.6881122829759565, 0.9226677835599907, 0.574649032943262, 0.2875789362056447, 0.45233276359010355, 0.045915056371938934, 0.3162133690163248, 0.5337984130640945, 0.7227514019186893, 0.7790898594524628, 0.7702065382740432, 0.5310793789244156, 0.190798888614663, 0.3845254268911068, 0.8510221707664553, 0.34640995855655454, 0.5580988589929421, 0.4899988370462578, 0.5040279191528347, 0.512325167771079, 0.8004299257601769, 0.4848038909225739, 0.677011030938516, 0.8672226465501673, 0.04664436545817885, 0.7758443674003616, 0.24662316170979603, 0.4679333927718715, 0.017520611517636064, 0.26693795156527855, 0.8003671377576445, 0.5222102621285061, 0.5447215603076837, 0.32002762032776, 0.70298964395039, 0.26242889779580014, 0.4214759081493593, 0.7587566077753214, 0.06071322545607871, 0.5616793766403034, 0.7242956794660044, 0.18286
 933060524235, 0.037562156007833525, 0.8795592866308624, 0.7889875886005645, 0.9761597174987287, 0.4751063923971357, 0.8173618068723619, 0.22441182096608947, 0.9111955615547368, 0.920268523066743, 0.16098524497499078, 0.7038870028385277, 0.18080542450657033, 0.9754086591269061, 0.7947732323972809, 0.7868350300872958, 0.019322445833274582, 0.35072278602737317, 0.2806276551769755, 0.6922054153759251, 0.8158374321728183, 0.1210602547102273, 0.2141191368695171, 0.7704136109746659, 0.7672210030811933, 0.4675864830948213, 0.6929527309681688, 0.4019730988439846, 0.6704327935735236, 0.6192870257641446, 0.3630293768790328, 0.18869685146774773, 0.41164339112322357, 0.8001299203073687, 0.5434763137441017, 0.5972883573052052, 0.07395202349943852, 0.734773142809937, 0.8744026260924345, 0.6653383068890373, 0.3559919809447112, 0.6690242796621383, 0.7797687439013284, 0.9861879696450729, 0.1704721943100841, 0.9208304760051743, 0.7926418567298239, 0.9518159859825075, 0.874712172553366, 0.6933959536633
 37, 0.4512413966345671, 0.15185004046801875, 0.5744849329800966, 0.25409577762398095, 0.04156233707526458, 0.9020008306214633, 0.3379904388063366, 0.6111742604250688, 0.9359962435453864, 0.10666068178182186, 0.958539899309466, 0.5047867527743277, 0.6572530761010865, 0.04810439289193369, 0.11253889429261132, 0.869764039912697, 0.08301967102853802, 0.23867566935789464, 0.3736155657909541, 0.924548818822202, 0.39917326095915606, 0.17253722783070746, 0.5642473920628077, 0.6940740097832543, 0.508001730522673, 0.6700084150952675, 0.35299815477778684, 0.08392309111909302, 0.6027646705734001, 0.5202092705026745, 0.838484661363187, 0.5889325378710776, 0.5310622080537286, 0.29667540672629045, 0.100450143662974, 0.34146752216251275, 0.38686166324077476, 0.5458032165076802, 0.343829586023504, 0.4829216129216162, 0.4946645465966969, 0.029430448741288084, 0.3134496183964479, 0.7765617988412156, 0.5791794905069685, 0.34055233106374416, 0.11501381628056306, 0.9706607525759751, 0.4499991949485759, 0
 .005118893123088442, 0.44228161037988967, 0.667927872740771, 0.9731627661920299, 0.13305410063762713, 0.480857253271346, 0.3605718313748213, 0.45011724527485886, 0.7880833055273724, 0.43576285352762734, 0.8498968399198641, 0.7794748651839504, 0.5086503848597909, 0.7129773084049392, 0.48515588978137825, 0.9380960149787511, 0.831151271212023, 0.2743809400712841, 0.5284537843626547, 0.3854097157969434, 0.24190610109292776, 0.5810533054837724, 0.48081380851238886, 0.9685173829014049, 0.7157374047496122, 0.841629378951784, 0.38909171129950393, 0.785507259501351, 0.8867170919405243, 0.9904274865769396, 0.9942412460751007, 0.25054064726107106, 0.9405901411663348, 0.9343196769374251, 0.0785056556101783, 0.2966536767434689, 0.518578649271087, 0.1278235052283786, 0.5224298951625054, 0.25085052907494243, 0.27871472434101463, 0.46265170044246906, 0.39215863539421936, 0.4209478985313817, 0.2735460826519791, 0.7909639946625474, 0.8433056867728854, 0.8447622747361028, 0.9720040208393559, 0.1408236
 3935063147, 0.7497501410418217, 0.9696152864382608, 0.5784181426668197, 0.8278681765267607, 0.7440356475286127, 0.2537644442364545, 0.2831077480026203, 0.2189627041580532, 0.790618956549776, 0.8112499932486378, 0.21667521600161643, 0.8616109860741181, 0.5040343445340554, 0.25778109283887773, 0.17407199033506138, 0.05715794479536185, 0.17111115987801084, 0.8576673115821983, 0.6247028122009419, 0.7263248804918596, 0.8397013028725729, 0.9204345063841997, 0.13127979417497282, 0.592376329893984, 0.8959447638151657, 0.1506468894212598, 0.06283219192409872, 0.29022553060760126, 0.2679206504574202, 0.3155894455275885, 0.406224468824523, 0.9990433183941252, 0.765602209227177, 0.1628485743618926, 0.5557289918787093, 0.5914426446346717, 0.09535424867667186, 0.9934606029823332, 0.19164646607961922, 0.09406941243686529, 0.7717818368811814, 0.6761816537561409, 0.7466613013809039, 0.46243738495932274, 0.490559436887117, 0.1745641769115578, 0.5025654141809082, 0.2871035285570175, 0.5229398551925248
 , 0.8509091110042714, 0.06017844060031685, 0.06878720194794297, 0.3459628950663939, 0.9097949441708687, 0.7817383941939957, 0.12679486238370963, 0.031953728921505165, 0.34876788123134117, 0.8177314856571287, 0.03811986615200946, 0.8867214770068355, 0.6949101761267372, 0.4990756548603804, 0.7438441956240472, 0.4479385285193771, 0.6233865850459849, 0.20076714080665314, 0.10497751543832967, 0.23350177913061343, 0.2777013709716829, 0.26454524880795593, 0.7416260150630134, 0.5642472322054733, 0.7412817310662908, 0.20582571230606506, 0.44739210354153547, 0.3762091706497832, 0.6461353613278975, 0.22409754029341744, 0.3351627872357331, 0.18468651302439099, 0.3467139329658908, 0.9861751194748926, 0.14095368496457028, 0.2656979532932692, 0.43574583811560075, 0.7581410152113619, 0.7892322891186839, 0.6799789955145156, 0.6336851838792013, 0.2245114218723886, 0.4061524533375549, 0.9804935476877459, 0.24455666321325287, 0.7521007384231915, 0.45638985258603815, 0.3165033130711562, 0.43835836884048
 707, 0.43125773104818066, 0.5747760352159968, 0.17555842676852051, 0.5938180938423008, 0.958590375050545, 0.9656860429981097, 0.18934342002214088, 0.815180631790393, 0.312992410472407, 0.9511432171990515, 0.27013240273472605, 0.5228373880611316, 0.2207294262601739, 0.3628616222175107, 0.8416780499148914, 0.7099914127461673, 0.6055380223198037, 0.9939553177761559, 0.08167915928828762, 0.05618469823407268, 0.5761365887344077, 0.2528007262019222, 0.902541841088051, 0.5933966599105088, 0.016241932164534623, 0.49454655314609597, 0.8629698070343218, 0.03155590812462716, 0.5111306722872846, 0.24502362318134563, 0.3249893265004905, 0.16769041229487636, 0.16998041023983768, 0.6773144370534758, 0.9493350483086297, 0.02657328322587993, 0.8846703797012678, 0.2530002731413845, 0.056692899108831574, 0.5106050266641926, 0.703416849746635, 0.3981117352500039, 0.47311685596981246, 0.3728054484162334, 0.29156395856152606, 0.4480658385535843, 0.2250062403389559, 0.2691711922247152, 0.9020456949960223,
  0.8663811114831105, 0.8523513194768945, 0.4952879127306953, 0.8844802601306992, 0.9187197172516658, 0.16852421118393235, 0.7669690590160729, 0.5245533480702964, 0.5668135460350576, 0.010713540438243796, 0.06582518594929898, 0.9242921479859026, 0.3360888525139135, 0.8971706562209252, 0.13805188419956116, 0.7090271177962141, 0.3063120883981503, 0.5357560893916405, 0.7083559675632027, 0.5782839479230808, 0.17633710731738683, 0.8128452318382258, 0.850823082619438, 0.10554365107849373, 0.5620419878563053, 0.2324720624634775, 0.2446626092822375, 0.7357535763766626, 0.30080143547247584, 0.7726606717120681, 0.2351482571926684, 0.6296679029366747, 0.5563483984801392, 0.7792142124140405, 0.7187059963683234, 0.8605411953218199, 0.9446365993505675, 0.8268014592633378, 0.8759833264718593, 0.4813941076450857, 0.7343232312970526, 0.4430012150685939, 0.6878671607172365, 0.15001536618340838, 0.41474924886972764, 0.21303550639835134, 0.9867999747595417, 0.3240598445719399, 0.5698019254603792, 0.0612
 42374277864386, 0.5193769870399788, 0.5200477386056075, 0.04780286558200786, 0.4399097859120732, 0.12608981779002548, 0.7677645195562787, 0.8755718378389475, 0.7469121318814547, 0.680215019592432, 0.18550810342310786, 0.2290748708150202, 0.5406614200003986, 0.8702283230924698, 0.5518233628693611, 0.04357322401679753, 0.024205395245136763, 0.2587588872667397, 0.8558104806094124, 0.5229467684120374, 0.8298751797104521, 0.3059798706008122, 0.07078862967766908, 0.890874159947114, 0.35170525001123, 0.20646531724019335, 0.8150417511681354, 0.2158091202121627, 0.7449190393365587, 0.45592519378987717, 0.13856856016174457, 0.0372006231192793, 0.24009388436850232, 0.037864277346812414, 0.4062041370422149, 0.8221621428584405, 0.6041885990298087, 9.244534227337287E-5, 0.13517603107718612, 0.9898030750325235, 0.4214385085476344, 0.9771473291261004, 0.6891026431261518, 0.354693448403968, 0.3641919227713246, 0.9070730903725, 0.9306376868800198, 0.36144054776824075, 0.7770829989911384, 0.7896313815
 36878, 0.291408832511417, 0.40911402277768594, 0.7075060743637271, 0.8379039978727406, 0.23059508732200207, 0.8972639117305748, 0.9811768497380956, 0.7863792476656438, 0.7224319670937531, 0.3645338081177145, 0.043636235489237785, 0.43599671733992706, 0.2381288757049047, 0.236379066577022, 0.06687260005685791, 0.7402582465714277, 0.669129145442831, 0.25015800538565447, 0.2548091727766527, 0.8663955720188732, 0.776337407841793, 0.5838716802545911, 0.3722228961829859, 0.7017179024192951, 0.04388528379847578, 0.0790855750284295, 0.26873184106807757, 0.21764550356699663, 0.2662010321788608, 0.719074687549844, 0.5728085573612428, 0.9525682885182765, 0.803594415631729, 0.6999908658365739, 0.7903466588596609, 0.6295510388051089, 0.20426324505899918, 0.015859480164364226, 0.7581288711041426, 0.5860399715583313, 0.8095735058122479, 0.46746988701844416, 0.7746145783833402, 0.2601076635370766, 0.03466000843820671, 0.4007325138140354, 0.7587853314036797, 0.4514070068045716, 0.2478401281604411, 0
 .2818324293107428, 0.4723573298416469, 0.45488668353019435, 0.5600284167831159, 0.42451819730845375, 0.1814899479511909, 0.4514542152148372, 0.3792849565566798, 0.7991211254617367, 0.1311897759980709, 0.7321551780140371, 0.28582605261481075, 0.8734094836350498, 0.8589298243897145, 0.43106988776504385, 0.6392707387609504, 0.8621387974685903, 0.6835741962192838, 0.44701187992047053, 0.3425922546801097, 0.595167482489662, 0.6715185599590932, 0.1452452150036222, 0.3561887105658682, 0.4532655906871298, 0.004026996587467635, 0.24782041685923628, 0.8806824966544584, 0.46625102999039836, 0.9480664631308262, 0.044761743581658076, 0.6085036649510017, 0.7198465641926225, 0.5675099110482069, 0.6882212760430113, 0.08200707800586604, 0.6756579447984591, 0.902400198648146, 0.8678481056259928, 0.9898192522596165, 0.8729316621929836, 0.5910361529592892, 0.6080036216825542, 0.18330564529899318, 0.7512587797228897, 0.8129635510945189, 0.27608710663505287, 0.30149442059198384, 0.01014865391660591, 0.99
 05234106425176, 0.14611805239435627, 0.15548851460082902, 0.004404567064937415, 0.9024303659902982, 0.022854753040795428, 0.44234237127068654, 0.1634909593256464, 0.7349246156904565, 0.4991119561326317, 0.9335760226944207, 0.3185897467585901, 0.12436085235183358, 0.3925720443462196, 0.014943303650517548, 0.12280103003334919, 0.4296677592320277, 0.7882683490223884, 0.34176752596260496, 0.4579514551880063, 0.31393276219817723, 0.8990511035312476, 0.41810222727085455, 0.8357308675001134, 0.21143596058035774, 0.6487295799339329, 0.10817077545930454, 0.3213865649040908, 0.409778385070534, 0.48121696982532625, 0.44498297406207576, 0.4349609215815927, 0.28631936894059806, 0.4038872118586432, 0.613723354085566, 0.8180676754726663, 0.7039587910932086, 0.6844857558216231, 0.6564497556591257, 0.19345834800650397, 0.6933695736098979, 0.9383191415513996, 0.26983729926867295, 0.2901391402026914, 0.3271065661150001, 0.5660810965281973, 0.04914751029250364, 0.9585562180347209, 0.12558836411757213, 
 0.6325555441709875, 0.6692978594540305, 0.6583746493619407, 0.42576822659063907, 0.31256979349669245, 0.7878522278475931, 0.8656348004343625, 0.8903817020021569, 0.9729543840088963, 0.9049482603167969, 0.7165774173756946, 0.7389775651917974, 0.8078977695106498, 0.02162263912363449, 0.38218603964187414, 0.341510246484498, 0.6999683686104746, 0.7006910341221856, 0.25050436795598463, 0.5310912265659747, 0.6402079894826282, 0.05328424181642244, 0.9830425663106469, 0.36456780982362524, 0.6080840650271198, 0.24933773598735143, 0.9070954466133277, 0.08791778658453453, 0.28572120227190667, 0.3817772802240307, 0.0075526695859793325, 0.7480187791407514, 0.36071452651224367, 0.9018378910464242, 0.989789283135142, 0.23317144823367697, 0.4579610324120881, 0.3868996884525452, 0.499482172431155, 0.5218260071580214, 0.9333271542280708, 0.10241867589664055, 0.6057067283635494, 0.572281813754138, 0.8606044925272168, 0.6759762192403386, 0.08042469149650333, 0.5957001564832858, 0.548087270558081, 0.214
 16642419629572, 0.8404831227783082, 0.4205906748200481, 0.8322392442600514, 0.361985237564446, 0.15750131971376713, 0.27900193283229757, 0.21329928505434714, 0.12958143744072415, 0.8865308286681607, 0.622967481528415, 0.6213088498544588, 0.7926741403772513, 0.016131339767511244, 0.5143283929852835, 0.23092641390231616, 0.810636989621356, 0.30051317396898336, 0.7218743351321173, 0.2795273496661549, 0.5001949188899485, 0.598654716053255, 0.9068146362094198, 0.28547030494106407, 0.15287401424463654, 0.8158110936828658, 0.18091212803497003, 0.41204408736878884, 0.39079087690505177, 0.019588144637416205, 0.8811819829877734, 0.9774916110779583, 0.8180628318024356, 0.38576366911056637, 0.620878638728845, 0.2726405875161728, 0.4440659694603538, 0.11829543118048935, 0.5485640019403066, 0.9150532590165298, 0.9299628771795103, 0.3834638284597338, 0.2753703318512791, 0.9422402507654671, 0.4413678423418316, 0.5998538645798396, 0.11514176444898716, 0.7263668824417828, 0.7777118968483573, 0.133890
 52082004904, 0.12295867849413278, 0.5790121722332175, 0.49281596967267394, 0.56489743824992, 0.9146191886837004, 0.6069245912648383, 0.9213756721945658, 0.2041585451112885, 0.4029726053806997, 0.9783507498093079, 0.38250590308070564, 0.4792185793457123, 0.3965701431136499, 0.2643964194112697, 0.3441898654828778, 0.12766726403908046, 0.47191794019589584, 0.9310117247990545, 0.0014212457182173344, 0.8839092452977815, 0.6526883723666194, 0.37457551099631603, 0.546024739333262, 0.4879295858300293, 0.20756555854447611, 0.11768647818069411, 0.387584237976608, 0.39682302561300187, 0.31297140143072066, 0.9116516183393316, 0.7226044877782395, 0.5378622575164639, 0.4177389880631971, 0.5282952770025584, 0.14297642660881826, 0.8061390263529559, 0.8938698908429338, 0.9660946810880138, 0.42501211603528266, 0.7803189344988724, 0.9608331975136162, 0.6442813016577672, 0.24676062447692348, 0.30680934660742154, 0.9304334423968714, 0.9134293653785709, 0.34401116748126936, 0.1856471074091841, 0.53509546
 79424774, 0.8721862454182808, 0.534335356966387, 0.18067084241856002, 0.47448442487162024, 0.04718210133437517, 0.11371573415171488, 0.9720502392442489, 0.3436996043497963, 0.9667888624807269, 0.36175300973492186, 0.4710028757145096, 0.7908506181484176, 0.03928157978564828, 0.27913969907447556, 0.12453636515603006, 0.09264525666952428, 0.43208791900751553, 0.22503319421242485, 0.4523120789841286, 0.3336885208884963, 0.8411943135679503, 0.822156788321677, 0.0782342023940572, 0.028512647378007516, 0.1384104540290817, 0.35390215063916153, 0.9461253340626886, 0.33492790407201933, 0.889076052918665, 0.8904023736389809, 0.24185847727206988, 0.9731050035263428, 0.6439269611529589, 0.5494898019162149, 0.7835723757398738, 0.5868694481208564, 0.7262207869891241, 0.6236484768760734, 0.28710512177362857, 0.562009117171716, 0.9588543308902996, 0.785515628232662, 0.965539166405824, 0.2509663066754457, 0.4060883668607215, 0.22218322255129297, 0.3797580541192036, 0.14410580301260434, 0.740628291205
 4164, 0.34249435734530176, 0.3211734151936715, 0.7748211292746872, 0.28919165506821376, 0.6517258693993154, 0.7782886801972562, 0.049061146589503335, 0.15832613394489337, 0.4944931545654627, 0.7155232313721192, 0.24942951266898206, 0.09712530245158568, 0.6294087044877167, 0.47572921178363525, 0.17983012559154632, 0.9146212486850073, 0.017971180668839515, 0.10709044190870276, 0.7887014651694283, 0.3370164836449676, 0.7853571915216523, 0.8106011396174785, 0.1372036242905469, 0.16355063163361583, 0.0422330006172128, 0.00607964491366908, 0.7632917718069001, 0.45967938655018203, 0.14439936634983053, 0.2226803074352287, 0.3334300883667066, 0.31237679426770637, 0.011026942276001761, 0.13412447673105032, 0.9707708521546027, 0.3647546141178508, 0.3224361171601786, 0.5793794854467017, 0.801943235028288, 0.9458247699561849, 0.7748197911589512, 0.33627191851770677, 0.07416708054421584, 0.11611458597848312, 0.08200300710225039, 0.7485299266077162, 0.660226909227303, 0.3609763063507696, 0.0832789
 2783448809, 0.04600636594607754, 0.8909119742647029, 0.9808164186214441, 0.1415555822985809, 0.3433431245481895, 0.4328484040159717, 0.45110175553005727, 0.9974934057837563, 0.8540943698277979, 0.36421953984886535, 0.1548833791148011, 0.38300889003726235, 0.8153340106410506, 0.6916915370116741, 0.9267112951942074, 0.9914647708710587, 0.9820315403259233, 0.2989308030571074, 0.789196130293629, 0.8520626509190438, 0.32098671221332464, 0.7204206594541195, 0.11210665599698744, 0.7623993809870385, 0.7108726939162763, 0.19385504363149075, 0.17877513808125411, 0.19584339834709896, 0.037944370155062024, 0.16020677806374994, 0.805714548728368, 0.01933319838868175, 0.06622647499532985, 0.7526388572870917, 0.22873557062972427, 0.43239469385663776, 0.856504085773403, 0.8437472865600634, 0.9424831228966031, 0.7696202396017667, 0.3101959261759193, 0.905194171132967, 0.5601955756562486, 0.19654675402424548, 0.6221983237840961, 0.27461031568552297, 0.046622538664277235, 0.8728162291045484, 0.0410460
 6780484199, 0.42171677457399903, 0.8694195264322148, 0.8566968135209638, 0.3552438040348581, 0.45893505703270543, 0.7129712444575744, 0.8814883510856601, 0.9513795956651847, 0.45132629072024855, 0.40347751768229734, 0.14703116516822756, 0.12263969314063394, 0.04651933029067512, 0.6052688708398962, 0.43848257843979577, 0.5456024503014384, 0.5673590683986991, 0.665701964242788, 0.2097205964414397, 0.4886955400292763, 0.9883254735367569, 0.709616982791797, 0.7501798532737649, 0.3674599273459799, 0.15134504645389169, 0.7142230233209155, 0.1916200217192695, 0.7197709826445398, 0.1674361316634957, 0.8314093385958428, 0.9052721436010674, 0.5384250003285972, 0.19770013748756055, 0.6668638049629947, 0.7900039328659085, 0.169007930575232, 0.7104909108082692, 0.5740124429829447, 0.058589941070223195, 0.6577197563156355, 0.8711649368546068, 0.27567666689007975, 0.39186853449959336, 0.5168368112731606, 0.9366351535003763, 0.2874778862765812, 0.7495120381404897, 0.3647516084098589, 0.399021564835
 84206, 0.6844018637827637, 0.7013322982933404, 0.7930377857943935, 0.15101234628544924, 0.7255677414791978, 0.8744893507158779, 0.11474516310586469, 0.1301192615403256, 0.7321299887348316, 0.9996655716932014, 0.3561386334851824, 0.7398311239693697, 0.39937402674946265, 0.08321355759587368, 0.9683511606629845, 0.8427406244141119, 0.4164489479661886, 0.43248114676333993, 0.2559301891953232, 0.2740381959770234, 0.8658139000954356, 0.617962091785779, 0.4460137613712818, 0.6333392047657812, 0.37935739080250563, 0.9686165512031083, 0.6548806822630164, 0.5786286134534869, 0.43928139256665255, 0.8656506499058487, 0.7478772903987734, 0.9681800477117638, 0.5292252838579422, 0.92523955746349, 0.23423028553709535, 0.9746243226233736, 0.06473717540683166, 0.1421244343671727, 0.8900139748658082, 0.7299143166349262, 0.16208315654761807, 0.20370797748189284, 0.760547156625458, 0.15075262420160696, 0.3405176517379408, 0.4782194808179955, 0.5577534356574417, 0.34111911717285226, 0.1955212209562518, 0
 .6307152315120795, 0.2385259741617689, 0.3718554024149614, 0.9291444137519146, 0.42712518867087357, 0.7268957879947036, 0.29683411631470247, 0.37847964732048045, 0.04925278091995833, 0.3258106461600362, 0.9810606763724152, 0.8444155720476006, 0.1625793848626209, 0.

<TRUNCATED>

[04/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index ed864cc..e52449c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -11121,4 +11121,63 @@
     </test-case>
   </test-group>
   &GeoQueries;
+  <test-group name="compression">
+    <test-case FilePath="compression">
+      <compilation-unit name="incompressible-pages/large-page">
+        <output-dir compare="Text">incompressible-pages/large-page</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="compression">
+      <compilation-unit name="incompressible-pages/small-page">
+        <output-dir compare="Text">incompressible-pages/small-page</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="compression">
+      <compilation-unit name="invalid-compression-scheme">
+        <output-dir compare="Text">invalid-compression-scheme</output-dir>
+        <expected-error>ASX1096: Unknown compression scheme zip. Supported schemes are [snappy,none]</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="compression">
+      <compilation-unit name="scheme-none">
+        <output-dir compare="Text">scheme-none</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="compression">
+      <compilation-unit name="scheme-snappy">
+        <output-dir compare="Text">scheme-snappy</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="ddl-with-clause">
+    <test-case FilePath="ddl-with-clause">
+      <compilation-unit name="missing-non-optional">
+        <output-dir compare="Text">missing-non-optional</output-dir>
+        <expected-error>ASX1061: Field "merge-policy.name" in the with clause cannot be null or missing</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl-with-clause">
+      <compilation-unit name="type-mismatch">
+        <output-dir compare="Text">type-mismatch</output-dir>
+        <expected-error>ASX1060: Field "merge-policy.parameters.max-mergable-component-size" in the with clause must be of type bigint, but found string</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl-with-clause">
+      <compilation-unit name="unsupported-field">
+        <output-dir compare="Text">unsupported-field</output-dir>
+        <expected-error>ASX1059: Field(s) [unknown-field] unsupported in the with clause</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl-with-clause">
+      <compilation-unit name="unsupported-subfield">
+        <output-dir compare="Text">unsupported-subfield</output-dir>
+        <expected-error>ASX1097: Subfield(s) [unknown-subfield] in "merge-policy" unsupported in the with clause</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
index 85e44ea..6bd73e7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
@@ -22,6 +22,7 @@ import static org.apache.hyracks.control.common.config.OptionTypes.DOUBLE;
 import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
 import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
 import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
 import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 
@@ -47,7 +48,8 @@ public class StorageProperties extends AbstractProperties {
         STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS(POSITIVE_INTEGER, 2),
         STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES(POSITIVE_INTEGER, 8),
         STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE(DOUBLE, 0.01d),
-        STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(UNSIGNED_INTEGER, 8);
+        STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(UNSIGNED_INTEGER, 8),
+        STORAGE_COMPRESSION_BLOCK(STRING, "none");
 
         private final IOptionType interpreter;
         private final Object defaultValue;
@@ -88,6 +90,8 @@ public class StorageProperties extends AbstractProperties {
                     return "The maximum acceptable false positive rate for bloom filters associated with LSM indexes";
                 case STORAGE_MAX_ACTIVE_WRITABLE_DATASETS:
                     return "The maximum number of datasets that can be concurrently modified";
+                case STORAGE_COMPRESSION_BLOCK:
+                    return "The default compression scheme for the storage";
                 default:
                     throw new IllegalStateException("NYI: " + this);
             }
@@ -179,6 +183,10 @@ public class StorageProperties extends AbstractProperties {
         return accessor.getInt(Option.STORAGE_MAX_ACTIVE_WRITABLE_DATASETS);
     }
 
+    public String getCompressionScheme() {
+        return accessor.getString(Option.STORAGE_COMPRESSION_BLOCK);
+    }
+
     protected int getMetadataDatasets() {
         return MetadataIndexImmutableProperties.METADATA_DATASETS_COUNT;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
index 18e3327..81ae3e1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
@@ -21,14 +21,15 @@ package org.apache.asterix.common.dataflow;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.api.INodeJobTracker;
-import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.ExtensionProperties;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
 import org.apache.asterix.common.replication.INcLifecycleCoordinator;
+import org.apache.asterix.common.storage.ICompressionManager;
 import org.apache.asterix.common.transactions.IResourceIdManager;
+import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.job.IJobLifecycleListener;
 import org.apache.hyracks.storage.common.IStorageManager;
@@ -127,4 +128,9 @@ public interface ICcApplicationContext extends IApplicationContext {
      * @return the transaction id factory
      */
     ITxnIdFactory getTxnIdFactory();
+
+    /**
+     * @return the compression manager
+     */
+    ICompressionManager getCompressionManager();
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
----------------------------------------------------------------------
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 0bf446e..54fd65c 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
@@ -147,7 +147,7 @@ public class ErrorCode {
     public static final int UNSUPPORTED_WITH_FIELD = 1059;
     public static final int WITH_FIELD_MUST_BE_OF_TYPE = 1060;
     public static final int WITH_FIELD_MUST_CONTAIN_SUB_FIELD = 1061;
-    public static final int MERGE_POLICY_PARAMETER_INVALID_TYPE = 1062;
+    public static final int CONFIGURATION_PARAMETER_INVALID_TYPE = 1062;
     public static final int UNKNOWN_DATAVERSE = 1063;
     public static final int ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION = 1064;
     public static final int CHOSEN_INDEX_COUNT_SHOULD_BE_GREATER_THAN_ONE = 1065;
@@ -181,6 +181,8 @@ public class ErrorCode {
     public static final int COMPILATION_TRANSLATION_ERROR = 1093;
     public static final int RANGE_MAP_ERROR = 1094;
     public static final int COMPILATION_EXPECTED_FUNCTION_CALL = 1095;
+    public static final int UNKNOWN_COMPRESSION_SCHEME = 1096;
+    public static final int UNSUPPORTED_WITH_SUBFIELD = 1097;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ICompressionManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ICompressionManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ICompressionManager.java
new file mode 100644
index 0000000..b9a0baf
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ICompressionManager.java
@@ -0,0 +1,49 @@
+/*
+ * 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.common.storage;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+
+/**
+ * An interface for the compression manager which handles all the registered
+ * schemes and validates the provided configurations.
+ */
+public interface ICompressionManager {
+
+    /**
+     * Get a registered CompressorDecompressorFactory
+     *
+     * @param schemeName
+     *            Compression scheme name
+     * @return Compressor/Decompressor factory if the scheme is specified or NOOP o.w
+     * @throws CompilationException
+     */
+    ICompressorDecompressorFactory getFactory(String schemeName) throws CompilationException;
+
+    /**
+     * Get the specified compression scheme in the DDL or the default one
+     *
+     * @param ddlScheme
+     *            Compression scheme name from DDL
+     * @return DDL or default compression scheme name
+     * @throws CompilationException
+     */
+    String getDdlOrDefaultCompressionScheme(String ddlScheme) throws CompilationException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
----------------------------------------------------------------------
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 8c17ec6..7aa9b91 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -132,10 +132,10 @@
 1056 = Too many options were specified for %1$s
 1057 = Expression of type %1$s is not supported in constant record
 1058 = Literal of type %1$s is not supported in constant record
-1059 = Field \"%1$s\" unsupported in the with clause
-1060 = Field \"%1$s\" in the with clause must be of type %2$s
-1061 = Field \"%1$s\" in the with clause must contain sub field \"%2$s\"
-1062 = Merge policy parameters cannot be of type %1$s
+1059 = Field(s) %1$s unsupported in the with clause
+1060 = Field \"%1$s\" in the with clause must be of type %2$s, but found %3$s
+1061 = Field \"%1$s\" in the with clause cannot be null or missing
+1062 = Configuration parameter cannot be of type %1$s
 1063 = Cannot find dataverse with name %1$s
 1064 = An error was occurred while converting type %1$s to type %2$s.
 1065 = There should be at least two applicable indexes.
@@ -168,6 +168,8 @@
 1093 = A parser error has occurred. The detail exception: %1$s
 1094 = Cannot parse range map: %1$s
 1095 = Expected function call
+1096 = Unknown compression scheme %1$s. Supported schemes are %2$s
+1097 = Subfield(s) %1$s in \"%2$s\" unsupported in the with clause
 
 # Feed Errors
 3001 = Illegal state.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-lang-common/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/pom.xml b/asterixdb/asterix-lang-common/pom.xml
index dde41e0..bb153a5 100644
--- a/asterixdb/asterix-lang-common/pom.xml
+++ b/asterixdb/asterix-lang-common/pom.xml
@@ -107,5 +107,9 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-data-std</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
index 3d5b815..9f01b1c 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
@@ -18,19 +18,19 @@
  */
 package org.apache.asterix.lang.common.statement;
 
+import java.util.Map;
+
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.util.ConfigurationUtil;
 import org.apache.asterix.lang.common.util.ExpressionUtils;
-import org.apache.asterix.lang.common.util.MergePolicyUtils;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.object.base.AdmObjectNode;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-import java.util.Map;
-
 /**
  * The new create feed statement only concerns the feed adaptor configuration.
  * All feeds are considered as primary feeds.
@@ -76,7 +76,7 @@ public class CreateFeedStatement extends AbstractStatement {
     }
 
     public Map<String, String> getConfiguration() throws CompilationException {
-        return MergePolicyUtils.toProperties(withObjectNode);
+        return ConfigurationUtil.toProperties(withObjectNode);
     }
 
     public AdmObjectNode getWithObjectNode() {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
index 4aeb6d3..0a17b24 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
@@ -18,31 +18,22 @@
  */
 package org.apache.asterix.lang.common.statement;
 
-import java.util.Arrays;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.lang.common.util.ExpressionUtils;
-import org.apache.asterix.lang.common.util.MergePolicyUtils;
+import org.apache.asterix.lang.common.util.ConfigurationUtil;
+import org.apache.asterix.lang.common.util.DatasetDeclParametersUtil;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.object.base.AdmObjectNode;
-import org.apache.asterix.object.base.AdmStringNode;
 import org.apache.asterix.object.base.IAdmNode;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.compression.CompressionManager;
 
 public class DatasetDecl extends AbstractStatement {
-    protected static final String[] WITH_OBJECT_FIELDS = new String[] { MergePolicyUtils.MERGE_POLICY_PARAMETER_NAME };
-    protected static final Set<String> WITH_OBJECT_FIELDS_SET = new HashSet<>(Arrays.asList(WITH_OBJECT_FIELDS));
-
     protected final Identifier name;
     protected final Identifier dataverse;
     protected final Identifier itemTypeDataverse;
@@ -76,14 +67,7 @@ public class DatasetDecl extends AbstractStatement {
         }
         this.nodegroupName = nodeGroupName;
         this.hints = hints;
-        try {
-            this.withObjectNode = withRecord == null ? null : ExpressionUtils.toNode(withRecord);
-        } catch (CompilationException e) {
-            throw e;
-        } catch (AlgebricksException e) {
-            // TODO(tillw) make signatures throw Algebricks exceptions
-            throw new CompilationException(e);
-        }
+        this.withObjectNode = DatasetDeclParametersUtil.validateAndGetWithObjectNode(withRecord);
         this.ifNotExists = ifNotExists;
         this.datasetType = datasetType;
         this.datasetDetailsDecl = idd;
@@ -141,50 +125,17 @@ public class DatasetDecl extends AbstractStatement {
         return nodegroupName;
     }
 
-    public String getCompactionPolicy() throws CompilationException {
-        AdmObjectNode mergePolicy = getMergePolicyObject();
-        if (mergePolicy == null) {
-            return null;
-        }
-        IAdmNode mergePolicyName = mergePolicy.get(MergePolicyUtils.MERGE_POLICY_NAME_PARAMETER_NAME);
-        if (mergePolicyName == null) {
-            throw new CompilationException(ErrorCode.WITH_FIELD_MUST_CONTAIN_SUB_FIELD,
-                    MergePolicyUtils.MERGE_POLICY_PARAMETER_NAME, MergePolicyUtils.MERGE_POLICY_NAME_PARAMETER_NAME);
-        }
-        if (mergePolicyName.getType() != ATypeTag.STRING) {
-            throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE,
-                    MergePolicyUtils.MERGE_POLICY_PARAMETER_NAME + '.'
-                            + MergePolicyUtils.MERGE_POLICY_NAME_PARAMETER_NAME,
-                    ATypeTag.STRING);
-        }
-        return ((AdmStringNode) mergePolicyName).get();
+    private AdmObjectNode getMergePolicyObject() {
+        return (AdmObjectNode) withObjectNode.get(DatasetDeclParametersUtil.MERGE_POLICY_PARAMETER_NAME);
     }
 
-    private static AdmObjectNode validateWithObject(AdmObjectNode withObject) throws CompilationException {
-        if (withObject == null) {
-            return null;
-        }
-        for (String name : withObject.getFieldNames()) {
-            if (!WITH_OBJECT_FIELDS_SET.contains(name)) {
-                throw new CompilationException(ErrorCode.UNSUPPORTED_WITH_FIELD, name);
-            }
-        }
-        return withObject;
-    }
-
-    private AdmObjectNode getMergePolicyObject() throws CompilationException {
-        if (withObjectNode == null) {
-            return null;
-        }
-        IAdmNode mergePolicy = validateWithObject(withObjectNode).get(MergePolicyUtils.MERGE_POLICY_PARAMETER_NAME);
+    public String getCompactionPolicy() {
+        AdmObjectNode mergePolicy = getMergePolicyObject();
         if (mergePolicy == null) {
             return null;
         }
-        if (!mergePolicy.isObject()) {
-            throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE,
-                    MergePolicyUtils.MERGE_POLICY_PARAMETER_NAME, ATypeTag.OBJECT);
-        }
-        return (AdmObjectNode) mergePolicy;
+
+        return mergePolicy.getOptionalString(DatasetDeclParametersUtil.MERGE_POLICY_NAME_PARAMETER_NAME);
     }
 
     public Map<String, String> getCompactionPolicyProperties() throws CompilationException {
@@ -192,17 +143,26 @@ public class DatasetDecl extends AbstractStatement {
         if (mergePolicy == null) {
             return null;
         }
-        IAdmNode mergePolicyParameters = mergePolicy.get(MergePolicyUtils.MERGE_POLICY_PARAMETERS_PARAMETER_NAME);
+        IAdmNode mergePolicyParameters =
+                mergePolicy.get(DatasetDeclParametersUtil.MERGE_POLICY_PARAMETERS_PARAMETER_NAME);
         if (mergePolicyParameters == null) {
             return null;
         }
-        if (mergePolicyParameters.getType() != ATypeTag.OBJECT) {
-            throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE,
-                    MergePolicyUtils.MERGE_POLICY_PARAMETER_NAME + '.'
-                            + MergePolicyUtils.MERGE_POLICY_PARAMETERS_PARAMETER_NAME,
-                    ATypeTag.OBJECT);
+        return ConfigurationUtil.toProperties((AdmObjectNode) mergePolicyParameters);
+    }
+
+    public String getDatasetCompressionScheme() {
+        if (datasetType != DatasetType.INTERNAL) {
+            return CompressionManager.NONE;
+        }
+
+        final AdmObjectNode storageBlockCompression =
+                (AdmObjectNode) withObjectNode.get(DatasetDeclParametersUtil.STORAGE_BLOCK_COMPRESSION_PARAMETER_NAME);
+        if (storageBlockCompression == null) {
+            return null;
         }
-        return MergePolicyUtils.toProperties((AdmObjectNode) mergePolicyParameters);
+        return storageBlockCompression
+                .getOptionalString(DatasetDeclParametersUtil.STORAGE_BLOCK_COMPRESSION_SCHEME_PARAMETER_NAME);
     }
 
     public Map<String, String> getHints() {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ConfigurationTypeValidator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ConfigurationTypeValidator.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ConfigurationTypeValidator.java
new file mode 100644
index 0000000..4d0baeb
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ConfigurationTypeValidator.java
@@ -0,0 +1,191 @@
+/*
+ * 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.lang.common.util;
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.object.base.AdmArrayNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.tuple.MutablePair;
+
+class ConfigurationTypeValidator {
+    //Error information
+    private final Set<String> unknownFieldNames;
+    private final Deque<String> path;
+    private final MutablePair<ATypeTag, ATypeTag> expectedActualTypePair;
+    private ErrorType result;
+
+    public enum ErrorType {
+        UNKNOWN_FIELD_NAMES,
+        TYPE_MISMATCH,
+        MISSING_UNOPTIONAL_FIELD
+    }
+
+    protected ConfigurationTypeValidator() {
+        unknownFieldNames = new HashSet<>();
+        path = new ArrayDeque<>();
+        expectedActualTypePair = new MutablePair<>(null, null);
+    }
+
+    public void validateType(IAType type, IAdmNode node) throws CompilationException {
+        if (!validate(type, node)) {
+            throwException();
+        }
+    }
+
+    private boolean validate(IAType type, IAdmNode node) {
+        if (type.getTypeTag().isDerivedType()) {
+            return validateDerivedType(type, node);
+        } else if (node == null) {
+            result = ErrorType.MISSING_UNOPTIONAL_FIELD;
+            return false;
+        } else if (type.getTypeTag() != node.getType()) {
+            setExpectedAndActualType(type.getTypeTag(), node.getType());
+            return false;
+        }
+
+        return true;
+    }
+
+    private boolean validateDerivedType(IAType type, IAdmNode node) {
+        final ATypeTag typeTag = type.getTypeTag();
+        switch (typeTag) {
+            case UNION:
+                return validateUnionType(type, node);
+            case OBJECT:
+                return validateObject(type, node);
+            case ARRAY:
+                return validateArray(type, node);
+            default:
+                throw new IllegalStateException("Unsupported derived type: " + typeTag);
+        }
+    }
+
+    private boolean validateUnionType(IAType type, IAdmNode node) {
+        if (node == null || node.getType() == ATypeTag.NULL) {
+            return true;
+        }
+        return validate(((AUnionType) type).getActualType(), node);
+    }
+
+    private boolean validateObject(IAType type, IAdmNode node) {
+        if (node.getType() != ATypeTag.OBJECT) {
+            setExpectedAndActualType(ATypeTag.OBJECT, node.getType());
+            return false;
+        }
+
+        final ARecordType recordType = (ARecordType) type;
+        final AdmObjectNode objectNode = (AdmObjectNode) node;
+
+        final String[] fieldNames = recordType.getFieldNames();
+
+        //Check field names
+        final Set<String> definedFieldNames = new HashSet<>(Arrays.asList(fieldNames));
+        final Set<String> objectFieldNames = objectNode.getFieldNames();
+        if (!definedFieldNames.containsAll(objectFieldNames)) {
+            setUnknownFieldNames(definedFieldNames, objectFieldNames);
+            return false;
+        }
+
+        final IAType[] fieldTypes = recordType.getFieldTypes();
+
+        for (int i = 0; i < fieldTypes.length; i++) {
+            if (!validate(fieldTypes[i], objectNode.get(fieldNames[i]))) {
+                addToPath(fieldNames[i]);
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private boolean validateArray(IAType type, IAdmNode node) {
+        if (node.getType() != ATypeTag.ARRAY) {
+            setExpectedAndActualType(ATypeTag.ARRAY, node.getType());
+            return false;
+        }
+
+        final IAType itemType = ((AOrderedListType) type).getItemType();
+        final AdmArrayNode array = (AdmArrayNode) node;
+        for (int i = 0; i < array.size(); i++) {
+            if (!validate(itemType, array.get(i))) {
+                addToPath(i);
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private void setUnknownFieldNames(Set<String> definedFieldNames, Set<String> objectFieldNames) {
+        unknownFieldNames.addAll(objectFieldNames);
+        unknownFieldNames.removeAll(definedFieldNames);
+        result = ErrorType.UNKNOWN_FIELD_NAMES;
+    }
+
+    private void setExpectedAndActualType(ATypeTag expectedTypeTag, ATypeTag actualTypeTag) {
+        expectedActualTypePair.left = expectedTypeTag;
+        expectedActualTypePair.right = actualTypeTag;
+        result = ErrorType.TYPE_MISMATCH;
+    }
+
+    private void addToPath(String fieldName) {
+        if (path.isEmpty()) {
+            path.push(fieldName);
+        } else {
+            path.push(fieldName + ".");
+        }
+    }
+
+    private void addToPath(int arrayIndex) {
+        path.push("[" + arrayIndex + "]");
+    }
+
+    private void throwException() throws CompilationException {
+        final StringBuilder pathBuilder = new StringBuilder();
+        while (!path.isEmpty()) {
+            pathBuilder.append(path.pop());
+        }
+        switch (result) {
+            case UNKNOWN_FIELD_NAMES:
+                if (pathBuilder.length() > 0) {
+                    throw new CompilationException(ErrorCode.UNSUPPORTED_WITH_SUBFIELD, unknownFieldNames.toString(),
+                            pathBuilder.toString());
+                }
+                throw new CompilationException(ErrorCode.UNSUPPORTED_WITH_FIELD, unknownFieldNames.toString());
+            case TYPE_MISMATCH:
+                throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, pathBuilder.toString(),
+                        expectedActualTypePair.left, expectedActualTypePair.right);
+            case MISSING_UNOPTIONAL_FIELD:
+                throw new CompilationException(ErrorCode.WITH_FIELD_MUST_CONTAIN_SUB_FIELD, pathBuilder.toString());
+
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ConfigurationUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ConfigurationUtil.java
new file mode 100644
index 0000000..4bb799b
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ConfigurationUtil.java
@@ -0,0 +1,74 @@
+/*
+ * 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.lang.common.util;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+
+public class ConfigurationUtil {
+
+    private ConfigurationUtil() {
+    }
+
+    /**
+     * Convert the parameters object to a Map<String,String>
+     * This method should go away once we store the with object as it is in storage
+     *
+     * @param parameters
+     *            the parameters passed for the merge policy in the with clause
+     * @return the parameters as a map
+     */
+    public static Map<String, String> toProperties(AdmObjectNode parameters) throws CompilationException {
+        Map<String, String> map = new HashMap<>();
+        for (Entry<String, IAdmNode> field : parameters.getFields()) {
+            IAdmNode value = field.getValue();
+            map.put(field.getKey(), getStringValue(value));
+        }
+        return map;
+    }
+
+    /**
+     * Get string value of {@link IAdmNode}
+     *
+     * @param value
+     *            IAdmNode value should be of type integer or string
+     * @return
+     *         string value of <code>value</code>
+     * @throws CompilationException
+     */
+    public static String getStringValue(IAdmNode value) throws CompilationException {
+        switch (value.getType()) {
+            case BOOLEAN:
+            case DOUBLE:
+            case BIGINT:
+                return value.toString();
+            case STRING:
+                return ((AdmStringNode) value).get();
+            default:
+                throw new CompilationException(ErrorCode.CONFIGURATION_PARAMETER_INVALID_TYPE, value.getType());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
new file mode 100644
index 0000000..effe4b8
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
@@ -0,0 +1,97 @@
+/*
+ * 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.lang.common.util;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+public class DatasetDeclParametersUtil {
+    /* ***********************************************
+     * Merge Policy Parameters
+     * ***********************************************
+     */
+    public static final String MERGE_POLICY_PARAMETER_NAME = "merge-policy";
+    public static final String MERGE_POLICY_NAME_PARAMETER_NAME = "name";
+    public static final String MERGE_POLICY_PARAMETERS_PARAMETER_NAME = "parameters";
+    public static final String MERGE_POLICY_MERGABLE_SIZE_PARAMETER_NAME = "max-mergable-component-size";
+    public static final String MERGE_POLICY_TOLERANCE_COUNT_PARAMETER_NAME = "max-tolerance-component-count";
+    public static final String MERGE_POLICY_NUMBER_COMPONENTS_PARAMETER_NAME = "num-components";
+
+    /* ***********************************************
+     * Storage Block Compression Parameters
+     * ***********************************************
+     */
+    public static final String STORAGE_BLOCK_COMPRESSION_PARAMETER_NAME = "storage-block-compression";
+    public static final String STORAGE_BLOCK_COMPRESSION_SCHEME_PARAMETER_NAME = "scheme";
+
+    /* ***********************************************
+     * Private members
+     * ***********************************************
+     */
+    private static final ARecordType WITH_OBJECT_TYPE = getWithObjectType();
+    private static final AdmObjectNode EMPTY_WITH_OBJECT = new AdmObjectNode();
+
+    private DatasetDeclParametersUtil() {
+    }
+
+    public static AdmObjectNode validateAndGetWithObjectNode(RecordConstructor withRecord) throws CompilationException {
+        if (withRecord == null) {
+            return EMPTY_WITH_OBJECT;
+        }
+        final ConfigurationTypeValidator validator = new ConfigurationTypeValidator();
+        final AdmObjectNode node = ExpressionUtils.toNode(withRecord);
+        validator.validateType(WITH_OBJECT_TYPE, node);
+        return node;
+    }
+
+    private static ARecordType getWithObjectType() {
+        final String[] withNames = { MERGE_POLICY_PARAMETER_NAME, STORAGE_BLOCK_COMPRESSION_PARAMETER_NAME };
+        final IAType[] withTypes = { AUnionType.createUnknownableType(getMergePolicyType()),
+                AUnionType.createUnknownableType(getStorageBlockCompressionType()) };
+        return new ARecordType("withObject", withNames, withTypes, false);
+    }
+
+    private static ARecordType getMergePolicyType() {
+        //merge-policy.parameters
+        final String[] parameterNames = { MERGE_POLICY_MERGABLE_SIZE_PARAMETER_NAME,
+                MERGE_POLICY_TOLERANCE_COUNT_PARAMETER_NAME, MERGE_POLICY_NUMBER_COMPONENTS_PARAMETER_NAME };
+        final IAType[] parametersTypes = { AUnionType.createUnknownableType(BuiltinType.AINT64),
+                AUnionType.createUnknownableType(BuiltinType.AINT64),
+                AUnionType.createUnknownableType(BuiltinType.AINT64) };
+        final ARecordType parameters =
+                new ARecordType(MERGE_POLICY_PARAMETERS_PARAMETER_NAME, parameterNames, parametersTypes, false);
+
+        //merge-policy
+        final String[] mergePolicyNames = { MERGE_POLICY_NAME_PARAMETER_NAME, MERGE_POLICY_PARAMETERS_PARAMETER_NAME };
+        final IAType[] mergePolicyTypes = { BuiltinType.ASTRING, AUnionType.createUnknownableType(parameters) };
+
+        return new ARecordType(MERGE_POLICY_PARAMETER_NAME, mergePolicyNames, mergePolicyTypes, false);
+    }
+
+    private static ARecordType getStorageBlockCompressionType() {
+        final String[] schemeName = { STORAGE_BLOCK_COMPRESSION_SCHEME_PARAMETER_NAME };
+        final IAType[] schemeType = { BuiltinType.ASTRING };
+        return new ARecordType(STORAGE_BLOCK_COMPRESSION_PARAMETER_NAME, schemeName, schemeType, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
index 25f9d07..6adb050 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
@@ -39,13 +39,12 @@ import org.apache.asterix.object.base.AdmNullNode;
 import org.apache.asterix.object.base.AdmObjectNode;
 import org.apache.asterix.object.base.AdmStringNode;
 import org.apache.asterix.object.base.IAdmNode;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public class ExpressionUtils {
     private ExpressionUtils() {
     }
 
-    public static IAdmNode toNode(Expression expr) throws AlgebricksException {
+    public static IAdmNode toNode(Expression expr) throws CompilationException {
         switch (expr.getKind()) {
             case LIST_CONSTRUCTOR_EXPRESSION:
                 return toNode((ListConstructor) expr);
@@ -58,7 +57,7 @@ public class ExpressionUtils {
         }
     }
 
-    public static AdmObjectNode toNode(RecordConstructor recordConstructor) throws AlgebricksException {
+    public static AdmObjectNode toNode(RecordConstructor recordConstructor) throws CompilationException {
         AdmObjectNode node = new AdmObjectNode();
         final List<FieldBinding> fbList = recordConstructor.getFbList();
         for (int i = 0; i < fbList.size(); i++) {
@@ -70,7 +69,7 @@ public class ExpressionUtils {
         return node;
     }
 
-    private static IAdmNode toNode(ListConstructor listConstructor) throws AlgebricksException {
+    private static IAdmNode toNode(ListConstructor listConstructor) throws CompilationException {
         final List<Expression> exprList = listConstructor.getExprList();
         AdmArrayNode array = new AdmArrayNode(exprList.size());
         for (int i = 0; i < exprList.size(); i++) {
@@ -79,7 +78,7 @@ public class ExpressionUtils {
         return array;
     }
 
-    private static IAdmNode toNode(LiteralExpr literalExpr) throws AlgebricksException {
+    private static IAdmNode toNode(LiteralExpr literalExpr) throws CompilationException {
         final Literal value = literalExpr.getValue();
         final Literal.Type literalType = value.getLiteralType();
         switch (literalType) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/MergePolicyUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/MergePolicyUtils.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/MergePolicyUtils.java
deleted file mode 100644
index 6bb5c36..0000000
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/MergePolicyUtils.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.lang.common.util;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.object.base.AdmObjectNode;
-import org.apache.asterix.object.base.AdmStringNode;
-import org.apache.asterix.object.base.IAdmNode;
-
-public class MergePolicyUtils {
-    public static final String MERGE_POLICY_PARAMETER_NAME = "merge-policy";
-    public static final String MERGE_POLICY_NAME_PARAMETER_NAME = "name";
-    public static final String MERGE_POLICY_PARAMETERS_PARAMETER_NAME = "parameters";
-
-    private MergePolicyUtils() {
-    }
-
-    /**
-     * Convert the parameters object to a Map<String,String>
-     * This method should go away once we store the with object as it is in storage
-     *
-     * @param parameters
-     *            the parameters passed for the merge policy in the with clause
-     * @return the parameters as a map
-     */
-    public static Map<String, String> toProperties(AdmObjectNode parameters) throws CompilationException {
-        Map<String, String> map = new HashMap<>();
-        for (Entry<String, IAdmNode> field : parameters.getFields()) {
-            IAdmNode value = field.getValue();
-            switch (value.getType()) {
-                case BOOLEAN:
-                case DOUBLE:
-                case BIGINT:
-                    map.put(field.getKey(), value.toString());
-                    break;
-                case STRING:
-                    map.put(field.getKey(), ((AdmStringNode) value).get());
-                    break;
-                default:
-                    throw new CompilationException(ErrorCode.MERGE_POLICY_PARAMETER_INVALID_TYPE, value.getType());
-            }
-        }
-        return map;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 95526f4..2380ab6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -87,6 +87,7 @@ import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
 import org.apache.hyracks.storage.common.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.LocalResource;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -338,7 +339,8 @@ public class MetadataBootstrap {
                     new AsterixVirtualBufferCacheProvider(datasetId),
                     storageComponentProvider.getIoOperationSchedulerProvider(),
                     appContext.getMetadataMergePolicyFactory(), GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, true,
-                    bloomFilterKeyFields, bloomFilterFalsePositiveRate, true, null);
+                    bloomFilterKeyFields, bloomFilterFalsePositiveRate, true, null,
+                    NoOpCompressorDecompressorFactory.INSTANCE);
             DatasetLocalResourceFactory dsLocalResourceFactory =
                     new DatasetLocalResourceFactory(datasetId, lsmBtreeFactory);
             // TODO(amoudi) Creating the index should be done through the same code path as

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index ba1ea03..d9309d9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -163,6 +163,11 @@ public final class MetadataRecordTypes {
     public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 11;
     public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 12;
     public static final int DATASET_ARECORD_PENDINGOP_FIELD_INDEX = 13;
+    //Optional open fields
+    public static final String DATASET_ARECORD_BLOCK_LEVEL_STORAGE_COMPRESSION_FIELD_NAME =
+            "BlockLevelStorageCompression";
+    public static final String DATASET_ARECORD_DATASET_COMPRESSION_SCHEME_FIELD_NAME = "DatasetCompressionScheme";
+    public static final String DATASET_ARECORD_REBALANCE_FIELD_NAME = "rebalanceCount";
     public static final ARecordType DATASET_RECORDTYPE = createRecordType(
             // RecordTypeName
             RECORD_NAME_DATASET,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java
index f5cfbb3..62a03ad 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java
@@ -31,6 +31,9 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
  */
 public class DatasetHints {
 
+    private DatasetHints() {
+    }
+
     /**
      * validate the use of a hint
      *

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 301aafb..2a1d551 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -38,6 +38,7 @@ import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
@@ -50,6 +51,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IResourceFactory;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
 
@@ -93,11 +95,20 @@ public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
             case INTERNAL:
                 AsterixVirtualBufferCacheProvider vbcProvider =
                         new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
+
+                final ICompressorDecompressorFactory compDecompFactory;
+                if (index.isPrimaryIndex()) {
+                    //Compress only primary index
+                    compDecompFactory = mdProvider.getCompressionManager().getFactory(dataset.getCompressionScheme());
+                } else {
+                    compDecompFactory = NoOpCompressorDecompressorFactory.INSTANCE;
+                }
+
                 return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                         filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
                         metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory,
                         mergePolicyProperties, true, bloomFilterFields, bloomFilterFalsePositiveRate,
-                        index.isPrimaryIndex(), btreeFields);
+                        index.isPrimaryIndex(), btreeFields, compDecompFactory);
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
                         dataset.getDatasetType().toString());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index e212d11..85beb95 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -37,6 +37,7 @@ import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.metadata.LockList;
+import org.apache.asterix.common.storage.ICompressionManager;
 import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
@@ -1616,4 +1617,8 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
     public ITxnIdFactory getTxnIdFactory() {
         return appCtx.getTxnIdFactory();
     }
+
+    public ICompressionManager getCompressionManager() {
+        return appCtx.getCompressionManager();
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index a25ed20..855cf78 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -25,8 +25,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.stream.IntStream;
 
-import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.asterix.active.IActiveNotificationHandler;
 import org.apache.asterix.common.api.IDatasetInfoProvider;
 import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -65,7 +63,7 @@ import org.apache.asterix.metadata.utils.RTreeResourceFactoryProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.asterix.transaction.management.opcallbacks.LockThenSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
@@ -149,6 +147,7 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
     private final String metaTypeName;
     private final long rebalanceCount;
     private int pendingOp;
+    private final String compressionScheme;
 
     public Dataset(String dataverseName, String datasetName, String recordTypeDataverseName, String recordTypeName,
             String nodeGroupName, String compactionPolicy, Map<String, String> compactionPolicyProperties,
@@ -156,29 +155,30 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
             int pendingOp) {
         this(dataverseName, datasetName, recordTypeDataverseName, recordTypeName, /*metaTypeDataverseName*/null,
                 /*metaTypeName*/null, nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails,
-                hints, datasetType, datasetId, pendingOp);
+                hints, datasetType, datasetId, pendingOp, CompressionManager.NONE);
     }
 
     public Dataset(String dataverseName, String datasetName, String itemTypeDataverseName, String itemTypeName,
             String metaItemTypeDataverseName, String metaItemTypeName, String nodeGroupName, String compactionPolicy,
             Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails, Map<String, String> hints,
-            DatasetType datasetType, int datasetId, int pendingOp) {
+            DatasetType datasetType, int datasetId, int pendingOp, String compressionScheme) {
         this(dataverseName, datasetName, itemTypeDataverseName, itemTypeName, metaItemTypeDataverseName,
                 metaItemTypeName, nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails, hints,
-                datasetType, datasetId, pendingOp, 0L);
+                datasetType, datasetId, pendingOp, 0L, compressionScheme);
     }
 
     public Dataset(Dataset dataset) {
         this(dataset.dataverseName, dataset.datasetName, dataset.recordTypeDataverseName, dataset.recordTypeName,
                 dataset.metaTypeDataverseName, dataset.metaTypeName, dataset.nodeGroupName,
                 dataset.compactionPolicyFactory, dataset.compactionPolicyProperties, dataset.datasetDetails,
-                dataset.hints, dataset.datasetType, dataset.datasetId, dataset.pendingOp, dataset.rebalanceCount);
+                dataset.hints, dataset.datasetType, dataset.datasetId, dataset.pendingOp, dataset.rebalanceCount,
+                dataset.compressionScheme);
     }
 
     public Dataset(String dataverseName, String datasetName, String itemTypeDataverseName, String itemTypeName,
             String metaItemTypeDataverseName, String metaItemTypeName, String nodeGroupName, String compactionPolicy,
             Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails, Map<String, String> hints,
-            DatasetType datasetType, int datasetId, int pendingOp, long rebalanceCount) {
+            DatasetType datasetType, int datasetId, int pendingOp, long rebalanceCount, String compressionScheme) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.recordTypeName = itemTypeName;
@@ -194,6 +194,7 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
         this.pendingOp = pendingOp;
         this.hints = hints;
         this.rebalanceCount = rebalanceCount;
+        this.compressionScheme = compressionScheme;
     }
 
     @Override
@@ -357,7 +358,7 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
                     new Dataset(dataverseName, datasetName, getItemTypeDataverseName(), getItemTypeName(),
                             getMetaItemTypeDataverseName(), getMetaItemTypeName(), getNodeGroupName(),
                             getCompactionPolicy(), getCompactionPolicyProperties(), getDatasetDetails(), getHints(),
-                            getDatasetType(), getDatasetId(), MetadataUtil.PENDING_DROP_OP));
+                            getDatasetType(), getDatasetId(), MetadataUtil.PENDING_DROP_OP, getCompressionScheme()));
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
             bActiveTxn.setValue(false);
@@ -644,6 +645,8 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
         tree.put("metaTypeDataverseName", metaTypeDataverseName);
         tree.put("metaTypeName", metaTypeName);
         tree.put("pendingOp", MetadataUtil.pendingOpToString(pendingOp));
+        tree.put("rebalanceCount", rebalanceCount);
+        tree.put("compressionScheme", compressionScheme);
         return tree;
     }
 
@@ -823,7 +826,8 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
         return new Dataset(this.dataverseName, this.datasetName, this.recordTypeDataverseName, this.recordTypeName,
                 this.metaTypeDataverseName, this.metaTypeName, targetNodeGroupName, this.compactionPolicyFactory,
                 this.compactionPolicyProperties, this.datasetDetails, this.hints, this.datasetType,
-                DatasetIdFactory.generateAlternatingDatasetId(this.datasetId), this.pendingOp, this.rebalanceCount + 1);
+                DatasetIdFactory.generateAlternatingDatasetId(this.datasetId), this.pendingOp, this.rebalanceCount + 1,
+                this.compressionScheme);
     }
 
     // Gets an array of partition numbers for this dataset.
@@ -840,4 +844,8 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
     public String getFullyQualifiedName() {
         return dataverseName + '.' + datasetName;
     }
+
+    public String getCompressionScheme() {
+        return compressionScheme;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index d5e179b..27978ab 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -60,10 +60,13 @@ import org.apache.asterix.om.base.ARecord;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.AUnorderedList;
 import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -77,7 +80,6 @@ public class DatasetTupleTranslator extends AbstractTupleTranslator<Dataset> {
     private static final long serialVersionUID = 1L;
     // Payload field containing serialized Dataset.
     public static final int DATASET_PAYLOAD_TUPLE_FIELD_INDEX = 2;
-    private static final String REBALANCE_ID_FIELD_NAME = "rebalanceCount";
 
     @SuppressWarnings("unchecked")
     protected final ISerializerDeserializer<ARecord> recordSerDes =
@@ -256,14 +258,33 @@ public class DatasetTupleTranslator extends AbstractTupleTranslator<Dataset> {
             metaTypeName = ((AString) datasetRecord.getValueByPos(metaTypeNameIndex)).getStringValue();
         }
 
-        // Read the rebalance count if there is one.
-        int rebalanceCountIndex = datasetRecord.getType().getFieldIndex(REBALANCE_ID_FIELD_NAME);
-        long rebalanceCount = rebalanceCountIndex >= 0
-                ? ((AInt64) datasetRecord.getValueByPos(rebalanceCountIndex)).getLongValue() : 0;
+        long rebalanceCount = getRebalanceCount(datasetRecord);
+        String compressionScheme = getCompressionScheme(datasetRecord);
 
         return new Dataset(dataverseName, datasetName, typeDataverseName, typeName, metaTypeDataverseName, metaTypeName,
                 nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails, hints, datasetType,
-                datasetId, pendingOp, rebalanceCount);
+                datasetId, pendingOp, rebalanceCount, compressionScheme);
+    }
+
+    private long getRebalanceCount(ARecord datasetRecord) {
+        // Read the rebalance count if there is one.
+        int rebalanceCountIndex =
+                datasetRecord.getType().getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_REBALANCE_FIELD_NAME);
+        return rebalanceCountIndex >= 0 ? ((AInt64) datasetRecord.getValueByPos(rebalanceCountIndex)).getLongValue()
+                : 0;
+    }
+
+    private String getCompressionScheme(ARecord datasetRecord) {
+        final ARecordType datasetType = datasetRecord.getType();
+        final int compressionIndex = datasetType
+                .getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_BLOCK_LEVEL_STORAGE_COMPRESSION_FIELD_NAME);
+        if (compressionIndex >= 0) {
+            final ARecordType compressionType = (ARecordType) datasetType.getFieldTypes()[compressionIndex];
+            final int schemeIndex = compressionType
+                    .getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_DATASET_COMPRESSION_SCHEME_FIELD_NAME);
+            return ((AString) datasetRecord.getValueByPos(schemeIndex)).getStringValue();
+        }
+        return CompressionManager.NONE;
     }
 
     @Override
@@ -392,8 +413,19 @@ public class DatasetTupleTranslator extends AbstractTupleTranslator<Dataset> {
         return tuple;
     }
 
+    /**
+     * Keep protected to allow other extensions to add additional fields
+     *
+     * @param dataset
+     * @throws HyracksDataException
+     */
     protected void writeOpenFields(Dataset dataset) throws HyracksDataException {
-        // write open fields
+        writeMetaPart(dataset);
+        writeRebalanceCount(dataset);
+        writeBlockLevelStorageCompression(dataset);
+    }
+
+    private void writeMetaPart(Dataset dataset) throws HyracksDataException {
         if (dataset.hasMetaPart()) {
             // write open field 1, the meta item type Dataverse name.
             fieldName.reset();
@@ -413,10 +445,35 @@ public class DatasetTupleTranslator extends AbstractTupleTranslator<Dataset> {
             stringSerde.serialize(aString, fieldValue.getDataOutput());
             recordBuilder.addField(fieldName, fieldValue);
         }
+    }
+
+    private void writeBlockLevelStorageCompression(Dataset dataset) throws HyracksDataException {
+        if (CompressionManager.NONE.equals(dataset.getCompressionScheme())) {
+            return;
+        }
+        RecordBuilder compressionObject = new RecordBuilder();
+        compressionObject.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        fieldName.reset();
+        aString.setValue(MetadataRecordTypes.DATASET_ARECORD_DATASET_COMPRESSION_SCHEME_FIELD_NAME);
+        stringSerde.serialize(aString, fieldName.getDataOutput());
+        fieldValue.reset();
+        aString.setValue(dataset.getCompressionScheme());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        compressionObject.addField(fieldName, fieldValue);
+
+        fieldName.reset();
+        aString.setValue(MetadataRecordTypes.DATASET_ARECORD_BLOCK_LEVEL_STORAGE_COMPRESSION_FIELD_NAME);
+        stringSerde.serialize(aString, fieldName.getDataOutput());
+        fieldValue.reset();
+        compressionObject.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(fieldName, fieldValue);
+    }
+
+    private void writeRebalanceCount(Dataset dataset) throws HyracksDataException {
         if (dataset.getRebalanceCount() > 0) {
             // Adds the field rebalanceCount.
             fieldName.reset();
-            aString.setValue("rebalanceCount");
+            aString.setValue(MetadataRecordTypes.DATASET_ARECORD_REBALANCE_FIELD_NAME);
             stringSerde.serialize(aString, fieldName.getDataOutput());
             fieldValue.reset();
             aBigInt.setValue(dataset.getRebalanceCount());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
index b87ef2a..902ee41 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
@@ -24,12 +24,12 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
 import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.junit.Assert;
@@ -51,9 +51,9 @@ public class DatasetTupleTranslatorTest {
                     indicator == null ? null : Collections.singletonList(indicator),
                     Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList());
 
-            Dataset dataset =
-                    new Dataset("test", "log", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES", "prefix",
-                            compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0);
+            Dataset dataset = new Dataset("test", "log", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES",
+                    "prefix", compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0,
+                    CompressionManager.NONE);
             DatasetTupleTranslator dtTranslator = new DatasetTupleTranslator(true);
             ITupleReference tuple = dtTranslator.getTupleFromMetadataEntity(dataset);
             Dataset deserializedDataset = dtTranslator.getMetadataEntityFromTuple(tuple);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index ab4229c..7080dee 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -30,7 +30,6 @@ import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Datatype;
@@ -41,6 +40,7 @@ import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningS
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.junit.Assert;
@@ -62,9 +62,9 @@ public class IndexTupleTranslatorTest {
                     indicator == null ? null : Collections.singletonList(indicator),
                     Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList());
 
-            Dataset dataset =
-                    new Dataset("test", "d1", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES", "prefix",
-                            compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0);
+            Dataset dataset = new Dataset("test", "d1", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES",
+                    "prefix", compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0,
+                    CompressionManager.NONE);
 
             Index index = new Index("test", "d1", "i1", IndexType.BTREE,
                     Collections.singletonList(Collections.singletonList("row_id")),

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
index 99b7176..6d2658b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
@@ -153,6 +153,14 @@ public class AdmObjectNode implements IAdmNode {
         return getString(this, field);
     }
 
+    public String getOptionalString(String field) {
+        final IAdmNode node = get(field);
+        if (node == null) {
+            return null;
+        }
+        return ((AdmStringNode) node).get();
+    }
+
     public static String getString(AdmObjectNode openFields, String field) throws HyracksDataException {
         IAdmNode node = openFields.get(field);
         if (node == null) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
new file mode 100644
index 0000000..3bffa9a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
@@ -0,0 +1,128 @@
+/*
+ * 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.runtime.compression;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.asterix.common.config.StorageProperties;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.storage.ICompressionManager;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
+import org.apache.hyracks.storage.common.compression.SnappyCompressorDecompressorFactory;
+
+public class CompressionManager implements ICompressionManager {
+    private static final Map<String, Class<? extends ICompressorDecompressorFactory>> REGISTERED_SCHEMES =
+            getRegisteredSchemes();
+    public static final String NONE = "none";
+    private final String defaultScheme;
+
+    /*
+     * New compression schemes can be added by registering the name and the factory class
+     *
+     * WARNING: Changing scheme name will breakdown storage back compatibility. Before upgrading to a newer
+     * version of the registered schemes, make sure it is also back-compatible with the previous version.
+     */
+    private static Map<String, Class<? extends ICompressorDecompressorFactory>> getRegisteredSchemes() {
+        final Map<String, Class<? extends ICompressorDecompressorFactory>> registeredSchemes = new HashMap<>();
+        //No compression
+        registeredSchemes.put(NONE, NoOpCompressorDecompressorFactory.class);
+        registeredSchemes.put("snappy", SnappyCompressorDecompressorFactory.class);
+        return registeredSchemes;
+    }
+
+    public CompressionManager(StorageProperties storageProperties) {
+        validateCompressionConfiguration(storageProperties);
+        defaultScheme = storageProperties.getCompressionScheme();
+    }
+
+    @Override
+    public ICompressorDecompressorFactory getFactory(String schemeName) throws CompilationException {
+        final String scheme = getDdlOrDefaultCompressionScheme(schemeName);
+        Class<? extends ICompressorDecompressorFactory> clazz = REGISTERED_SCHEMES.get(scheme);
+        try {
+            return clazz.newInstance();
+        } catch (IllegalAccessException | InstantiationException e) {
+            throw new IllegalStateException("Failed to instantiate compressor/decompressor: " + scheme, e);
+        }
+    }
+
+    @Override
+    public String getDdlOrDefaultCompressionScheme(String ddlScheme) throws CompilationException {
+        if (ddlScheme != null && !isRegisteredScheme(ddlScheme)) {
+            throw new CompilationException(ErrorCode.UNKNOWN_COMPRESSION_SCHEME, ddlScheme, formatSupportedValues());
+        }
+
+        return ddlScheme != null ? ddlScheme : defaultScheme;
+    }
+
+    /**
+     * Register factory classes for persisted resources
+     *
+     * @param registeredClasses
+     */
+    public static void registerCompressorDecompressorsFactoryClasses(
+            Map<String, Class<? extends IJsonSerializable>> registeredClasses) {
+        for (Class<? extends ICompressorDecompressorFactory> clazz : REGISTERED_SCHEMES.values()) {
+            registeredClasses.put(clazz.getSimpleName(), clazz);
+        }
+    }
+
+    /**
+     * @param schemeName
+     *            Compression scheme name
+     * @return
+     *         true if it is registered
+     */
+    private boolean isRegisteredScheme(String schemeName) {
+        return schemeName != null && REGISTERED_SCHEMES.containsKey(schemeName.toLowerCase());
+    }
+
+    /**
+     * Validate the configuration of StorageProperties
+     *
+     * @param storageProperties
+     */
+    private void validateCompressionConfiguration(StorageProperties storageProperties) {
+        if (!isRegisteredScheme(storageProperties.getCompressionScheme())) {
+            final String option = StorageProperties.Option.STORAGE_COMPRESSION_BLOCK.ini();
+            final String value = storageProperties.getCompressionScheme();
+            throw new IllegalStateException("Invalid compression configuration (" + option + " = " + value
+                    + "). Valid values are: " + formatSupportedValues());
+        }
+
+    }
+
+    private String formatSupportedValues() {
+        final StringBuilder schemes = new StringBuilder();
+        final Iterator<String> iterator = REGISTERED_SCHEMES.keySet().iterator();
+        schemes.append('[');
+        schemes.append(iterator.next());
+        while (iterator.hasNext()) {
+            schemes.append(',');
+            schemes.append(iterator.next());
+        }
+        schemes.append(']');
+        return schemes.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
index 4157e16..0d2a1df 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
@@ -24,7 +24,6 @@ import java.util.function.Supplier;
 import org.apache.asterix.common.api.ICoordinationService;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.api.INodeJobTracker;
-import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.ActiveProperties;
@@ -44,7 +43,10 @@ import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
 import org.apache.asterix.common.replication.INcLifecycleCoordinator;
+import org.apache.asterix.common.storage.ICompressionManager;
 import org.apache.asterix.common.transactions.IResourceIdManager;
+import org.apache.asterix.common.transactions.ITxnIdFactory;
+import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.asterix.runtime.job.listener.NodeJobTracker;
 import org.apache.asterix.runtime.transaction.ResourceIdManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -87,6 +89,7 @@ public class CcApplicationContext implements ICcApplicationContext {
     private IClusterStateManager clusterStateManager;
     private final INodeJobTracker nodeJobTracker;
     private final ITxnIdFactory txnIdFactory;
+    private final ICompressionManager compressionManager;
 
     public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
             ILibraryManager libraryManager, Supplier<IMetadataBootstrap> metadataBootstrapSupplier,
@@ -121,6 +124,7 @@ public class CcApplicationContext implements ICcApplicationContext {
         this.resourceIdManager = new ResourceIdManager(clusterStateManager);
         nodeJobTracker = new NodeJobTracker();
         txnIdFactory = new BulkTxnIdFactory();
+        compressionManager = new CompressionManager(storageProperties);
 
     }
 
@@ -270,7 +274,13 @@ public class CcApplicationContext implements ICcApplicationContext {
         return NoOpCoordinationService.INSTANCE;
     }
 
+    @Override
     public ITxnIdFactory getTxnIdFactory() {
         return txnIdFactory;
     }
+
+    @Override
+    public ICompressionManager getCompressionManager() {
+        return compressionManager;
+    }
 }


[05/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.13.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.13.query.sqlpp
new file mode 100644
index 0000000..cd777cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.13.query.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 tpch;
+
+select value count(*) from LineItem;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.14.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.14.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.14.sto.cmd
new file mode 100644
index 0000000..1e192f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.14.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc2 /removeReplica 2 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.15.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.15.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.15.sto.cmd
new file mode 100644
index 0000000..530432f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.15.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc2 /removeReplica 3 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.2.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.2.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.2.sto.cmd
new file mode 100644
index 0000000..f3810f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.2.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc2 /addReplica 3 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.3.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.3.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.3.pollget.http
new file mode 100644
index 0000000..4ea16d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.3.pollget.http
@@ -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.
+ */
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.4.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.4.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.4.pollget.http
new file mode 100644
index 0000000..22558bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.4.pollget.http
@@ -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.
+ */
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.5.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.5.ddl.sqlpp
new file mode 100644
index 0000000..5a7e48b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.5.ddl.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * 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 tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create dataset LineItem(LineItemType)
+primary key l_orderkey,l_linenumber
+with {"storage-block-compression": {"scheme": "snappy"}};
+
+create feed LineItemFeed with {
+  "adapter-name": "localfs",
+  "type-name": "LineItemType",
+  "path": "asterix_nc1://data/tpch0.001/lineitem.tbl",
+  "format": "delimited-text",
+  "delimiter": "|"
+};
+
+connect feed LineItemFeed to dataset LineItem;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.6.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.6.update.sqlpp
new file mode 100644
index 0000000..7e2dbd5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.6.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 tpch;
+
+start feed LineItemFeed;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.7.sleep.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.7.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.7.sleep.sqlpp
new file mode 100644
index 0000000..5f299bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.7.sleep.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+5000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.8.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.8.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.8.sto.cmd
new file mode 100644
index 0000000..389cf68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.8.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc1 /promote 2 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.9.sto.cmd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.9.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.9.sto.cmd
new file mode 100644
index 0000000..257f26a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.9.sto.cmd
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+nc:asterix_nc1 /promote 3 asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml b/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
index ba084a0..ebafab8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
@@ -77,5 +77,26 @@ STATUS LINE: HTTP/1.1 400 Bad Request
 ERROR_BODY: {"results":"to rebalance a particular dataset, the parameter dataverseName must be given"}</expected-error>
       </compilation-unit>
     </test-case>
+    <!-- Rebalance with compression -->
+    <test-case FilePath="rebalance">
+      <compilation-unit name="single_dataset_compressed">
+        <output-dir compare="Text">single_dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="rebalance">
+      <compilation-unit name="single_dataset_with_index_compressed">
+        <output-dir compare="Text">single_dataset_with_index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="rebalance">
+      <compilation-unit name="single_dataverse_compressed">
+        <output-dir compare="Text">single_dataverse</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="rebalance">
+      <compilation-unit name="all_datasets_compressed">
+        <output-dir compare="Text">all_datasets</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml b/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
index 469446d..da2842f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
@@ -49,5 +49,16 @@
         <output-dir compare="Text">release_partition</output-dir>
       </compilation-unit>
     </test-case>
+    <!-- Replication with compression -->
+    <test-case FilePath="replication">
+      <compilation-unit name="flushed_component_compressed">
+        <output-dir compare="Text">bulkload</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="replication">
+      <compilation-unit name="bulkload_with_compression">
+        <output-dir compare="Text">bulkload</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/incompressible-pages/large-page/large-page.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/incompressible-pages/large-page/large-page.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/incompressible-pages/large-page/large-page.1.adm
new file mode 100644
index 0000000..864c7c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/incompressible-pages/large-page/large-page.1.adm
@@ -0,0 +1 @@
+409600

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/incompressible-pages/small-page/small-page.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/incompressible-pages/small-page/small-page.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/incompressible-pages/small-page/small-page.1.adm
new file mode 100644
index 0000000..9e923e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/incompressible-pages/small-page/small-page.1.adm
@@ -0,0 +1,2 @@
+182191
+182191

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/scheme-none/scheme-none.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/scheme-none/scheme-none.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/scheme-none/scheme-none.1.adm
new file mode 100644
index 0000000..12984c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/scheme-none/scheme-none.1.adm
@@ -0,0 +1,100 @@
+{ "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }
+{ "id": 2, "dblpid": "books/acm/kim95/Blakeley95", "title": "OQL[C++]  Extending C++ with an Object Query Capability.", "authors": "José A. Blakeley", "misc": "2002-01-03 69-88 Modern Database Systems db/books/collections/kim95.html#Blakeley95 1995" }
+{ "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }
+{ "id": 6, "dblpid": "books/acm/kim95/DittrichD95", "title": "Where Object-Oriented DBMSs Should Do Better  A Critique Based on Early Experiences.", "authors": "Angelika Kotz Dittrich Klaus R. Dittrich", "misc": "2002-01-03 238-254 1995 Modern Database Systems db/books/collections/kim95.html#DittrichD95" }
+{ "id": 7, "dblpid": "books/acm/kim95/Garcia-MolinaH95", "title": "Distributed Databases.", "authors": "Hector Garcia-Molina Meichun Hsu", "misc": "2002-01-03 477-493 1995 Modern Database Systems db/books/collections/kim95.html#Garcia-MolinaH95" }
+{ "id": 8, "dblpid": "books/acm/kim95/Goodman95", "title": "An Object-Oriented DBMS War Story  Developing a Genome Mapping Database in C++.", "authors": "Nathan Goodman", "misc": "2002-01-03 216-237 1995 Modern Database Systems db/books/collections/kim95.html#Goodman95" }
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
+{ "id": 10, "dblpid": "books/acm/kim95/KelleyGKRG95", "title": "Schema Architecture of the UniSQL/M Multidatabase System", "authors": "William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham", "misc": "2004-03-08 Modern Database Systems books/acm/Kim95 621-648 1995 db/books/collections/kim95.html#KelleyGKRG95" }
+{ "id": 11, "dblpid": "books/acm/kim95/KemperM95", "title": "Physical Object Management.", "authors": "Alfons Kemper Guido Moerkotte", "misc": "2002-01-03 175-202 1995 Modern Database Systems db/books/collections/kim95.html#KemperM95" }
+{ "id": 12, "dblpid": "books/acm/kim95/Kim95", "title": "Introduction to Part 1  Next-Generation Database Technology.", "authors": "Won Kim", "misc": "2002-01-03 5-17 1995 Modern Database Systems db/books/collections/kim95.html#Kim95" }
+{ "id": 13, "dblpid": "books/acm/kim95/Kim95a", "title": "Object-Oriented Database Systems  Promises, Reality, and Future.", "authors": "Won Kim", "misc": "2002-01-03 255-280 1995 Modern Database Systems db/books/collections/kim95.html#Kim95a" }
+{ "id": 14, "dblpid": "books/acm/kim95/Kim95b", "title": "Introduction to Part 2  Technology for Interoperating Legacy Databases.", "authors": "Won Kim", "misc": "2002-01-03 515-520 1995 Modern Database Systems db/books/collections/kim95.html#Kim95b" }
+{ "id": 15, "dblpid": "books/acm/kim95/KimCGS95", "title": "On Resolving Schematic Heterogeneity in Multidatabase Systems.", "authors": "Won Kim Injun Choi Sunit K. Gala Mark Scheevel", "misc": "2002-01-03 521-550 1995 Modern Database Systems db/books/collections/kim95.html#KimCGS95" }
+{ "id": 16, "dblpid": "books/acm/kim95/KimG95", "title": "Requirements for a Performance Benchmark for Object-Oriented Database Systems.", "authors": "Won Kim Jorge F. Garza", "misc": "2002-01-03 203-215 1995 Modern Database Systems db/books/collections/kim95.html#KimG95" }
+{ "id": 17, "dblpid": "books/acm/kim95/KimK95", "title": "On View Support in Object-Oriented Databases Systems.", "authors": "Won Kim William Kelley", "misc": "2002-01-03 108-129 1995 Modern Database Systems db/books/collections/kim95.html#KimK95" }
+{ "id": 18, "dblpid": "books/acm/kim95/Kowalski95", "title": "The POSC Solution to Managing E&P Data.", "authors": "Vincent J. Kowalski", "misc": "2002-01-03 281-301 1995 Modern Database Systems db/books/collections/kim95.html#Kowalski95" }
+{ "id": 19, "dblpid": "books/acm/kim95/KriegerA95", "title": "C++ Bindings to an Object Database.", "authors": "David Krieger Tim Andrews", "misc": "2002-01-03 89-107 1995 Modern Database Systems db/books/collections/kim95.html#KriegerA95" }
+{ "id": 20, "dblpid": "books/acm/kim95/Lunt95", "title": "Authorization in Object-Oriented Databases.", "authors": "Teresa F. Lunt", "misc": "2002-01-03 130-145 1995 Modern Database Systems db/books/collections/kim95.html#Lunt95" }
+{ "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
+{ "id": 23, "dblpid": "books/acm/kim95/Omiecinski95", "title": "Parallel Relational Database Systems.", "authors": "Edward Omiecinski", "misc": "2002-01-03 494-512 1995 Modern Database Systems db/books/collections/kim95.html#Omiecinski95" }
+{ "id": 24, "dblpid": "books/acm/kim95/OzsuB95", "title": "Query Processing in Object-Oriented Database Systems.", "authors": "M. Tamer Özsu José A. Blakeley", "misc": "2002-01-03 146-174 1995 Modern Database Systems db/books/collections/kim95.html#OzsuB95" }
+{ "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }
+{ "id": 26, "dblpid": "books/acm/kim95/Samet95", "title": "Spatial Data Structures.", "authors": "Hanan Samet", "misc": "2004-03-08 361-385 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#Samet95 1995" }
+{ "id": 27, "dblpid": "books/acm/kim95/SametA95", "title": "Spatial Data Models and Query Processing.", "authors": "Hanan Samet Walid G. Aref", "misc": "2002-01-03 338-360 1995 Modern Database Systems db/books/collections/kim95.html#SametA95" }
+{ "id": 28, "dblpid": "books/acm/kim95/ShanADDK95", "title": "Pegasus  A Heterogeneous Information Management System.", "authors": "Ming-Chien Shan Rafi Ahmed Jim Davis Weimin Du William Kent", "misc": "2004-03-08 664-682 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#ShanADDK95 1995" }
+{ "id": 29, "dblpid": "books/acm/kim95/Snodgrass95", "title": "Temporal Object-Oriented Databases  A Critical Comparison.", "authors": "Richard T. Snodgrass", "misc": "2002-01-03 386-408 1995 Modern Database Systems db/books/collections/kim95.html#Snodgrass95" }
+{ "id": 30, "dblpid": "books/acm/kim95/SoleyK95", "title": "The OMG Object Model.", "authors": "Richard Mark Soley William Kent", "misc": "2002-01-03 18-41 1995 Modern Database Systems db/books/collections/kim95.html#SoleyK95" }
+{ "id": 31, "dblpid": "books/acm/kim95/Stout95", "title": "EDA/SQL.", "authors": "Ralph L. Stout", "misc": "2004-03-08 649-663 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#Stout95 1995" }
+{ "id": 32, "dblpid": "books/acm/kim95/Thompson95", "title": "The Changing Database Standards Landscape.", "authors": "Craig W. Thompson", "misc": "2002-01-03 302-317 1995 Modern Database Systems db/books/collections/kim95.html#Thompson95" }
+{ "id": 33, "dblpid": "books/acm/kim95/BreitbartR95", "title": "Overview of the ADDS System.", "authors": "Yuri Breitbart Tom C. Reyes", "misc": "2009-06-12 683-701 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartR95 1995" }
+{ "id": 34, "dblpid": "books/acm/Kim95", "title": "Modern Database Systems  The Object Model, Interoperability, and Beyond.", "authors": "", "misc": "2004-03-08 Won Kim Modern Database Systems ACM Press and Addison-Wesley 1995 0-201-59098-0 db/books/collections/kim95.html" }
+{ "id": 35, "dblpid": "books/ap/MarshallO79", "title": "Inequalities  Theory of Majorization and Its Application.", "authors": "Albert W. Marshall Ingram Olkin", "misc": "2002-01-03 Academic Press 1979 0-12-473750-1" }
+{ "id": 36, "dblpid": "books/aw/kimL89/BjornerstedtH89", "title": "Version Control in an Object-Oriented Architecture.", "authors": "Anders Björnerstedt Christer Hulten", "misc": "2006-02-24 451-485 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#BjornerstedtH89" }
+{ "id": 37, "dblpid": "books/aw/kimL89/BretlMOPSSWW89", "title": "The GemStone Data Management System.", "authors": "Robert Bretl David Maier Allen Otis D. Jason Penney Bruce Schuchardt Jacob Stein E. Harold Williams Monty Williams", "misc": "2002-01-03 283-308 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#BretlMOPSSWW89" }
+{ "id": 38, "dblpid": "books/aw/kimL89/CareyDRS89", "title": "Storage Management in EXODUS.", "authors": "Michael J. Carey David J. DeWitt Joel E. Richardson Eugene J. Shekita", "misc": "2002-01-03 341-369 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#CareyDRS89" }
+{ "id": 39, "dblpid": "books/aw/kimL89/Decouchant89", "title": "A Distributed Object Manager for the Smalltalk-80 System.", "authors": "Dominique Decouchant", "misc": "2002-01-03 487-520 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Decouchant89" }
+{ "id": 40, "dblpid": "books/aw/kimL89/DiederichM89", "title": "Objects, Messages, and Rules in Database Design.", "authors": "Jim Diederich Jack Milton", "misc": "2002-01-03 177-197 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#DiederichM89" }
+{ "id": 41, "dblpid": "books/aw/kimL89/EllisG89", "title": "Active Objects  Ealities and Possibilities.", "authors": "Clarence A. Ellis Simon J. Gibbs", "misc": "2002-01-03 561-572 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#EllisG89" }
+{ "id": 42, "dblpid": "books/aw/kimL89/FishmanABCCDHHKLLMNRSW89", "title": "Overview of the Iris DBMS.", "authors": "Daniel H. Fishman Jurgen Annevelink David Beech E. C. Chow Tim Connors J. W. Davis Waqar Hasan C. G. Hoch William Kent S. Leichner Peter Lyngbæk Brom Mahbod Marie-Anne Neimat Tore Risch Ming-Chien Shan W. Kevin Wilkinson", "misc": "2002-01-03 219-250 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#FishmanABCCDHHKLLMNRSW89" }
+{ "id": 43, "dblpid": "books/aw/kimL89/KimBCGW89", "title": "Features of the ORION Object-Oriented Database System.", "authors": "Won Kim Nat Ballou Hong-Tai Chou Jorge F. Garza Darrell Woelk", "misc": "2002-01-03 251-282 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimBCGW89" }
+{ "id": 44, "dblpid": "books/aw/kimL89/KimKD89", "title": "Indexing Techniques for Object-Oriented Databases.", "authors": "Won Kim Kyung-Chang Kim Alfred G. Dale", "misc": "2002-01-03 371-394 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimKD89" }
+{ "id": 45, "dblpid": "books/aw/kimL89/King89", "title": "My Cat Is Object-Oriented.", "authors": "Roger King", "misc": "2002-01-03 23-30 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#King89" }
+{ "id": 46, "dblpid": "books/aw/kimL89/Maier89", "title": "Making Database Systems Fast Enough for CAD Applications.", "authors": "David Maier", "misc": "2002-01-03 573-582 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Maier89" }
+{ "id": 47, "dblpid": "books/aw/kimL89/MellenderRS89", "title": "Optimizing Smalltalk Message Performance.", "authors": "Fred Mellender Steve Riegel Andrew Straw", "misc": "2002-01-03 423-450 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#MellenderRS89" }
+{ "id": 48, "dblpid": "books/aw/kimL89/Moon89", "title": "The Common List Object-Oriented Programming Language Standard.", "authors": "David A. Moon", "misc": "2002-01-03 49-78 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Moon89" }
+{ "id": 49, "dblpid": "books/aw/kimL89/Moss89", "title": "Object Orientation as Catalyst for Language-Database Inegration.", "authors": "J. Eliot B. Moss", "misc": "2002-01-03 583-592 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Moss89" }
+{ "id": 50, "dblpid": "books/aw/kimL89/Nierstrasz89", "title": "A Survey of Object-Oriented Concepts.", "authors": "Oscar Nierstrasz", "misc": "2002-01-03 3-21 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Nierstrasz89" }
+{ "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }
+{ "id": 52, "dblpid": "books/aw/kimL89/Russinoff89", "title": "Proteus  A Frame-Based Nonmonotonic Inference System.", "authors": "David M. Russinoff", "misc": "2002-01-03 127-150 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#Russinoff89" }
+{ "id": 53, "dblpid": "books/aw/kimL89/SkarraZ89", "title": "Concurrency Control and Object-Oriented Databases.", "authors": "Andrea H. Skarra Stanley B. Zdonik", "misc": "2002-01-03 395-421 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#SkarraZ89" }
+{ "id": 54, "dblpid": "books/aw/kimL89/SteinLU89", "title": "A Shared View of Sharing  The Treaty of Orlando.", "authors": "Lynn Andrea Stein Henry Lieberman David Ungar", "misc": "2002-01-03 31-48 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#SteinLU89" }
+{ "id": 55, "dblpid": "books/aw/kimL89/TarltonT89", "title": "Pogo  A Declarative Representation System for Graphics.", "authors": "Mark A. Tarlton P. Nong Tarlton", "misc": "2002-01-03 151-176 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TarltonT89" }
+{ "id": 56, "dblpid": "books/aw/kimL89/TomlinsonS89", "title": "Concurrent Object-Oriented Programming Languages.", "authors": "Chris Tomlinson Mark Scheevel", "misc": "2002-01-03 79-124 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TomlinsonS89" }
+{ "id": 57, "dblpid": "books/aw/kimL89/TsichritzisN89", "title": "Directions in Object-Oriented Research.", "authors": "Dennis Tsichritzis Oscar Nierstrasz", "misc": "2002-01-03 523-536 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TsichritzisN89" }
+{ "id": 58, "dblpid": "books/aw/kimL89/Wand89", "title": "A Proposal for a Formal Model of Objects.", "authors": "Yair Wand", "misc": "2002-01-03 537-559 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Wand89" }
+{ "id": 59, "dblpid": "books/aw/kimL89/WeiserL89", "title": "OZ+  An Object-Oriented Database System.", "authors": "Stephen P. Weiser Frederick H. Lochovsky", "misc": "2002-01-03 309-337 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#WeiserL89" }
+{ "id": 60, "dblpid": "books/aw/stonebraker86/RoweS86", "title": "The Commercial INGRES Epilogue.", "authors": "Lawrence A. Rowe Michael Stonebraker", "misc": "2002-01-03 63-82 1986 The INGRES Papers db/books/collections/Stonebraker86.html#RoweS86 db/books/collections/Stonebraker86/RoweS86.html ingres/P063.pdf" }
+{ "id": 61, "dblpid": "books/aw/stonebraker86/Stonebraker86", "title": "Design of Relational Systems (Introduction to Section 1).", "authors": "Michael Stonebraker", "misc": "2002-01-03 1-3 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86 db/books/collections/Stonebraker86/Stonebraker86.html ingres/P001.pdf" }
+{ "id": 62, "dblpid": "books/aw/stonebraker86/Stonebraker86a", "title": "Supporting Studies on Relational Systems (Introduction to Section 2).", "authors": "Michael Stonebraker", "misc": "2002-01-03 83-85 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86a db/books/collections/Stonebraker86/Stonebraker86a.html ingres/P083.pdf" }
+{ "id": 63, "dblpid": "books/aw/stonebraker86/Stonebraker86b", "title": "Distributed Database Systems (Introduction to Section 3).", "authors": "Michael Stonebraker", "misc": "2002-01-03 183-186 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86b db/books/collections/Stonebraker86/Stonebraker86b.html ingres/P183.pdf" }
+{ "id": 64, "dblpid": "books/aw/stonebraker86/Stonebraker86c", "title": "The Design and Implementation of Distributed INGRES.", "authors": "Michael Stonebraker", "misc": "2002-01-03 187-196 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86c db/books/collections/Stonebraker86/Stonebraker86c.html ingres/P187.pdf" }
+{ "id": 65, "dblpid": "books/aw/stonebraker86/Stonebraker86d", "title": "User Interfaces for Database Systems (Introduction to Section 4).", "authors": "Michael Stonebraker", "misc": "2002-01-03 243-245 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86d db/books/collections/Stonebraker86/Stonebraker86d.html ingres/P243.pdf" }
+{ "id": 66, "dblpid": "books/aw/stonebraker86/Stonebraker86e", "title": "Extended Semantics for the Relational Model (Introduction to Section 5).", "authors": "Michael Stonebraker", "misc": "2002-01-03 313-316 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86e db/books/collections/Stonebraker86/Stonebraker86e.html ingres/P313.pdf" }
+{ "id": 67, "dblpid": "books/aw/stonebraker86/Stonebraker86f", "title": "Database Design (Introduction to Section 6).", "authors": "Michael Stonebraker", "misc": "2002-01-03 393-394 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86f db/books/collections/Stonebraker86/Stonebraker86f.html ingres/P393.pdf" }
+{ "id": 68, "dblpid": "books/aw/stonebraker86/X86", "title": "Title, Preface, Contents.", "authors": "", "misc": "2002-01-03 1986 The INGRES Papers db/books/collections/Stonebraker86.html#X86 db/books/collections/Stonebraker86/X86.html ingres/frontmatter.pdf" }
+{ "id": 69, "dblpid": "books/aw/stonebraker86/X86a", "title": "References.", "authors": "", "misc": "2002-01-03 429-444 1986 The INGRES Papers db/books/collections/Stonebraker86.html#X86a db/books/collections/Stonebraker86/X86a.html ingres/P429.pdf" }
+{ "id": 70, "dblpid": "books/aw/Knuth86a", "title": "TeX  The Program", "authors": "Donald E. Knuth", "misc": "2002-01-03 Addison-Wesley 1986 0-201-13437-3" }
+{ "id": 71, "dblpid": "conf/appt/LiDCMY07", "title": "Design and Implementation of a High-Speed Reconfigurable Modular Arithmetic Unit.", "authors": "Wei Li Zibin Dai Tao Chen Tao Meng Xuan Yang", "misc": "2007-11-09 50-59 2007 conf/appt/2007 APPT http //dx.doi.org/10.1007/978-3-540-76837-1_9 db/conf/appt/appt2007.html#LiDCMY07" }
+{ "id": 72, "dblpid": "books/aw/Lamport86", "title": "LaTeX  User's Guide & Reference Manual", "authors": "Leslie Lamport", "misc": "2002-01-03 Addison-Wesley 1986 0-201-15790-X" }
+{ "id": 73, "dblpid": "books/aw/AhoHU74", "title": "The Design and Analysis of Computer Algorithms.", "authors": "Alfred V. Aho John E. Hopcroft Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1974 0-201-00029-6" }
+{ "id": 74, "dblpid": "books/aw/Lamport2002", "title": "Specifying Systems, The TLA+ Language and Tools for Hardware and Software Engineers", "authors": "Leslie Lamport", "misc": "2005-07-28 Addison-Wesley 2002 0-3211-4306-X http //research.microsoft.com/users/lamport/tla/book.html" }
+{ "id": 75, "dblpid": "books/aw/AhoHU83", "title": "Data Structures and Algorithms.", "authors": "Alfred V. Aho John E. Hopcroft Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1983 0-201-00023-7" }
+{ "id": 76, "dblpid": "books/aw/LewisBK01", "title": "Databases and Transaction Processing  An Application-Oriented Approach", "authors": "Philip M. Lewis Arthur J. Bernstein Michael Kifer", "misc": "2002-01-03 Addison-Wesley 2001 0-201-70872-8" }
+{ "id": 77, "dblpid": "books/aw/AhoKW88", "title": "The AWK Programming Language", "authors": "Alfred V. Aho Brian W. Kernighan Peter J. Weinberger", "misc": "2002-01-03 Addison-Wesley 1988" }
+{ "id": 78, "dblpid": "books/aw/LindholmY97", "title": "The Java Virtual Machine Specification", "authors": "Tim Lindholm Frank Yellin", "misc": "2002-01-28 Addison-Wesley 1997 0-201-63452-X" }
+{ "id": 79, "dblpid": "books/aw/AhoSU86", "title": "Compilers  Princiles, Techniques, and Tools.", "authors": "Alfred V. Aho Ravi Sethi Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1986 0-201-10088-6" }
+{ "id": 80, "dblpid": "books/aw/Sedgewick83", "title": "Algorithms", "authors": "Robert Sedgewick", "misc": "2002-01-03 Addison-Wesley 1983 0-201-06672-6" }
+{ "id": 81, "dblpid": "journals/siamcomp/AspnesW96", "title": "Randomized Consensus in Expected O(n log² n) Operations Per Processor.", "authors": "James Aspnes Orli Waarts", "misc": "2002-01-03 1024-1044 1996 25 SIAM J. Comput. 5 db/journals/siamcomp/siamcomp25.html#AspnesW96" }
+{ "id": 82, "dblpid": "conf/focs/AspnesW92", "title": "Randomized Consensus in Expected O(n log ^2 n) Operations Per Processor", "authors": "James Aspnes Orli Waarts", "misc": "2006-04-25 137-146 conf/focs/FOCS33 1992 FOCS db/conf/focs/focs92.html#AspnesW92" }
+{ "id": 83, "dblpid": "journals/siamcomp/Bloniarz83", "title": "A Shortest-Path Algorithm with Expected Time O(n² log n log* n).", "authors": "Peter A. Bloniarz", "misc": "2002-01-03 588-600 1983 12 SIAM J. Comput. 3 db/journals/siamcomp/siamcomp12.html#Bloniarz83" }
+{ "id": 84, "dblpid": "conf/stoc/Bloniarz80", "title": "A Shortest-Path Algorithm with Expected Time O(n^2 log n log ^* n)", "authors": "Peter A. Bloniarz", "misc": "2006-04-25 378-384 conf/stoc/STOC12 1980 STOC db/conf/stoc/stoc80.html#Bloniarz80" }
+{ "id": 85, "dblpid": "journals/siamcomp/Megiddo83a", "title": "Linear-Time Algorithms for Linear Programming in R³ and Related Problems.", "authors": "Nimrod Megiddo", "misc": "2002-01-03 759-776 1983 12 SIAM J. Comput. 4 db/journals/siamcomp/siamcomp12.html#Megiddo83a" }
+{ "id": 86, "dblpid": "conf/focs/Megiddo82", "title": "Linear-Time Algorithms for Linear Programming in R^3 and Related Problems", "authors": "Nimrod Megiddo", "misc": "2006-04-25 329-338 conf/focs/FOCS23 1982 FOCS db/conf/focs/focs82.html#Megiddo82" }
+{ "id": 87, "dblpid": "journals/siamcomp/MoffatT87", "title": "An All Pairs Shortest Path Algorithm with Expected Time O(n² log n).", "authors": "Alistair Moffat Tadao Takaoka", "misc": "2002-01-03 1023-1031 1987 16 SIAM J. Comput. 6 db/journals/siamcomp/siamcomp16.html#MoffatT87" }
+{ "id": 88, "dblpid": "conf/focs/MoffatT85", "title": "An All Pairs Shortest Path Algorithm with Expected Running Time O(n^2 log n)", "authors": "Alistair Moffat Tadao Takaoka", "misc": "2006-04-25 101-105 conf/focs/FOCS26 1985 FOCS db/conf/focs/focs85.html#MoffatT85" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
+{ "id": 91, "dblpid": "journals/corr/abs-0802-2861", "title": "Geometric Set Cover and Hitting Sets for Polytopes in $R^3$", "authors": "Sören Laue", "misc": "2008-03-03 http //arxiv.org/abs/0802.2861 2008 CoRR abs/0802.2861 db/journals/corr/corr0802.html#abs-0802-2861 informal publication" }
+{ "id": 92, "dblpid": "conf/stacs/Laue08", "title": "Geometric Set Cover and Hitting Sets for Polytopes in R³.", "authors": "Sören Laue", "misc": "2008-03-04 2008 STACS 479-490 http //drops.dagstuhl.de/opus/volltexte/2008/1367 conf/stacs/2008 db/conf/stacs/stacs2008.html#Laue08" }
+{ "id": 93, "dblpid": "journals/iandc/IbarraJCR91", "title": "Some Classes of Languages in NC¹", "authors": "Oscar H. Ibarra Tao Jiang Jik H. Chang Bala Ravikumar", "misc": "2006-04-25 86-106 Inf. Comput. January 1991 90 1 db/journals/iandc/iandc90.html#IbarraJCR91" }
+{ "id": 94, "dblpid": "conf/awoc/IbarraJRC88", "title": "On Some Languages in NC.", "authors": "Oscar H. Ibarra Tao Jiang Bala Ravikumar Jik H. Chang", "misc": "2002-08-06 64-73 1988 conf/awoc/1988 AWOC db/conf/awoc/awoc88.html#IbarraJRC88" }
+{ "id": 95, "dblpid": "journals/jacm/GalilHLSW87", "title": "An O(n³log n) deterministic and an O(n³) Las Vegs isomorphism test for trivalent graphs.", "authors": "Zvi Galil Christoph M. Hoffmann Eugene M. Luks Claus-Peter Schnorr Andreas Weber", "misc": "2003-11-20 513-531 1987 34 J. ACM 3 http //doi.acm.org/10.1145/28869.28870 db/journals/jacm/jacm34.html#GalilHLSW87" }
+{ "id": 96, "dblpid": "conf/focs/GalilHLSW82", "title": "An O(n^3 log n) Deterministic and an O(n^3) Probabilistic Isomorphism Test for Trivalent Graphs", "authors": "Zvi Galil Christoph M. Hoffmann Eugene M. Luks Claus-Peter Schnorr Andreas Weber", "misc": "2006-04-25 118-125 conf/focs/FOCS23 1982 FOCS db/conf/focs/focs82.html#GalilHLSW82" }
+{ "id": 97, "dblpid": "journals/jacm/GalilT88", "title": "An O(n²(m + n log n)log n) min-cost flow algorithm.", "authors": "Zvi Galil Éva Tardos", "misc": "2003-11-20 374-386 1988 35 J. ACM 2 http //doi.acm.org/10.1145/42282.214090 db/journals/jacm/jacm35.html#GalilT88" }
+{ "id": 98, "dblpid": "conf/focs/GalilT86", "title": "An O(n^2 (m + n log n) log n) Min-Cost Flow Algorithm", "authors": "Zvi Galil Éva Tardos", "misc": "2006-04-25 1-9 conf/focs/FOCS27 1986 FOCS db/conf/focs/focs86.html#GalilT86" }
+{ "id": 99, "dblpid": "series/synthesis/2009Weintraub", "title": "Jordan Canonical Form  Theory and Practice", "authors": "Steven H. Weintraub", "misc": "2009-09-06 Jordan Canonical Form  Theory and Practice http //dx.doi.org/10.2200/S00218ED1V01Y200908MAS006 http //dx.doi.org/10.2200/S00218ED1V01Y200908MAS006 2009 Synthesis Lectures on Mathematics & Statistics Morgan & Claypool Publishers" }
+{ "id": 100, "dblpid": "series/synthesis/2009Brozos", "title": "The Geometry of Walker Manifolds", "authors": "Miguel Brozos-Vázquez Eduardo García-Río Peter Gilkey Stana Nikcevic Rámon Vázquez-Lorenzo", "misc": "2009-09-06 The Geometry of Walker Manifolds http //dx.doi.org/10.2200/S00197ED1V01Y200906MAS005 http //dx.doi.org/10.2200/S00197ED1V01Y200906MAS005 2009 Synthesis Lectures on Mathematics & Statistics Morgan & Claypool Publishers" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/scheme-snappy/scheme-snappy.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/scheme-snappy/scheme-snappy.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/scheme-snappy/scheme-snappy.1.adm
new file mode 100644
index 0000000..12984c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/compression/scheme-snappy/scheme-snappy.1.adm
@@ -0,0 +1,100 @@
+{ "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }
+{ "id": 2, "dblpid": "books/acm/kim95/Blakeley95", "title": "OQL[C++]  Extending C++ with an Object Query Capability.", "authors": "José A. Blakeley", "misc": "2002-01-03 69-88 Modern Database Systems db/books/collections/kim95.html#Blakeley95 1995" }
+{ "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }
+{ "id": 6, "dblpid": "books/acm/kim95/DittrichD95", "title": "Where Object-Oriented DBMSs Should Do Better  A Critique Based on Early Experiences.", "authors": "Angelika Kotz Dittrich Klaus R. Dittrich", "misc": "2002-01-03 238-254 1995 Modern Database Systems db/books/collections/kim95.html#DittrichD95" }
+{ "id": 7, "dblpid": "books/acm/kim95/Garcia-MolinaH95", "title": "Distributed Databases.", "authors": "Hector Garcia-Molina Meichun Hsu", "misc": "2002-01-03 477-493 1995 Modern Database Systems db/books/collections/kim95.html#Garcia-MolinaH95" }
+{ "id": 8, "dblpid": "books/acm/kim95/Goodman95", "title": "An Object-Oriented DBMS War Story  Developing a Genome Mapping Database in C++.", "authors": "Nathan Goodman", "misc": "2002-01-03 216-237 1995 Modern Database Systems db/books/collections/kim95.html#Goodman95" }
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
+{ "id": 10, "dblpid": "books/acm/kim95/KelleyGKRG95", "title": "Schema Architecture of the UniSQL/M Multidatabase System", "authors": "William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham", "misc": "2004-03-08 Modern Database Systems books/acm/Kim95 621-648 1995 db/books/collections/kim95.html#KelleyGKRG95" }
+{ "id": 11, "dblpid": "books/acm/kim95/KemperM95", "title": "Physical Object Management.", "authors": "Alfons Kemper Guido Moerkotte", "misc": "2002-01-03 175-202 1995 Modern Database Systems db/books/collections/kim95.html#KemperM95" }
+{ "id": 12, "dblpid": "books/acm/kim95/Kim95", "title": "Introduction to Part 1  Next-Generation Database Technology.", "authors": "Won Kim", "misc": "2002-01-03 5-17 1995 Modern Database Systems db/books/collections/kim95.html#Kim95" }
+{ "id": 13, "dblpid": "books/acm/kim95/Kim95a", "title": "Object-Oriented Database Systems  Promises, Reality, and Future.", "authors": "Won Kim", "misc": "2002-01-03 255-280 1995 Modern Database Systems db/books/collections/kim95.html#Kim95a" }
+{ "id": 14, "dblpid": "books/acm/kim95/Kim95b", "title": "Introduction to Part 2  Technology for Interoperating Legacy Databases.", "authors": "Won Kim", "misc": "2002-01-03 515-520 1995 Modern Database Systems db/books/collections/kim95.html#Kim95b" }
+{ "id": 15, "dblpid": "books/acm/kim95/KimCGS95", "title": "On Resolving Schematic Heterogeneity in Multidatabase Systems.", "authors": "Won Kim Injun Choi Sunit K. Gala Mark Scheevel", "misc": "2002-01-03 521-550 1995 Modern Database Systems db/books/collections/kim95.html#KimCGS95" }
+{ "id": 16, "dblpid": "books/acm/kim95/KimG95", "title": "Requirements for a Performance Benchmark for Object-Oriented Database Systems.", "authors": "Won Kim Jorge F. Garza", "misc": "2002-01-03 203-215 1995 Modern Database Systems db/books/collections/kim95.html#KimG95" }
+{ "id": 17, "dblpid": "books/acm/kim95/KimK95", "title": "On View Support in Object-Oriented Databases Systems.", "authors": "Won Kim William Kelley", "misc": "2002-01-03 108-129 1995 Modern Database Systems db/books/collections/kim95.html#KimK95" }
+{ "id": 18, "dblpid": "books/acm/kim95/Kowalski95", "title": "The POSC Solution to Managing E&P Data.", "authors": "Vincent J. Kowalski", "misc": "2002-01-03 281-301 1995 Modern Database Systems db/books/collections/kim95.html#Kowalski95" }
+{ "id": 19, "dblpid": "books/acm/kim95/KriegerA95", "title": "C++ Bindings to an Object Database.", "authors": "David Krieger Tim Andrews", "misc": "2002-01-03 89-107 1995 Modern Database Systems db/books/collections/kim95.html#KriegerA95" }
+{ "id": 20, "dblpid": "books/acm/kim95/Lunt95", "title": "Authorization in Object-Oriented Databases.", "authors": "Teresa F. Lunt", "misc": "2002-01-03 130-145 1995 Modern Database Systems db/books/collections/kim95.html#Lunt95" }
+{ "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
+{ "id": 23, "dblpid": "books/acm/kim95/Omiecinski95", "title": "Parallel Relational Database Systems.", "authors": "Edward Omiecinski", "misc": "2002-01-03 494-512 1995 Modern Database Systems db/books/collections/kim95.html#Omiecinski95" }
+{ "id": 24, "dblpid": "books/acm/kim95/OzsuB95", "title": "Query Processing in Object-Oriented Database Systems.", "authors": "M. Tamer Özsu José A. Blakeley", "misc": "2002-01-03 146-174 1995 Modern Database Systems db/books/collections/kim95.html#OzsuB95" }
+{ "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }
+{ "id": 26, "dblpid": "books/acm/kim95/Samet95", "title": "Spatial Data Structures.", "authors": "Hanan Samet", "misc": "2004-03-08 361-385 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#Samet95 1995" }
+{ "id": 27, "dblpid": "books/acm/kim95/SametA95", "title": "Spatial Data Models and Query Processing.", "authors": "Hanan Samet Walid G. Aref", "misc": "2002-01-03 338-360 1995 Modern Database Systems db/books/collections/kim95.html#SametA95" }
+{ "id": 28, "dblpid": "books/acm/kim95/ShanADDK95", "title": "Pegasus  A Heterogeneous Information Management System.", "authors": "Ming-Chien Shan Rafi Ahmed Jim Davis Weimin Du William Kent", "misc": "2004-03-08 664-682 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#ShanADDK95 1995" }
+{ "id": 29, "dblpid": "books/acm/kim95/Snodgrass95", "title": "Temporal Object-Oriented Databases  A Critical Comparison.", "authors": "Richard T. Snodgrass", "misc": "2002-01-03 386-408 1995 Modern Database Systems db/books/collections/kim95.html#Snodgrass95" }
+{ "id": 30, "dblpid": "books/acm/kim95/SoleyK95", "title": "The OMG Object Model.", "authors": "Richard Mark Soley William Kent", "misc": "2002-01-03 18-41 1995 Modern Database Systems db/books/collections/kim95.html#SoleyK95" }
+{ "id": 31, "dblpid": "books/acm/kim95/Stout95", "title": "EDA/SQL.", "authors": "Ralph L. Stout", "misc": "2004-03-08 649-663 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#Stout95 1995" }
+{ "id": 32, "dblpid": "books/acm/kim95/Thompson95", "title": "The Changing Database Standards Landscape.", "authors": "Craig W. Thompson", "misc": "2002-01-03 302-317 1995 Modern Database Systems db/books/collections/kim95.html#Thompson95" }
+{ "id": 33, "dblpid": "books/acm/kim95/BreitbartR95", "title": "Overview of the ADDS System.", "authors": "Yuri Breitbart Tom C. Reyes", "misc": "2009-06-12 683-701 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartR95 1995" }
+{ "id": 34, "dblpid": "books/acm/Kim95", "title": "Modern Database Systems  The Object Model, Interoperability, and Beyond.", "authors": "", "misc": "2004-03-08 Won Kim Modern Database Systems ACM Press and Addison-Wesley 1995 0-201-59098-0 db/books/collections/kim95.html" }
+{ "id": 35, "dblpid": "books/ap/MarshallO79", "title": "Inequalities  Theory of Majorization and Its Application.", "authors": "Albert W. Marshall Ingram Olkin", "misc": "2002-01-03 Academic Press 1979 0-12-473750-1" }
+{ "id": 36, "dblpid": "books/aw/kimL89/BjornerstedtH89", "title": "Version Control in an Object-Oriented Architecture.", "authors": "Anders Björnerstedt Christer Hulten", "misc": "2006-02-24 451-485 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#BjornerstedtH89" }
+{ "id": 37, "dblpid": "books/aw/kimL89/BretlMOPSSWW89", "title": "The GemStone Data Management System.", "authors": "Robert Bretl David Maier Allen Otis D. Jason Penney Bruce Schuchardt Jacob Stein E. Harold Williams Monty Williams", "misc": "2002-01-03 283-308 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#BretlMOPSSWW89" }
+{ "id": 38, "dblpid": "books/aw/kimL89/CareyDRS89", "title": "Storage Management in EXODUS.", "authors": "Michael J. Carey David J. DeWitt Joel E. Richardson Eugene J. Shekita", "misc": "2002-01-03 341-369 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#CareyDRS89" }
+{ "id": 39, "dblpid": "books/aw/kimL89/Decouchant89", "title": "A Distributed Object Manager for the Smalltalk-80 System.", "authors": "Dominique Decouchant", "misc": "2002-01-03 487-520 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Decouchant89" }
+{ "id": 40, "dblpid": "books/aw/kimL89/DiederichM89", "title": "Objects, Messages, and Rules in Database Design.", "authors": "Jim Diederich Jack Milton", "misc": "2002-01-03 177-197 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#DiederichM89" }
+{ "id": 41, "dblpid": "books/aw/kimL89/EllisG89", "title": "Active Objects  Ealities and Possibilities.", "authors": "Clarence A. Ellis Simon J. Gibbs", "misc": "2002-01-03 561-572 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#EllisG89" }
+{ "id": 42, "dblpid": "books/aw/kimL89/FishmanABCCDHHKLLMNRSW89", "title": "Overview of the Iris DBMS.", "authors": "Daniel H. Fishman Jurgen Annevelink David Beech E. C. Chow Tim Connors J. W. Davis Waqar Hasan C. G. Hoch William Kent S. Leichner Peter Lyngbæk Brom Mahbod Marie-Anne Neimat Tore Risch Ming-Chien Shan W. Kevin Wilkinson", "misc": "2002-01-03 219-250 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#FishmanABCCDHHKLLMNRSW89" }
+{ "id": 43, "dblpid": "books/aw/kimL89/KimBCGW89", "title": "Features of the ORION Object-Oriented Database System.", "authors": "Won Kim Nat Ballou Hong-Tai Chou Jorge F. Garza Darrell Woelk", "misc": "2002-01-03 251-282 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimBCGW89" }
+{ "id": 44, "dblpid": "books/aw/kimL89/KimKD89", "title": "Indexing Techniques for Object-Oriented Databases.", "authors": "Won Kim Kyung-Chang Kim Alfred G. Dale", "misc": "2002-01-03 371-394 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimKD89" }
+{ "id": 45, "dblpid": "books/aw/kimL89/King89", "title": "My Cat Is Object-Oriented.", "authors": "Roger King", "misc": "2002-01-03 23-30 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#King89" }
+{ "id": 46, "dblpid": "books/aw/kimL89/Maier89", "title": "Making Database Systems Fast Enough for CAD Applications.", "authors": "David Maier", "misc": "2002-01-03 573-582 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Maier89" }
+{ "id": 47, "dblpid": "books/aw/kimL89/MellenderRS89", "title": "Optimizing Smalltalk Message Performance.", "authors": "Fred Mellender Steve Riegel Andrew Straw", "misc": "2002-01-03 423-450 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#MellenderRS89" }
+{ "id": 48, "dblpid": "books/aw/kimL89/Moon89", "title": "The Common List Object-Oriented Programming Language Standard.", "authors": "David A. Moon", "misc": "2002-01-03 49-78 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Moon89" }
+{ "id": 49, "dblpid": "books/aw/kimL89/Moss89", "title": "Object Orientation as Catalyst for Language-Database Inegration.", "authors": "J. Eliot B. Moss", "misc": "2002-01-03 583-592 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Moss89" }
+{ "id": 50, "dblpid": "books/aw/kimL89/Nierstrasz89", "title": "A Survey of Object-Oriented Concepts.", "authors": "Oscar Nierstrasz", "misc": "2002-01-03 3-21 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Nierstrasz89" }
+{ "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }
+{ "id": 52, "dblpid": "books/aw/kimL89/Russinoff89", "title": "Proteus  A Frame-Based Nonmonotonic Inference System.", "authors": "David M. Russinoff", "misc": "2002-01-03 127-150 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#Russinoff89" }
+{ "id": 53, "dblpid": "books/aw/kimL89/SkarraZ89", "title": "Concurrency Control and Object-Oriented Databases.", "authors": "Andrea H. Skarra Stanley B. Zdonik", "misc": "2002-01-03 395-421 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#SkarraZ89" }
+{ "id": 54, "dblpid": "books/aw/kimL89/SteinLU89", "title": "A Shared View of Sharing  The Treaty of Orlando.", "authors": "Lynn Andrea Stein Henry Lieberman David Ungar", "misc": "2002-01-03 31-48 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#SteinLU89" }
+{ "id": 55, "dblpid": "books/aw/kimL89/TarltonT89", "title": "Pogo  A Declarative Representation System for Graphics.", "authors": "Mark A. Tarlton P. Nong Tarlton", "misc": "2002-01-03 151-176 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TarltonT89" }
+{ "id": 56, "dblpid": "books/aw/kimL89/TomlinsonS89", "title": "Concurrent Object-Oriented Programming Languages.", "authors": "Chris Tomlinson Mark Scheevel", "misc": "2002-01-03 79-124 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TomlinsonS89" }
+{ "id": 57, "dblpid": "books/aw/kimL89/TsichritzisN89", "title": "Directions in Object-Oriented Research.", "authors": "Dennis Tsichritzis Oscar Nierstrasz", "misc": "2002-01-03 523-536 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TsichritzisN89" }
+{ "id": 58, "dblpid": "books/aw/kimL89/Wand89", "title": "A Proposal for a Formal Model of Objects.", "authors": "Yair Wand", "misc": "2002-01-03 537-559 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Wand89" }
+{ "id": 59, "dblpid": "books/aw/kimL89/WeiserL89", "title": "OZ+  An Object-Oriented Database System.", "authors": "Stephen P. Weiser Frederick H. Lochovsky", "misc": "2002-01-03 309-337 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#WeiserL89" }
+{ "id": 60, "dblpid": "books/aw/stonebraker86/RoweS86", "title": "The Commercial INGRES Epilogue.", "authors": "Lawrence A. Rowe Michael Stonebraker", "misc": "2002-01-03 63-82 1986 The INGRES Papers db/books/collections/Stonebraker86.html#RoweS86 db/books/collections/Stonebraker86/RoweS86.html ingres/P063.pdf" }
+{ "id": 61, "dblpid": "books/aw/stonebraker86/Stonebraker86", "title": "Design of Relational Systems (Introduction to Section 1).", "authors": "Michael Stonebraker", "misc": "2002-01-03 1-3 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86 db/books/collections/Stonebraker86/Stonebraker86.html ingres/P001.pdf" }
+{ "id": 62, "dblpid": "books/aw/stonebraker86/Stonebraker86a", "title": "Supporting Studies on Relational Systems (Introduction to Section 2).", "authors": "Michael Stonebraker", "misc": "2002-01-03 83-85 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86a db/books/collections/Stonebraker86/Stonebraker86a.html ingres/P083.pdf" }
+{ "id": 63, "dblpid": "books/aw/stonebraker86/Stonebraker86b", "title": "Distributed Database Systems (Introduction to Section 3).", "authors": "Michael Stonebraker", "misc": "2002-01-03 183-186 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86b db/books/collections/Stonebraker86/Stonebraker86b.html ingres/P183.pdf" }
+{ "id": 64, "dblpid": "books/aw/stonebraker86/Stonebraker86c", "title": "The Design and Implementation of Distributed INGRES.", "authors": "Michael Stonebraker", "misc": "2002-01-03 187-196 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86c db/books/collections/Stonebraker86/Stonebraker86c.html ingres/P187.pdf" }
+{ "id": 65, "dblpid": "books/aw/stonebraker86/Stonebraker86d", "title": "User Interfaces for Database Systems (Introduction to Section 4).", "authors": "Michael Stonebraker", "misc": "2002-01-03 243-245 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86d db/books/collections/Stonebraker86/Stonebraker86d.html ingres/P243.pdf" }
+{ "id": 66, "dblpid": "books/aw/stonebraker86/Stonebraker86e", "title": "Extended Semantics for the Relational Model (Introduction to Section 5).", "authors": "Michael Stonebraker", "misc": "2002-01-03 313-316 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86e db/books/collections/Stonebraker86/Stonebraker86e.html ingres/P313.pdf" }
+{ "id": 67, "dblpid": "books/aw/stonebraker86/Stonebraker86f", "title": "Database Design (Introduction to Section 6).", "authors": "Michael Stonebraker", "misc": "2002-01-03 393-394 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86f db/books/collections/Stonebraker86/Stonebraker86f.html ingres/P393.pdf" }
+{ "id": 68, "dblpid": "books/aw/stonebraker86/X86", "title": "Title, Preface, Contents.", "authors": "", "misc": "2002-01-03 1986 The INGRES Papers db/books/collections/Stonebraker86.html#X86 db/books/collections/Stonebraker86/X86.html ingres/frontmatter.pdf" }
+{ "id": 69, "dblpid": "books/aw/stonebraker86/X86a", "title": "References.", "authors": "", "misc": "2002-01-03 429-444 1986 The INGRES Papers db/books/collections/Stonebraker86.html#X86a db/books/collections/Stonebraker86/X86a.html ingres/P429.pdf" }
+{ "id": 70, "dblpid": "books/aw/Knuth86a", "title": "TeX  The Program", "authors": "Donald E. Knuth", "misc": "2002-01-03 Addison-Wesley 1986 0-201-13437-3" }
+{ "id": 71, "dblpid": "conf/appt/LiDCMY07", "title": "Design and Implementation of a High-Speed Reconfigurable Modular Arithmetic Unit.", "authors": "Wei Li Zibin Dai Tao Chen Tao Meng Xuan Yang", "misc": "2007-11-09 50-59 2007 conf/appt/2007 APPT http //dx.doi.org/10.1007/978-3-540-76837-1_9 db/conf/appt/appt2007.html#LiDCMY07" }
+{ "id": 72, "dblpid": "books/aw/Lamport86", "title": "LaTeX  User's Guide & Reference Manual", "authors": "Leslie Lamport", "misc": "2002-01-03 Addison-Wesley 1986 0-201-15790-X" }
+{ "id": 73, "dblpid": "books/aw/AhoHU74", "title": "The Design and Analysis of Computer Algorithms.", "authors": "Alfred V. Aho John E. Hopcroft Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1974 0-201-00029-6" }
+{ "id": 74, "dblpid": "books/aw/Lamport2002", "title": "Specifying Systems, The TLA+ Language and Tools for Hardware and Software Engineers", "authors": "Leslie Lamport", "misc": "2005-07-28 Addison-Wesley 2002 0-3211-4306-X http //research.microsoft.com/users/lamport/tla/book.html" }
+{ "id": 75, "dblpid": "books/aw/AhoHU83", "title": "Data Structures and Algorithms.", "authors": "Alfred V. Aho John E. Hopcroft Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1983 0-201-00023-7" }
+{ "id": 76, "dblpid": "books/aw/LewisBK01", "title": "Databases and Transaction Processing  An Application-Oriented Approach", "authors": "Philip M. Lewis Arthur J. Bernstein Michael Kifer", "misc": "2002-01-03 Addison-Wesley 2001 0-201-70872-8" }
+{ "id": 77, "dblpid": "books/aw/AhoKW88", "title": "The AWK Programming Language", "authors": "Alfred V. Aho Brian W. Kernighan Peter J. Weinberger", "misc": "2002-01-03 Addison-Wesley 1988" }
+{ "id": 78, "dblpid": "books/aw/LindholmY97", "title": "The Java Virtual Machine Specification", "authors": "Tim Lindholm Frank Yellin", "misc": "2002-01-28 Addison-Wesley 1997 0-201-63452-X" }
+{ "id": 79, "dblpid": "books/aw/AhoSU86", "title": "Compilers  Princiles, Techniques, and Tools.", "authors": "Alfred V. Aho Ravi Sethi Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1986 0-201-10088-6" }
+{ "id": 80, "dblpid": "books/aw/Sedgewick83", "title": "Algorithms", "authors": "Robert Sedgewick", "misc": "2002-01-03 Addison-Wesley 1983 0-201-06672-6" }
+{ "id": 81, "dblpid": "journals/siamcomp/AspnesW96", "title": "Randomized Consensus in Expected O(n log² n) Operations Per Processor.", "authors": "James Aspnes Orli Waarts", "misc": "2002-01-03 1024-1044 1996 25 SIAM J. Comput. 5 db/journals/siamcomp/siamcomp25.html#AspnesW96" }
+{ "id": 82, "dblpid": "conf/focs/AspnesW92", "title": "Randomized Consensus in Expected O(n log ^2 n) Operations Per Processor", "authors": "James Aspnes Orli Waarts", "misc": "2006-04-25 137-146 conf/focs/FOCS33 1992 FOCS db/conf/focs/focs92.html#AspnesW92" }
+{ "id": 83, "dblpid": "journals/siamcomp/Bloniarz83", "title": "A Shortest-Path Algorithm with Expected Time O(n² log n log* n).", "authors": "Peter A. Bloniarz", "misc": "2002-01-03 588-600 1983 12 SIAM J. Comput. 3 db/journals/siamcomp/siamcomp12.html#Bloniarz83" }
+{ "id": 84, "dblpid": "conf/stoc/Bloniarz80", "title": "A Shortest-Path Algorithm with Expected Time O(n^2 log n log ^* n)", "authors": "Peter A. Bloniarz", "misc": "2006-04-25 378-384 conf/stoc/STOC12 1980 STOC db/conf/stoc/stoc80.html#Bloniarz80" }
+{ "id": 85, "dblpid": "journals/siamcomp/Megiddo83a", "title": "Linear-Time Algorithms for Linear Programming in R³ and Related Problems.", "authors": "Nimrod Megiddo", "misc": "2002-01-03 759-776 1983 12 SIAM J. Comput. 4 db/journals/siamcomp/siamcomp12.html#Megiddo83a" }
+{ "id": 86, "dblpid": "conf/focs/Megiddo82", "title": "Linear-Time Algorithms for Linear Programming in R^3 and Related Problems", "authors": "Nimrod Megiddo", "misc": "2006-04-25 329-338 conf/focs/FOCS23 1982 FOCS db/conf/focs/focs82.html#Megiddo82" }
+{ "id": 87, "dblpid": "journals/siamcomp/MoffatT87", "title": "An All Pairs Shortest Path Algorithm with Expected Time O(n² log n).", "authors": "Alistair Moffat Tadao Takaoka", "misc": "2002-01-03 1023-1031 1987 16 SIAM J. Comput. 6 db/journals/siamcomp/siamcomp16.html#MoffatT87" }
+{ "id": 88, "dblpid": "conf/focs/MoffatT85", "title": "An All Pairs Shortest Path Algorithm with Expected Running Time O(n^2 log n)", "authors": "Alistair Moffat Tadao Takaoka", "misc": "2006-04-25 101-105 conf/focs/FOCS26 1985 FOCS db/conf/focs/focs85.html#MoffatT85" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
+{ "id": 91, "dblpid": "journals/corr/abs-0802-2861", "title": "Geometric Set Cover and Hitting Sets for Polytopes in $R^3$", "authors": "Sören Laue", "misc": "2008-03-03 http //arxiv.org/abs/0802.2861 2008 CoRR abs/0802.2861 db/journals/corr/corr0802.html#abs-0802-2861 informal publication" }
+{ "id": 92, "dblpid": "conf/stacs/Laue08", "title": "Geometric Set Cover and Hitting Sets for Polytopes in R³.", "authors": "Sören Laue", "misc": "2008-03-04 2008 STACS 479-490 http //drops.dagstuhl.de/opus/volltexte/2008/1367 conf/stacs/2008 db/conf/stacs/stacs2008.html#Laue08" }
+{ "id": 93, "dblpid": "journals/iandc/IbarraJCR91", "title": "Some Classes of Languages in NC¹", "authors": "Oscar H. Ibarra Tao Jiang Jik H. Chang Bala Ravikumar", "misc": "2006-04-25 86-106 Inf. Comput. January 1991 90 1 db/journals/iandc/iandc90.html#IbarraJCR91" }
+{ "id": 94, "dblpid": "conf/awoc/IbarraJRC88", "title": "On Some Languages in NC.", "authors": "Oscar H. Ibarra Tao Jiang Bala Ravikumar Jik H. Chang", "misc": "2002-08-06 64-73 1988 conf/awoc/1988 AWOC db/conf/awoc/awoc88.html#IbarraJRC88" }
+{ "id": 95, "dblpid": "journals/jacm/GalilHLSW87", "title": "An O(n³log n) deterministic and an O(n³) Las Vegs isomorphism test for trivalent graphs.", "authors": "Zvi Galil Christoph M. Hoffmann Eugene M. Luks Claus-Peter Schnorr Andreas Weber", "misc": "2003-11-20 513-531 1987 34 J. ACM 3 http //doi.acm.org/10.1145/28869.28870 db/journals/jacm/jacm34.html#GalilHLSW87" }
+{ "id": 96, "dblpid": "conf/focs/GalilHLSW82", "title": "An O(n^3 log n) Deterministic and an O(n^3) Probabilistic Isomorphism Test for Trivalent Graphs", "authors": "Zvi Galil Christoph M. Hoffmann Eugene M. Luks Claus-Peter Schnorr Andreas Weber", "misc": "2006-04-25 118-125 conf/focs/FOCS23 1982 FOCS db/conf/focs/focs82.html#GalilHLSW82" }
+{ "id": 97, "dblpid": "journals/jacm/GalilT88", "title": "An O(n²(m + n log n)log n) min-cost flow algorithm.", "authors": "Zvi Galil Éva Tardos", "misc": "2003-11-20 374-386 1988 35 J. ACM 2 http //doi.acm.org/10.1145/42282.214090 db/journals/jacm/jacm35.html#GalilT88" }
+{ "id": 98, "dblpid": "conf/focs/GalilT86", "title": "An O(n^2 (m + n log n) log n) Min-Cost Flow Algorithm", "authors": "Zvi Galil Éva Tardos", "misc": "2006-04-25 1-9 conf/focs/FOCS27 1986 FOCS db/conf/focs/focs86.html#GalilT86" }
+{ "id": 99, "dblpid": "series/synthesis/2009Weintraub", "title": "Jordan Canonical Form  Theory and Practice", "authors": "Steven H. Weintraub", "misc": "2009-09-06 Jordan Canonical Form  Theory and Practice http //dx.doi.org/10.2200/S00218ED1V01Y200908MAS006 http //dx.doi.org/10.2200/S00218ED1V01Y200908MAS006 2009 Synthesis Lectures on Mathematics & Statistics Morgan & Claypool Publishers" }
+{ "id": 100, "dblpid": "series/synthesis/2009Brozos", "title": "The Geometry of Walker Manifolds", "authors": "Miguel Brozos-Vázquez Eduardo García-Río Peter Gilkey Stana Nikcevic Rámon Vázquez-Lorenzo", "misc": "2009-09-06 The Geometry of Walker Manifolds http //dx.doi.org/10.2200/S00197ED1V01Y200906MAS005 http //dx.doi.org/10.2200/S00197ED1V01Y200906MAS005 2009 Synthesis Lectures on Mathematics & Statistics Morgan & Claypool Publishers" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.ast
index 6c05c55..c3f5b77 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.ast
@@ -1,6 +1,70 @@
 Query:
-SELECT ELEMENT [
-Variable [ Name=$l ]
+SELECT [
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DataverseName
+]
+DataverseName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetName
+]
+DatasetName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatatypeDataverseName
+]
+DatatypeDataverseName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatatypeName
+]
+DatatypeName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetType
+]
+DatasetType
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=GroupName
+]
+GroupName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=CompactionPolicy
+]
+CompactionPolicy
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=CompactionPolicyProperties
+]
+CompactionPolicyProperties
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=InternalDetails
+]
+InternalDetails
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=Hints
+]
+Hints
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=Timestamp
+]
+Timestamp
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetId
+]
+DatasetId
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=PendingOp
+]
+PendingOp
 ]
 FROM [  FunctionCall asterix.dataset@1[
     LiteralExpr [STRING] [Metadata.Dataset]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.ast
index 6c05c55..c3f5b77 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.ast
@@ -1,6 +1,70 @@
 Query:
-SELECT ELEMENT [
-Variable [ Name=$l ]
+SELECT [
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DataverseName
+]
+DataverseName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetName
+]
+DatasetName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatatypeDataverseName
+]
+DatatypeDataverseName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatatypeName
+]
+DatatypeName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetType
+]
+DatasetType
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=GroupName
+]
+GroupName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=CompactionPolicy
+]
+CompactionPolicy
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=CompactionPolicyProperties
+]
+CompactionPolicyProperties
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=InternalDetails
+]
+InternalDetails
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=Hints
+]
+Hints
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=Timestamp
+]
+Timestamp
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetId
+]
+DatasetId
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=PendingOp
+]
+PendingOp
 ]
 FROM [  FunctionCall asterix.dataset@1[
     LiteralExpr [STRING] [Metadata.Dataset]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.ast
index c2f1488..b3796f1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.ast
@@ -1,6 +1,75 @@
 Query:
-SELECT ELEMENT [
-Variable [ Name=$l ]
+SELECT [
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DataverseName
+]
+DataverseName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetName
+]
+DatasetName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatatypeDataverseName
+]
+DatatypeDataverseName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatatypeName
+]
+DatatypeName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetType
+]
+DatasetType
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=GroupName
+]
+GroupName
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=CompactionPolicy
+]
+CompactionPolicy
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=CompactionPolicyProperties
+]
+CompactionPolicyProperties
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=ExternalDetails
+]
+ExternalDetails
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=InternalDetails
+]
+InternalDetails
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=Hints
+]
+Hints
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=Timestamp
+]
+Timestamp
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=DatasetId
+]
+DatasetId
+FieldAccessor [
+  Variable [ Name=$l ]
+  Field=PendingOp
+]
+PendingOp
 ]
 FROM [  FunctionCall asterix.dataset@1[
     LiteralExpr [STRING] [Metadata.Dataset]


[02/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCacheHeaderHelper.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCacheHeaderHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCacheHeaderHelper.java
new file mode 100644
index 0000000..a913513
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCacheHeaderHelper.java
@@ -0,0 +1,90 @@
+/*
+ * 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.hyracks.storage.common.buffercache;
+
+import static org.apache.hyracks.storage.common.buffercache.IBufferCache.RESERVED_HEADER_BYTES;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+
+public class BufferCacheHeaderHelper {
+    private static final int FRAME_MULTIPLIER_OFF = 0;
+    private static final int EXTRA_BLOCK_PAGE_ID_OFF = FRAME_MULTIPLIER_OFF + 4; // 4
+
+    private final ByteBuffer[] array;
+    private final int pageSizeWithHeader;
+    private ByteBuffer buf;
+
+    public BufferCacheHeaderHelper(int pageSize) {
+        this.pageSizeWithHeader = RESERVED_HEADER_BYTES + pageSize;
+        buf = ByteBuffer.allocate(pageSizeWithHeader);
+        array = new ByteBuffer[] { buf, null };
+    }
+
+    public ByteBuffer[] prepareWrite(CachedPage cPage) {
+        setPageInfo(cPage);
+        buf.position(0);
+        buf.limit(RESERVED_HEADER_BYTES);
+        array[1] = cPage.buffer;
+        return array;
+    }
+
+    public ByteBuffer prepareWrite(CachedPage cPage, int requiredSize) {
+        ensureBufferCapacity(requiredSize);
+        setPageInfo(cPage);
+        buf.position(RESERVED_HEADER_BYTES);
+        buf.limit(buf.capacity());
+        return buf;
+    }
+
+    public ByteBuffer prepareRead(int size) {
+        buf.position(0);
+        buf.limit(size);
+        return buf;
+    }
+
+    public ByteBuffer processHeader(CachedPage cPage) {
+        cPage.setFrameSizeMultiplier(buf.getInt(FRAME_MULTIPLIER_OFF));
+        cPage.setExtraBlockPageId(buf.getInt(EXTRA_BLOCK_PAGE_ID_OFF));
+        buf.position(RESERVED_HEADER_BYTES);
+        return buf;
+    }
+
+    private void setPageInfo(CachedPage cPage) {
+        buf.putInt(FRAME_MULTIPLIER_OFF, cPage.getFrameSizeMultiplier());
+        buf.putInt(EXTRA_BLOCK_PAGE_ID_OFF, cPage.getExtraBlockPageId());
+    }
+
+    /**
+     * {@link ICompressorDecompressor#compress(byte[], int, int, byte[], int)} may require additional
+     * space to do the compression. see {@link ICompressorDecompressor#computeCompressedBufferSize(int)}.
+     *
+     * @param compressor
+     * @param size
+     */
+    private void ensureBufferCapacity(int size) {
+        final int requiredSize = size + RESERVED_HEADER_BYTES;
+        if (buf.capacity() < requiredSize) {
+            buf = ByteBuffer.allocate(requiredSize);
+            array[0] = buf;
+        }
+        buf.limit(buf.capacity());
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
index 6ec12aa..cf0553c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
@@ -45,6 +45,8 @@ public class CachedPage implements ICachedPageInternal {
     private IQueueInfo queueInfo;
     private int multiplier;
     private int extraBlockPageId;
+    private long compressedOffset;
+    private int compressedSize;
     // DEBUG
     private static final boolean DEBUG = false;
     private final StackTraceElement[] ctorStack;
@@ -224,4 +226,23 @@ public class CachedPage implements ICachedPageInternal {
             LOGGER.error("An IO Failure took place but the failure callback is not set", e);
         }
     }
+
+    public void setCompressedPageOffset(long offset) {
+        this.compressedOffset = offset;
+    }
+
+    @Override
+    public long getCompressedPageOffset() {
+        return compressedOffset;
+    }
+
+    @Override
+    public void setCompressedPageSize(int size) {
+        this.compressedSize = size;
+    }
+
+    @Override
+    public int getCompressedPageSize() {
+        return compressedSize;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
index 21d3677..c762dd9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
@@ -21,6 +21,7 @@ package org.apache.hyracks.storage.common.buffercache;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.replication.IIOReplicationManager;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
 
 public interface IBufferCache {
 
@@ -278,4 +279,11 @@ public interface IBufferCache {
      */
     void closeFileIfOpen(FileReference fileRef);
 
+    /**
+     * @return compressed page writer
+     */
+    default ICompressedPageWriter getCompressedPageWriter(int fileId) {
+        throw new UnsupportedOperationException(this.getClass().getName() + " does not support compressed pages");
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java
index d900852..04e93db 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java
@@ -35,4 +35,12 @@ public interface ICachedPageInternal extends ICachedPage {
     int getExtraBlockPageId();
 
     void setExtraBlockPageId(int extraBlockPageId);
+
+    void setCompressedPageOffset(long offset);
+
+    long getCompressedPageOffset();
+
+    void setCompressedPageSize(int size);
+
+    int getCompressedPageSize();
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/NoOpCompressorDecompressor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/NoOpCompressorDecompressor.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/NoOpCompressorDecompressor.java
new file mode 100644
index 0000000..c4855bd
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/NoOpCompressorDecompressor.java
@@ -0,0 +1,46 @@
+/*
+ * 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.hyracks.storage.common.compression;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NoOpCompressorDecompressor implements ICompressorDecompressor {
+    public static final NoOpCompressorDecompressor INSTANCE = new NoOpCompressorDecompressor();
+
+    private NoOpCompressorDecompressor() {
+    }
+
+    @Override
+    public int computeCompressedBufferSize(int uBufferSize) {
+        return 0;
+    }
+
+    @Override
+    public ByteBuffer compress(ByteBuffer uBuffer, ByteBuffer cBuffer) throws HyracksDataException {
+        return uBuffer;
+    }
+
+    @Override
+    public ByteBuffer uncompress(ByteBuffer cBuffer, ByteBuffer uBuffer) throws HyracksDataException {
+        return cBuffer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/NoOpCompressorDecompressorFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/NoOpCompressorDecompressorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/NoOpCompressorDecompressorFactory.java
new file mode 100644
index 0000000..690f4a2
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/NoOpCompressorDecompressorFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.hyracks.storage.common.compression;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class NoOpCompressorDecompressorFactory implements ICompressorDecompressorFactory {
+    private static final long serialVersionUID = 1L;
+    public static final ICompressorDecompressorFactory INSTANCE = new NoOpCompressorDecompressorFactory();
+
+    @Override
+    public ICompressorDecompressor createInstance() {
+        return NoOpCompressorDecompressor.INSTANCE;
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/SnappyCompressorDecompressor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/SnappyCompressorDecompressor.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/SnappyCompressorDecompressor.java
new file mode 100644
index 0000000..16c9a2d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/SnappyCompressorDecompressor.java
@@ -0,0 +1,66 @@
+/*
+ * 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.hyracks.storage.common.compression;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.xerial.snappy.Snappy;
+
+/**
+ * Built-in Snappy compressor/decompressor wrapper
+ */
+public class SnappyCompressorDecompressor implements ICompressorDecompressor {
+    protected static final SnappyCompressorDecompressor INSTANCE = new SnappyCompressorDecompressor();
+
+    private SnappyCompressorDecompressor() {
+
+    }
+
+    @Override
+    public int computeCompressedBufferSize(int uncompressedBufferSize) {
+        return Snappy.maxCompressedLength(uncompressedBufferSize);
+    }
+
+    @Override
+    public ByteBuffer compress(ByteBuffer uBuffer, ByteBuffer cBuffer) throws HyracksDataException {
+        try {
+            final int cLength = Snappy.compress(uBuffer.array(), uBuffer.position(), uBuffer.remaining(),
+                    cBuffer.array(), cBuffer.position());
+            cBuffer.limit(cBuffer.position() + cLength);
+            return cBuffer;
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public ByteBuffer uncompress(ByteBuffer cBuffer, ByteBuffer uBuffer) throws HyracksDataException {
+        try {
+            final int uLength = Snappy.uncompress(cBuffer.array(), cBuffer.position(), cBuffer.remaining(),
+                    uBuffer.array(), uBuffer.position());
+            uBuffer.limit(uBuffer.position() + uLength);
+            return uBuffer;
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/SnappyCompressorDecompressorFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/SnappyCompressorDecompressorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/SnappyCompressorDecompressorFactory.java
new file mode 100644
index 0000000..93f31bd
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/SnappyCompressorDecompressorFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hyracks.storage.common.compression;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class SnappyCompressorDecompressorFactory implements ICompressorDecompressorFactory {
+    private static final long serialVersionUID = 1L;
+    private static final ICompressorDecompressorFactory INSTANCE = new SnappyCompressorDecompressorFactory();
+
+    @Override
+    public ICompressorDecompressor createInstance() {
+        return SnappyCompressorDecompressor.INSTANCE;
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
new file mode 100644
index 0000000..70d1388
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
@@ -0,0 +1,285 @@
+/*
+ * 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.hyracks.storage.common.compression.file;
+
+import java.nio.ByteBuffer;
+import java.util.EnumSet;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.ICachedPageInternal;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+/**
+ * CompressedFileManager is responsible to manage the Look Aside File (LAF file), which contains
+ * the compression information. LAF file format is as follow:
+ *
+ * [<offset0, size0>, <offset1, size1> .... <offsetN, sizeN>]
+ * Each entry <offsetM, sizeM> is an entry of 16-bytes for page M (8 bytes for offset and 8 for size).
+ *
+ * The file is responsible to store the beginning and the size of each page after compression.
+ */
+public class CompressedFileManager {
+    protected static final int SIZE_ENTRY_OFFSET = 8; // 0 is for the compressed page offset
+    protected static final int ENTRY_LENGTH = 16; //<offset(8-bytes),size(8-bytes)>
+    protected static final int EOF = -1;
+    private static final EnumSet<State> CLOSED = EnumSet.of(State.CLOSED);
+    private static final EnumSet<State> READABLE_WRITABLE = EnumSet.of(State.READABLE, State.WRITABLE);
+    private static final EnumSet<State> READABLE = EnumSet.of(State.READABLE);
+    private static final EnumSet<State> WRITABLE = EnumSet.of(State.WRITABLE);
+
+    private enum State {
+        READABLE,
+        WRITABLE,
+        CLOSED
+    }
+
+    private final IBufferCache bufferCache;
+    private final int fileId;
+    private final ICompressorDecompressor compressorDecompressor;
+
+    private State state;
+    private int totalNumOfPages;
+
+    private LAFWriter lafWriter;
+
+    public CompressedFileManager(IBufferCache bufferCache, int fileId, CompressedFileReference fileRef) {
+        state = State.CLOSED;
+        totalNumOfPages = 0;
+        this.bufferCache = bufferCache;
+        this.fileId = fileId;
+        this.compressorDecompressor = fileRef.getCompressorDecompressor();
+    }
+
+    /**
+     * If the file is empty (i.e. the number of pages is zero)
+     * Then the state will be WRITABLE.
+     *
+     * @throws HyracksDataException
+     */
+    public void open() throws HyracksDataException {
+        ensureState(CLOSED);
+        changeToFunctionalState();
+    }
+
+    /**
+     * Close the LAF file.
+     *
+     * @throws HyracksDataException
+     */
+    public void close() {
+        ensureState(READABLE_WRITABLE);
+        state = State.CLOSED;
+    }
+
+    /* ************************
+     * LAF writing methods
+     * ************************
+     */
+
+    public ICompressedPageWriter getCompressedPageWriter() {
+        ensureState(WRITABLE);
+        return lafWriter;
+    }
+
+    /**
+     * Add page information (offset, size) after compression.
+     *
+     * @param dpid
+     * @param size
+     * @return offset for the compressed page.
+     * @throws HyracksDataException
+     */
+    public long writePageInfo(long dpid, long size) throws HyracksDataException {
+        final int pageId = BufferedFileHandle.getPageId(dpid);
+        //Write the page (extraPageIndex = 0)
+        return writeExtraPageInfo(pageId, size, 0);
+    }
+
+    /**
+     * Add extra page information (offset, size) after compression.
+     *
+     * @param extraPageId
+     *            extra page ID
+     * @param size
+     *            size of the extra page
+     * @param extraPageIndex
+     *            the index of the extra page (starting from 0)
+     * @return offset for the compressed page.
+     * @throws HyracksDataException
+     */
+    public long writeExtraPageInfo(int extraPageId, long size, int extraPageIndex) throws HyracksDataException {
+        ensureState(WRITABLE);
+
+        final long compressedPageOffset;
+        try {
+            compressedPageOffset = lafWriter.writePageInfo(extraPageId + extraPageIndex, size);
+        } catch (HyracksDataException e) {
+            lafWriter.abort();
+            throw e;
+        }
+
+        return compressedPageOffset;
+    }
+
+    /**
+     * This methods is used by {@link LAFWriter#endWriting()} to signal the end of writing.
+     * After calling this methods, LAF file will be READ-ONLY.
+     *
+     * @param totalNumOfPages
+     *            The total number of pages of the index
+     * @throws HyracksDataException
+     */
+    void endWriting(int totalNumOfPages) {
+        ensureState(WRITABLE);
+        this.totalNumOfPages = totalNumOfPages;
+        lafWriter = null;
+        state = State.READABLE;
+    }
+
+    /* ************************
+     * LAF reading methods
+     * ************************
+     */
+
+    /**
+     * Set the compressed page offset and size
+     *
+     * @param compressedPage
+     * @throws HyracksDataException
+     */
+    public void setCompressedPageInfo(ICachedPageInternal compressedPage) throws HyracksDataException {
+        setCompressedPageInfo(BufferedFileHandle.getPageId(compressedPage.getDiskPageId()), compressedPage);
+    }
+
+    /**
+     * Set the extra compressed page offset and size
+     *
+     * @param compressedPage
+     * @param extraPageIndex
+     * @throws HyracksDataException
+     */
+    public void setExtraCompressedPageInfo(ICachedPageInternal compressedPage, int extraPageIndex)
+            throws HyracksDataException {
+        setCompressedPageInfo(compressedPage.getExtraBlockPageId() + extraPageIndex, compressedPage);
+    }
+
+    /* ************************
+     * LAF general methods
+     * ************************
+     */
+
+    /**
+     * Get the number of compressed pages
+     *
+     * @return
+     */
+    public int getNumberOfPages() {
+        return totalNumOfPages;
+    }
+
+    public int getFileId() {
+        return fileId;
+    }
+
+    public ICompressorDecompressor getCompressorDecompressor() {
+        return compressorDecompressor;
+    }
+
+    /* ************************
+     * Private methods
+     * ************************
+     */
+
+    private void ensureState(EnumSet<State> expectedStates) {
+        if (!expectedStates.contains(state)) {
+            throw new IllegalStateException(
+                    "Expecting the state to be " + expectedStates + ". Currently it is " + state);
+        }
+    }
+
+    private void changeToFunctionalState() throws HyracksDataException {
+        if (bufferCache.getNumPagesOfFile(fileId) == 0) {
+            state = State.WRITABLE;
+            lafWriter = new LAFWriter(this, bufferCache);
+        } else {
+            state = State.READABLE;
+            init();
+        }
+    }
+
+    private void init() throws HyracksDataException {
+        final int numOfPages = bufferCache.getNumPagesOfFile(fileId);
+        //Maximum number of entries in a page
+        final int numOfEntriesPerPage = bufferCache.getPageSize() / ENTRY_LENGTH;
+        //get the last page which may contain less entries than maxNumOfEntries
+        final long dpid = getDiskPageId(numOfPages - 1);
+        final ICachedPage page = bufferCache.pin(dpid, false);
+        try {
+            final ByteBuffer buf = page.getBuffer();
+
+            //Start at 1 since it is impossible to have EOF at the first entry of a page
+            int i = 1;
+            //Seek EOF and count number of entries
+            while (i < numOfEntriesPerPage && buf.getLong(i * ENTRY_LENGTH) != EOF) {
+                i++;
+            }
+
+            totalNumOfPages = (numOfPages - 1) * numOfEntriesPerPage + i;
+        } finally {
+            bufferCache.unpin(page);
+        }
+    }
+
+    private ICachedPage pinAndGetPage(int compressedPageId) throws HyracksDataException {
+        final int pageId = compressedPageId * ENTRY_LENGTH / bufferCache.getPageSize();
+        return bufferCache.pin(getDiskPageId(pageId), false);
+    }
+
+    private long getDiskPageId(int pageId) {
+        return BufferedFileHandle.getDiskPageId(fileId, pageId);
+    }
+
+    private void setCompressedPageInfo(int compressedPageId, ICachedPageInternal compressedPage)
+            throws HyracksDataException {
+        ensureState(READABLE);
+        if (totalNumOfPages == 0) {
+            /*
+             * It seems it is legal to pin empty file.
+             * Return the page information as it is not compressed.
+             */
+            compressedPage.setCompressedPageOffset(0);
+            compressedPage.setCompressedPageSize(bufferCache.getPageSize());
+            return;
+        }
+        final ICachedPage page = pinAndGetPage(compressedPageId);
+        try {
+            // No need for read latches as pages are immutable.
+            final ByteBuffer buf = page.getBuffer();
+            final int entryOffset = compressedPageId * ENTRY_LENGTH % bufferCache.getPageSize();
+            compressedPage.setCompressedPageOffset(buf.getLong(entryOffset));
+            compressedPage.setCompressedPageSize((int) buf.getLong(entryOffset + SIZE_ENTRY_OFFSET));
+        } finally {
+            bufferCache.unpin(page);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileReference.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileReference.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileReference.java
new file mode 100644
index 0000000..3fb682c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileReference.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.hyracks.storage.common.compression.file;
+
+import java.util.Objects;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IODeviceHandle;
+
+public class CompressedFileReference extends FileReference {
+    private static final long serialVersionUID = 1L;
+
+    private final String lafPath;
+    private final FileReference lafFileRef;
+    private final transient ICompressorDecompressor compressorDecompressor;
+
+    public CompressedFileReference(IODeviceHandle dev, ICompressorDecompressor compressorDecompressor, String path,
+            String lafPath) {
+        super(dev, path);
+        this.lafPath = lafPath;
+        lafFileRef = new FileReference(dev, lafPath);
+        this.compressorDecompressor = compressorDecompressor;
+    }
+
+    public FileReference getLAFFileReference() {
+        return lafFileRef;
+    }
+
+    public ICompressorDecompressor getCompressorDecompressor() {
+        return compressorDecompressor;
+    }
+
+    @Override
+    public boolean delete() {
+        return lafFileRef.delete() && super.delete();
+    }
+
+    @Override
+    public boolean isCompressed() {
+        return true;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof CompressedFileReference)) {
+            return false;
+        }
+        return super.equals(o) && lafPath.equals(((CompressedFileReference) o).lafPath);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(super.getRelativePath(), lafPath);
+    }
+
+    /**
+     * @return the relative path for LAF file
+     */
+    public String getLAFRelativePath() {
+        return lafPath;
+    }
+
+    /**
+     * @return the absolute path for LAF file
+     */
+    public String getLAFAbsolutePath() {
+        return lafFileRef.getAbsolutePath();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/ICompressedPageWriter.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/ICompressedPageWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/ICompressedPageWriter.java
new file mode 100644
index 0000000..86525b0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/ICompressedPageWriter.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hyracks.storage.common.compression.file;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+/**
+ * An interface that exposes the Look Aside File (LAF) writer to the indexes.
+ */
+public interface ICompressedPageWriter {
+    /**
+     * Before the index can write a compressed page, the index has to prepare the writer.
+     *
+     * @param cPage
+     * @throws HyracksDataException
+     */
+    public void prepareWrite(ICachedPage cPage) throws HyracksDataException;
+
+    /**
+     * Signal the writer to abort.
+     */
+    public void abort();
+
+    /**
+     * Finalize the writing of the compressed pages.
+     *
+     * @return
+     * @throws HyracksDataException
+     */
+    void endWriting() throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/LAFWriter.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/LAFWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/LAFWriter.java
new file mode 100644
index 0000000..dcccc52
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/LAFWriter.java
@@ -0,0 +1,260 @@
+/*
+ * 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.hyracks.storage.common.compression.file;
+
+import static org.apache.hyracks.storage.common.compression.file.CompressedFileManager.ENTRY_LENGTH;
+import static org.apache.hyracks.storage.common.compression.file.CompressedFileManager.EOF;
+import static org.apache.hyracks.storage.common.compression.file.CompressedFileManager.SIZE_ENTRY_OFFSET;
+
+import java.util.ArrayDeque;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.ICachedPageInternal;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.annotations.NotThreadSafe;
+
+/**
+ * Look Aside File writer
+ * This class is called by two threads simultaneously:
+ * - a thread to prepare the LAF page (bulk-loader)
+ * - and a writer thread to write the LAF page (buffer cache writer thread)
+ * Hence, it is not thread safe to have more than one thread to prepare or to write LAF pages.
+ */
+@NotThreadSafe
+class LAFWriter implements ICompressedPageWriter {
+    private final CompressedFileManager compressedFileManager;
+    private final IBufferCache bufferCache;
+    private final IFIFOPageQueue queue;
+    private final Map<Integer, LAFFrame> cachedFrames;
+    private final Queue<LAFFrame> recycledFrames;
+    private final int fileId;
+    private final int maxNumOfEntries;
+    private final PageWriteFailureCallback callBack;
+    private LAFFrame currentFrame;
+    private int currentPageId;
+    private int maxPageId;
+
+    private long lastOffset;
+    private int totalNumOfPages;
+
+    public LAFWriter(CompressedFileManager compressedFileManager, IBufferCache bufferCache) {
+        this.compressedFileManager = compressedFileManager;
+        this.bufferCache = bufferCache;
+        queue = bufferCache.createFIFOQueue();
+        cachedFrames = new HashMap<>();
+        recycledFrames = new ArrayDeque<>();
+        this.fileId = compressedFileManager.getFileId();
+        callBack = new PageWriteFailureCallback();
+
+        maxNumOfEntries = bufferCache.getPageSize() / ENTRY_LENGTH;
+        lastOffset = 0;
+        totalNumOfPages = 0;
+        maxPageId = -1;
+        currentPageId = -1;
+    }
+
+    /* ************************************
+     * ICompressedPageWriter methods
+     * Called by non-BufferCache thread (Bulk-loader)
+     * ************************************
+     */
+
+    @Override
+    public void prepareWrite(ICachedPage cPage) throws HyracksDataException {
+        final ICachedPageInternal internalPage = (ICachedPageInternal) cPage;
+        final int entryPageId = getLAFEntryPageId(BufferedFileHandle.getPageId(internalPage.getDiskPageId()));
+
+        synchronized (cachedFrames) {
+            if (!cachedFrames.containsKey(entryPageId)) {
+                try {
+                    //Writing new page(s). Confiscate the page(s) from the buffer cache.
+                    prepareFrames(entryPageId, internalPage);
+                } catch (HyracksDataException e) {
+                    abort();
+                    throw e;
+                }
+            }
+        }
+    }
+
+    private void prepareFrames(int entryPageId, ICachedPageInternal cPage) throws HyracksDataException {
+        //Confiscate the first page
+        confiscatePage(entryPageId);
+        //check if extra pages spans to the next entry page
+        for (int i = 0; i < cPage.getFrameSizeMultiplier() - 1; i++) {
+            final int extraEntryPageId = getLAFEntryPageId(cPage.getExtraBlockPageId() + i);
+            if (!cachedFrames.containsKey(extraEntryPageId)) {
+                confiscatePage(extraEntryPageId);
+            }
+        }
+    }
+
+    private void confiscatePage(int pageId) throws HyracksDataException {
+        //Writing new page. Confiscate the page from the buffer cache.
+        final ICachedPage newPage = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, pageId));
+        cachedFrames.put(pageId, getLAFFrame(newPage));
+        maxPageId = Math.max(maxPageId, pageId);
+    }
+
+    private LAFFrame getLAFFrame(ICachedPage cPage) {
+        LAFFrame lafFrame = recycledFrames.poll();
+        if (lafFrame == null) {
+            lafFrame = new LAFFrame();
+        }
+        lafFrame.setCachedPage(cPage);
+        return lafFrame;
+    }
+
+    @Override
+    public void endWriting() throws HyracksDataException {
+        if (callBack.hasFailed()) {
+            //if write failed, return confiscated pages
+            abort();
+            throw HyracksDataException.create(callBack.getFailure());
+        }
+        synchronized (cachedFrames) {
+            final LAFFrame lastPage = cachedFrames.get(maxPageId);
+            if (lastPage != null && !lastPage.isFull()) {
+                /*
+                 * The last page may or may not be filled. In case it is not filled (i.e do not have
+                 * the max number of entries). Then, write an indicator after the last entry.
+                 * If it has been written (i.e lastPage = null), that means it has been filled.
+                 */
+                lastPage.setEOF();
+            }
+            for (Entry<Integer, LAFFrame> entry : cachedFrames.entrySet()) {
+                queue.put(entry.getValue().cPage, callBack);
+            }
+            bufferCache.finishQueue();
+
+            //Signal the compressedFileManager to change its state
+            compressedFileManager.endWriting(totalNumOfPages);
+        }
+    }
+
+    @Override
+    public void abort() {
+        synchronized (cachedFrames) {
+            for (Entry<Integer, LAFFrame> frame : cachedFrames.entrySet()) {
+                bufferCache.returnPage(frame.getValue().cPage);
+            }
+        }
+    }
+
+    /* ************************************
+     * Local methods:
+     * Called by BufferCache writer thread
+     * ************************************
+     */
+
+    public long writePageInfo(int pageId, long size) throws HyracksDataException {
+        final LAFFrame frame = getPageBuffer(pageId);
+
+        final long pageOffset = lastOffset;
+        frame.writePageInfo(pageId, pageOffset, size);
+        lastOffset += size;
+        totalNumOfPages++;
+
+        writeFullPage();
+        return pageOffset;
+    }
+
+    private LAFFrame getPageBuffer(int compressedPageId) {
+        final int pageId = getLAFEntryPageId(compressedPageId);
+
+        if (currentPageId == pageId) {
+            return currentFrame;
+        }
+
+        final LAFFrame frame;
+        synchronized (cachedFrames) {
+            //Check if the frame is cached
+            frame = cachedFrames.get(pageId);
+            if (frame == null) {
+                //Trying to write unprepared page
+                abort();
+                throw new IllegalStateException("Unprepared compressed-write for page ID: " + pageId);
+            }
+        }
+
+        currentFrame = frame;
+        currentPageId = pageId;
+        return frame;
+    }
+
+    private void writeFullPage() throws HyracksDataException {
+        if (currentFrame.isFull()) {
+            //The LAF page is filled. We do not need to keep it.
+            //Write it to the file and remove it from the cachedFrames map
+            queue.put(currentFrame.cPage, callBack);
+            synchronized (cachedFrames) {
+                //Recycle the frame
+                final LAFFrame frame = cachedFrames.remove(currentPageId);
+                frame.setCachedPage(null);
+                recycledFrames.add(frame);
+            }
+            currentFrame = null;
+            currentPageId = -1;
+        }
+    }
+
+    private int getLAFEntryPageId(int compressedPageId) {
+        return compressedPageId * ENTRY_LENGTH / bufferCache.getPageSize();
+    }
+
+    private class LAFFrame {
+        private ICachedPage cPage;
+        private int numOfEntries;
+        private int maxEntryOffset;
+
+        public void setCachedPage(ICachedPage cPage) {
+            this.cPage = cPage;
+            numOfEntries = 0;
+            maxEntryOffset = -1;
+        }
+
+        public void writePageInfo(int compressedPageId, long offset, long size) {
+            final int entryOffset = compressedPageId * ENTRY_LENGTH % bufferCache.getPageSize();
+            //Put page offset
+            cPage.getBuffer().putLong(entryOffset, offset);
+            //Put page size
+            cPage.getBuffer().putLong(entryOffset + SIZE_ENTRY_OFFSET, size);
+            //Keep the max entry offset to set EOF (if needed)
+            maxEntryOffset = Math.max(maxEntryOffset, entryOffset);
+            numOfEntries++;
+        }
+
+        public void setEOF() {
+            cPage.getBuffer().putLong(maxEntryOffset + ENTRY_LENGTH, EOF);
+        }
+
+        public boolean isFull() {
+            return numOfEntries == maxNumOfEntries;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/NoOpLAFWriter.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/NoOpLAFWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/NoOpLAFWriter.java
new file mode 100644
index 0000000..8f957a9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/NoOpLAFWriter.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.common.compression.file;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+public class NoOpLAFWriter implements ICompressedPageWriter {
+    public static final NoOpLAFWriter INSTACNE = new NoOpLAFWriter();
+
+    private NoOpLAFWriter() {
+    }
+
+    @Override
+    public void prepareWrite(ICachedPage cPage) throws HyracksDataException {
+        //NoOp
+    }
+
+    @Override
+    public void abort() {
+        //NoOp
+    }
+
+    @Override
+    public void endWriting() throws HyracksDataException {
+        //NoOp
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
index 4f15588..11862dc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
@@ -18,18 +18,32 @@
  */
 package org.apache.hyracks.storage.common.file;
 
-import java.util.concurrent.atomic.AtomicInteger;
+import static org.apache.hyracks.storage.common.buffercache.BufferCache.DEBUG;
 
-import org.apache.hyracks.api.io.IFileHandle;
+import java.nio.ByteBuffer;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
 
-public class BufferedFileHandle {
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.common.buffercache.AbstractBufferedFileIOManager;
+import org.apache.hyracks.storage.common.buffercache.BufferCache;
+import org.apache.hyracks.storage.common.buffercache.BufferCacheHeaderHelper;
+import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageReplacementStrategy;
+import org.apache.hyracks.storage.common.compression.file.CompressedFileReference;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+import org.apache.hyracks.storage.common.compression.file.NoOpLAFWriter;
+
+public class BufferedFileHandle extends AbstractBufferedFileIOManager {
     private final int fileId;
-    private volatile IFileHandle handle;
     private final AtomicInteger refCount;
 
-    public BufferedFileHandle(int fileId, IFileHandle handle) {
+    protected BufferedFileHandle(int fileId, BufferCache bufferCache, IIOManager ioManager,
+            BlockingQueue<BufferCacheHeaderHelper> headerPageCache, IPageReplacementStrategy pageReplacementStrategy) {
+        super(bufferCache, ioManager, headerPageCache, pageReplacementStrategy);
         this.fileId = fileId;
-        this.handle = handle;
         refCount = new AtomicInteger();
     }
 
@@ -37,22 +51,6 @@ public class BufferedFileHandle {
         return fileId;
     }
 
-    public void setFileHandle(IFileHandle fileHandle) {
-        this.handle = fileHandle;
-    }
-
-    public IFileHandle getFileHandle() {
-        return handle;
-    }
-
-    public void markAsDeleted() {
-        handle = null;
-    }
-
-    public boolean fileHasBeenDeleted() {
-        return handle == null;
-    }
-
     public int incReferenceCount() {
         return refCount.incrementAndGet();
     }
@@ -69,6 +67,86 @@ public class BufferedFileHandle {
         return getDiskPageId(fileId, pageId);
     }
 
+    @Override
+    public void read(CachedPage cPage) throws HyracksDataException {
+        final BufferCacheHeaderHelper header = checkoutHeaderHelper();
+        try {
+            long bytesRead =
+                    readToBuffer(header.prepareRead(bufferCache.getPageSizeWithHeader()), getFirstPageOffset(cPage));
+
+            if (!verifyBytesRead(bufferCache.getPageSizeWithHeader(), bytesRead)) {
+                return;
+            }
+
+            final ByteBuffer buf = header.processHeader(cPage);
+            cPage.getBuffer().put(buf);
+        } finally {
+            returnHeaderHelper(header);
+        }
+
+        readExtraPages(cPage);
+    }
+
+    private void readExtraPages(CachedPage cPage) throws HyracksDataException {
+        final int totalPages = cPage.getFrameSizeMultiplier();
+        if (totalPages > 1) {
+            pageReplacementStrategy.fixupCapacityOnLargeRead(cPage);
+            cPage.getBuffer().position(bufferCache.getPageSize());
+            cPage.getBuffer().limit(totalPages * bufferCache.getPageSize());
+            readToBuffer(cPage.getBuffer(), getExtraPageOffset(cPage));
+        }
+    }
+
+    @Override
+    protected void write(CachedPage cPage, BufferCacheHeaderHelper header, int totalPages, int extraBlockPageId)
+            throws HyracksDataException {
+        final ByteBuffer buf = cPage.getBuffer();
+        final boolean contiguousLargePages = getPageId(cPage.getDiskPageId()) + 1 == extraBlockPageId;
+        long bytesWritten;
+        try {
+            buf.limit(contiguousLargePages ? bufferCache.getPageSize() * totalPages : bufferCache.getPageSize());
+            buf.position(0);
+            bytesWritten = writeToFile(header.prepareWrite(cPage), getFirstPageOffset(cPage));
+        } finally {
+            returnHeaderHelper(header);
+        }
+
+        if (totalPages > 1 && !contiguousLargePages) {
+            buf.limit(totalPages * bufferCache.getPageSize());
+            bytesWritten += writeToFile(buf, getExtraPageOffset(cPage));
+        }
+
+        final int expectedWritten = bufferCache.getPageSizeWithHeader() + bufferCache.getPageSize() * (totalPages - 1);
+        verifyBytesWritten(expectedWritten, bytesWritten);
+    }
+
+    @Override
+    public int getNumberOfPages() {
+        if (DEBUG) {
+            assert getFileSize() % bufferCache.getPageSizeWithHeader() == 0;
+        }
+        return (int) (getFileSize() / bufferCache.getPageSizeWithHeader());
+    }
+
+    @Override
+    public ICompressedPageWriter getCompressedPageWriter() {
+        return NoOpLAFWriter.INSTACNE;
+    }
+
+    @Override
+    protected long getFirstPageOffset(CachedPage cPage) {
+        return getPageOffset(getPageId(cPage.getDiskPageId()));
+    }
+
+    @Override
+    protected long getExtraPageOffset(CachedPage cPage) {
+        return getPageOffset(cPage.getExtraBlockPageId());
+    }
+
+    private long getPageOffset(long pageId) {
+        return pageId * bufferCache.getPageSizeWithHeader();
+    }
+
     public static long getDiskPageId(int fileId, int pageId) {
         return (((long) fileId) << 32) + pageId;
     }
@@ -80,4 +158,15 @@ public class BufferedFileHandle {
     public static int getPageId(long dpid) {
         return (int) dpid;
     }
+
+    public static BufferedFileHandle create(FileReference fileRef, int fileId, BufferCache bufferCache,
+            IIOManager ioManager, BlockingQueue<BufferCacheHeaderHelper> headerPageCache,
+            IPageReplacementStrategy pageReplacementStrategy) {
+        if (fileRef.isCompressed()) {
+            final CompressedFileReference cFileRef = (CompressedFileReference) fileRef;
+            return new CompressedBufferedFileHandle(fileId, cFileRef.getLAFFileReference(), bufferCache, ioManager,
+                    headerPageCache, pageReplacementStrategy);
+        }
+        return new BufferedFileHandle(fileId, bufferCache, ioManager, headerPageCache, pageReplacementStrategy);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/CompressedBufferedFileHandle.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/CompressedBufferedFileHandle.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/CompressedBufferedFileHandle.java
new file mode 100644
index 0000000..235e144
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/CompressedBufferedFileHandle.java
@@ -0,0 +1,261 @@
+/*
+ * 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.hyracks.storage.common.file;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.BlockingQueue;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.common.buffercache.BufferCache;
+import org.apache.hyracks.storage.common.buffercache.BufferCacheHeaderHelper;
+import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageReplacementStrategy;
+import org.apache.hyracks.storage.common.compression.file.CompressedFileManager;
+import org.apache.hyracks.storage.common.compression.file.CompressedFileReference;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+
+public class CompressedBufferedFileHandle extends BufferedFileHandle {
+    private final FileReference lafFileRef;
+    private volatile CompressedFileManager compressedFileManager;
+
+    protected CompressedBufferedFileHandle(int fileId, FileReference lafFileRef, BufferCache bufferCache,
+            IIOManager ioManager, BlockingQueue<BufferCacheHeaderHelper> headerPageCache,
+            IPageReplacementStrategy pageReplacementStrategy) {
+        super(fileId, bufferCache, ioManager, headerPageCache, pageReplacementStrategy);
+        this.lafFileRef = lafFileRef;
+    }
+
+    @Override
+    public void read(CachedPage cPage) throws HyracksDataException {
+        final BufferCacheHeaderHelper header = checkoutHeaderHelper();
+        try {
+            compressedFileManager.setCompressedPageInfo(cPage);
+            long bytesRead = readToBuffer(header.prepareRead(cPage.getCompressedPageSize()), getFirstPageOffset(cPage));
+
+            if (!verifyBytesRead(cPage.getCompressedPageSize(), bytesRead)) {
+                return;
+            }
+            final ByteBuffer cBuffer = header.processHeader(cPage);
+            final ByteBuffer uBuffer = cPage.getBuffer();
+            fixBufferPointers(uBuffer, 0);
+            if (cPage.getCompressedPageSize() < bufferCache.getPageSizeWithHeader()) {
+                uncompressToPageBuffer(cBuffer, uBuffer);
+            } else {
+                cPage.getBuffer().put(cBuffer);
+            }
+
+            final int totalPages = cPage.getFrameSizeMultiplier();
+            if (totalPages > 1) {
+                pageReplacementStrategy.fixupCapacityOnLargeRead(cPage);
+                readExtraPages(cPage, cBuffer);
+            }
+        } finally {
+            returnHeaderHelper(header);
+        }
+    }
+
+    private void readExtraPages(CachedPage cPage, ByteBuffer cBuffer) throws HyracksDataException {
+        final ByteBuffer uBuffer = cPage.getBuffer();
+
+        final int totalPages = cPage.getFrameSizeMultiplier();
+        for (int i = 1; i < totalPages; i++) {
+            fixBufferPointers(uBuffer, i);
+            compressedFileManager.setExtraCompressedPageInfo(cPage, i - 1);
+            if (cPage.getCompressedPageSize() < bufferCache.getPageSize()) {
+                cBuffer.position(0);
+                cBuffer.limit(cPage.getCompressedPageSize());
+                readToBuffer(cBuffer, getExtraPageOffset(cPage));
+                cBuffer.flip();
+                uncompressToPageBuffer(cBuffer, cPage.getBuffer());
+            } else {
+                readToBuffer(uBuffer, getExtraPageOffset(cPage));
+            }
+        }
+    }
+
+    @Override
+    protected void write(CachedPage cPage, BufferCacheHeaderHelper header, int totalPages, int extraBlockPageId)
+            throws HyracksDataException {
+        try {
+            final ByteBuffer cBuffer = header.prepareWrite(cPage, getRequiredBufferSize());
+            final ByteBuffer uBuffer = cPage.getBuffer();
+            final long pageId = cPage.getDiskPageId();
+
+            final long bytesWritten;
+            final long expectedBytesWritten;
+
+            fixBufferPointers(uBuffer, 0);
+            if (compressToWriteBuffer(uBuffer, cBuffer) < bufferCache.getPageSize()) {
+                cBuffer.position(0);
+                final long offset = compressedFileManager.writePageInfo(pageId, cBuffer.remaining());
+                expectedBytesWritten = cBuffer.limit();
+                bytesWritten = writeToFile(cBuffer, offset);
+            } else {
+                //Compression did not gain any savings
+                final ByteBuffer[] buffers = header.prepareWrite(cPage);
+                final long offset = compressedFileManager.writePageInfo(pageId, bufferCache.getPageSizeWithHeader());
+                expectedBytesWritten = buffers[0].limit() + (long) buffers[1].limit();
+                bytesWritten = writeToFile(buffers, offset);
+            }
+
+            verifyBytesWritten(expectedBytesWritten, bytesWritten);
+
+            //Write extra pages
+            if (totalPages > 1) {
+                writeExtraCompressedPages(cPage, cBuffer, totalPages, extraBlockPageId);
+            }
+
+        } finally {
+            returnHeaderHelper(header);
+        }
+    }
+
+    private void writeExtraCompressedPages(CachedPage cPage, ByteBuffer cBuffer, int totalPages, int extraBlockPageId)
+            throws HyracksDataException {
+        final ByteBuffer uBuffer = cPage.getBuffer();
+        long expectedBytesWritten = 0;
+        long bytesWritten = 0;
+        for (int i = 1; i < totalPages; i++) {
+            fixBufferPointers(uBuffer, i);
+            cBuffer.position(0);
+
+            final ByteBuffer writeBuffer;
+            if (compressToWriteBuffer(uBuffer, cBuffer) < bufferCache.getPageSize()) {
+                writeBuffer = cBuffer;
+            } else {
+                writeBuffer = uBuffer;
+            }
+            final int length = writeBuffer.remaining();
+            final long offset = compressedFileManager.writeExtraPageInfo(extraBlockPageId, length, i - 1);
+            expectedBytesWritten += length;
+            bytesWritten += writeToFile(writeBuffer, offset);
+        }
+
+        verifyBytesWritten(expectedBytesWritten, bytesWritten);
+
+    }
+
+    @Override
+    public void open(FileReference fileRef) throws HyracksDataException {
+        final CompressedFileReference cFileRef = (CompressedFileReference) fileRef;
+        final int lafFileId = bufferCache.openFile(cFileRef.getLAFFileReference());
+
+        compressedFileManager = new CompressedFileManager(bufferCache, lafFileId, cFileRef);
+        compressedFileManager.open();
+        super.open(fileRef);
+    }
+
+    /**
+     * Decrement the reference counter for LAF file.
+     * It is up to {@link BufferCache} to physically close the file.
+     * see {@link BufferCache#deleteFile(FileReference)} and {@link BufferCache#purgeHandle(int)}
+     */
+    @Override
+    public void close() throws HyracksDataException {
+        if (hasBeenOpened()) {
+            compressedFileManager.close();
+            bufferCache.closeFile(compressedFileManager.getFileId());
+        }
+        super.close();
+    }
+
+    @Override
+    public void purge() throws HyracksDataException {
+        super.purge();
+        compressedFileManager.close();
+        bufferCache.closeFile(compressedFileManager.getFileId());
+        bufferCache.purgeHandle(compressedFileManager.getFileId());
+    }
+
+    @Override
+    public void markAsDeleted() throws HyracksDataException {
+        if (hasBeenOpened()) {
+            bufferCache.deleteFile(compressedFileManager.getFileId());
+            compressedFileManager = null;
+        } else {
+            bufferCache.deleteFile(lafFileRef);
+        }
+        super.markAsDeleted();
+    }
+
+    @Override
+    public void force(boolean metadata) throws HyracksDataException {
+        super.force(metadata);
+        bufferCache.force(compressedFileManager.getFileId(), metadata);
+    }
+
+    @Override
+    public int getNumberOfPages() {
+        return compressedFileManager.getNumberOfPages();
+    }
+
+    @Override
+    protected long getFirstPageOffset(CachedPage cPage) {
+        return cPage.getCompressedPageOffset();
+    }
+
+    @Override
+    protected long getExtraPageOffset(CachedPage cPage) {
+        return getFirstPageOffset(cPage);
+    }
+
+    @Override
+    public ICompressedPageWriter getCompressedPageWriter() {
+        return compressedFileManager.getCompressedPageWriter();
+    }
+
+    /* ********************************
+     * Compression methods
+     * ********************************
+     */
+
+    private void fixBufferPointers(ByteBuffer uBuffer, int i) {
+        //Fix the uncompressed buffer to point at the i^th extra page
+        uBuffer.position(bufferCache.getPageSize() * i);
+        //Similarly, fix the limit to a page-worth of data from the i^th page
+        uBuffer.limit(uBuffer.position() + bufferCache.getPageSize());
+    }
+
+    private void uncompressToPageBuffer(ByteBuffer cBuffer, ByteBuffer uBuffer) throws HyracksDataException {
+        final ICompressorDecompressor compDecomp = compressedFileManager.getCompressorDecompressor();
+        compDecomp.uncompress(cBuffer, uBuffer);
+        verifyUncompressionSize(bufferCache.getPageSize(), uBuffer.remaining());
+    }
+
+    private int compressToWriteBuffer(ByteBuffer uBuffer, ByteBuffer cBuffer) throws HyracksDataException {
+        final ICompressorDecompressor compDecomp = compressedFileManager.getCompressorDecompressor();
+        compDecomp.compress(uBuffer, cBuffer);
+        return cBuffer.remaining();
+    }
+
+    private int getRequiredBufferSize() {
+        final ICompressorDecompressor compDecomp = compressedFileManager.getCompressorDecompressor();
+        return compDecomp.computeCompressedBufferSize(bufferCache.getPageSize());
+    }
+
+    private void verifyUncompressionSize(int expected, int actual) {
+        if (expected != actual) {
+            throwException("Uncompressed", expected, actual);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index aee7e90..b8727b0 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -37,6 +37,7 @@ import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.hyracks.util.trace.ITracer;
 import org.junit.After;
 import org.junit.Before;
@@ -62,7 +63,8 @@ public class LSMBTreeExamplesTest extends OrderedIndexExamplesTest {
                 bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(), true,
                 filterTypeTraits, filterCmpFactories, btreeFields, filterFields, true,
-                harness.getMetadataPageManagerFactory(), false, ITracer.NONE);
+                harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
+                NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Before

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 4fafb38..d83f475 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -31,6 +31,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.hyracks.util.trace.ITracer;
 import org.junit.Test;
 
@@ -52,7 +53,8 @@ public class LSMBTreeModificationOperationCallbackTest extends AbstractModificat
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
                 harness.getIOOperationCallbackFactory(), true, null, null, null, null, true,
-                harness.getMetadataPageManagerFactory(), false, ITracer.NONE);
+                harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
+                NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index 3dfb369..59c9ebb 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -40,6 +40,7 @@ import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.hyracks.util.trace.ITracer;
 import org.junit.Assert;
 import org.junit.Test;
@@ -61,7 +62,8 @@ public class LSMBTreeSearchOperationCallbackTest extends AbstractSearchOperation
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
                 harness.getIOOperationCallbackFactory(), true, null, null, null, null, true,
-                harness.getMetadataPageManagerFactory(), false, ITracer.NONE);
+                harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
+                NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
index 0914541..c14474b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
@@ -41,6 +41,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.hyracks.util.trace.ITracer;
 import org.junit.After;
 import org.junit.Assert;
@@ -74,7 +75,8 @@ public class LSMBTreeUpdateInPlaceTest extends AbstractOperationCallbackTest {
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
                 harness.getIOOperationCallbackFactory(), true, null, null, null, null, true,
-                harness.getMetadataPageManagerFactory(), true, ITracer.NONE);
+                harness.getMetadataPageManagerFactory(), true, ITracer.NONE,
+                NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index b25a229..baa95e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -39,6 +39,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
@@ -57,12 +58,13 @@ public class TestLsmBtreeLocalResource extends LSMBTreeLocalResource {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
                 storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, durable);
+                vbcProvider, ioSchedulerProvider, durable, NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     protected TestLsmBtreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
             double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) throws HyracksDataException {
-        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields);
+        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
+                NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
index 6b13f56..3d7c520 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
@@ -32,6 +32,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 public class TestLsmBtreeLocalResourceFactory extends LSMBTreeLocalResourceFactory {
     private static final long serialVersionUID = 1L;
@@ -47,7 +48,7 @@ public class TestLsmBtreeLocalResourceFactory extends LSMBTreeLocalResourceFacto
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                isPrimary, btreeFields);
+                isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index 6950f86..85038c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -32,6 +32,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.common.datagen.ProbabilityHelper;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.hyracks.util.trace.ITracer;
 
 public class LSMBTreeMultiThreadTest extends OrderedIndexMultiThreadTest {
@@ -57,7 +58,8 @@ public class LSMBTreeMultiThreadTest extends OrderedIndexMultiThreadTest {
                 harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, cmpFactories,
                 bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(), true,
-                null, null, null, null, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE);
+                null, null, null, null, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
+                NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 2462c85..3bb1fa4 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -36,9 +36,9 @@ import org.apache.hyracks.storage.am.common.datagen.TupleBatch;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
@@ -48,6 +48,7 @@ import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
 import org.apache.hyracks.test.support.TestUtils;
 import org.apache.hyracks.util.ExitUtil;
@@ -125,7 +126,7 @@ public class LSMTreeRunner implements IExperimentRunner {
                 cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(),
                 new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallbackFactory.INSTANCE, true, null, null,
                 null, null, true, TestStorageManagerComponentHolder.getMetadataPageManagerFactory(), false,
-                ITracer.NONE);
+                ITracer.NONE, NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override


[03/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/compression/ICompressorDecompressor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/compression/ICompressorDecompressor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/compression/ICompressorDecompressor.java
new file mode 100644
index 0000000..3a7e901
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/compression/ICompressorDecompressor.java
@@ -0,0 +1,64 @@
+/*
+ * 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.hyracks.api.compression;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * An API for block compressor/decompressor.
+ *
+ * Note: Should never allocate any buffer in compress/uncompress operations and it must be stateless to be thread safe.
+ */
+public interface ICompressorDecompressor {
+    /**
+     * Computes the required buffer size for <i>compress()</i>.
+     *
+     * @param uBufferSize
+     *            The size of the uncompressed buffer.
+     * @return The required buffer size for compression
+     */
+    int computeCompressedBufferSize(int uBufferSize);
+
+    /**
+     * Compress <i>uBuffer</i> into <i>cBuffer</i>
+     *
+     * @param uBuffer
+     *            Uncompressed source buffer
+     * @param cBuffer
+     *            Compressed destination buffer
+     * @return Buffer after compression. ({@link ByteBuffer#limit()} is set to the compressed size
+     * @throws HyracksDataException
+     */
+    ByteBuffer compress(ByteBuffer uBuffer, ByteBuffer cBuffer) throws HyracksDataException;
+
+    /**
+     * Uncompress <i>cBuffer</i> into <i>uBuffer</i>
+     *
+     * @param cBuffer
+     *            Compressed source buffer
+     * @param uBuffer
+     *            Uncompressed destination buffer
+     * @return Buffer after decompression. ({@link ByteBuffer#limit()} is set to the uncompressed size
+     * @throws HyracksDataException
+     *             An exception will be thrown if the <i>uBuffer</i> size is not sufficient.
+     */
+    ByteBuffer uncompress(ByteBuffer cBuffer, ByteBuffer uBuffer) throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/compression/ICompressorDecompressorFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/compression/ICompressorDecompressorFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/compression/ICompressorDecompressorFactory.java
new file mode 100644
index 0000000..b813afb
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/compression/ICompressorDecompressorFactory.java
@@ -0,0 +1,39 @@
+/*
+ * 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.hyracks.api.compression;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.io.IJsonSerializable;
+
+/**
+ * {@link ICompressorDecompressor} factory.
+ *
+ * New factory of this interface must implement two methods as well if the compression is intended for storage:
+ * - {@link IJsonSerializable#toJson(org.apache.hyracks.api.io.IPersistedResourceRegistry)}
+ * - a static method fromJson(IPersistedResourceRegistry registry, JsonNode json)
+ */
+public interface ICompressorDecompressorFactory extends Serializable, IJsonSerializable {
+    /**
+     * Create a compressor/decompressor instance
+     *
+     * @return {@code ICompressorDecompressor}
+     */
+    ICompressorDecompressor createInstance();
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java
index 4ded855..8b93d07 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java
@@ -27,7 +27,7 @@ import java.util.Date;
  * Used to identify a file in the local Node Controller.
  * Only used for files which are stored inside an IO device.
  */
-public final class FileReference implements Serializable {
+public class FileReference implements Serializable {
     private static final long serialVersionUID = 1L;
     private final File file;
     private final IODeviceHandle dev;
@@ -90,7 +90,11 @@ public final class FileReference implements Serializable {
     }
 
     public FileReference getChild(String name) {
-        return new FileReference(dev, path + File.separator + name);
+        return new FileReference(dev, getChildPath(name));
+    }
+
+    public String getChildPath(String name) {
+        return path + File.separator + name;
     }
 
     public void register() {
@@ -111,4 +115,8 @@ public final class FileReference implements Serializable {
         }
         registrationTime = 0;
     }
+
+    public boolean isCompressed() {
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IPersistedResourceRegistry.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IPersistedResourceRegistry.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IPersistedResourceRegistry.java
index 38162c6..333b373 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IPersistedResourceRegistry.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IPersistedResourceRegistry.java
@@ -46,4 +46,16 @@ public interface IPersistedResourceRegistry {
      * @throws HyracksDataException
      */
     IJsonSerializable deserialize(JsonNode json) throws HyracksDataException;
+
+    /**
+     * This method must be used for optional fields or newly added fields to ensure back-compatibility
+     *
+     * @param json
+     * @param clazz
+     * @return A class object of the type id in {@code json} if exists
+     *         or a class object of type <code>clazz</code> otherwise.
+     * @throws HyracksDataException
+     */
+    IJsonSerializable deserializeOrDefault(JsonNode json, Class<? extends IJsonSerializable> clazz)
+            throws HyracksDataException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index 91acea0..b8cf066 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -29,6 +29,7 @@ import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvid
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IResourceFactory;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 import org.apache.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
 
 public class LSMBTreeOperatorTestHelper extends LSMTreeOperatorTestHelper {
@@ -46,6 +47,6 @@ public class LSMBTreeOperatorTestHelper extends LSMTreeOperatorTestHelper {
                 NoOpIOOperationCallbackFactory.INSTANCE, pageManagerFactory, getVirtualBufferCacheProvider(),
                 SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY, MERGE_POLICY_PROPERTIES, DURABLE,
                 bloomFilterKeyFields, LSMTreeOperatorTestHelper.DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true,
-                btreefields);
+                btreefields, NoOpCompressorDecompressorFactory.INSTANCE);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 4fc8af9..5d2cd26 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -1043,9 +1043,9 @@ public class BTree extends AbstractTreeIndex {
 
                         ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
 
-                        queue.put(leafFrontier.page, this);
+                        putInQueue(leafFrontier.page);
                         for (ICachedPage c : pagesToWrite) {
-                            queue.put(c, this);
+                            putInQueue(c);
                         }
                         pagesToWrite.clear();
                         splitKey.setRightPage(leafFrontier.pageId);
@@ -1152,7 +1152,7 @@ public class BTree extends AbstractTreeIndex {
                 ICachedPage lastLeaf = nodeFrontiers.get(level).page;
                 int lastLeafPage = nodeFrontiers.get(level).pageId;
                 lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
-                queue.put(lastLeaf, this);
+                putInQueue(lastLeaf);
                 nodeFrontiers.get(level).page = null;
                 persistFrontiers(level + 1, lastLeafPage);
                 return;
@@ -1167,7 +1167,7 @@ public class BTree extends AbstractTreeIndex {
             ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
             int finalPageId = freePageManager.takePage(metaFrame);
             frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
-            queue.put(frontier.page, this);
+            putInQueue(frontier.page);
             frontier.pageId = finalPageId;
             persistFrontiers(level + 1, finalPageId);
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
index 97e7ed7..7d43ed8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
@@ -33,6 +33,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
 import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
 public class AppendOnlyLinkedMetadataPageManager implements IMetadataPageManager {
@@ -221,10 +222,13 @@ public class AppendOnlyLinkedMetadataPageManager implements IMetadataPageManager
             }
             int finalMetaPage = getMaxPageId(metaFrame) + 1;
             confiscatedPage.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalMetaPage));
+            final ICompressedPageWriter compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
+            compressedPageWriter.prepareWrite(confiscatedPage);
             // WARNING: flushing the metadata page should be done after releasing the write latch; otherwise, the page
             // won't be flushed to disk because it won't be dirty until the write latch has been released.
             queue.put(confiscatedPage, callback);
             bufferCache.finishQueue();
+            compressedPageWriter.endWriting();
             metadataPage = getMetadataPageId();
             ready = false;
         } else if (confiscatedPage != null) {
@@ -249,7 +253,8 @@ public class AppendOnlyLinkedMetadataPageManager implements IMetadataPageManager
         }
         int pages = bufferCache.getNumPagesOfFile(fileId);
         if (pages == 0) {
-            return 0;
+            //At least there are 2 pages to consider the index is not empty
+            return IBufferCache.INVALID_PAGEID;
         }
         metadataPage = pages - 1;
         return metadataPage;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index b77f14f..f83a27d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -39,6 +39,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
 import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
 public abstract class AbstractTreeIndex implements ITreeIndex {
@@ -241,8 +242,9 @@ public abstract class AbstractTreeIndex implements ITreeIndex {
         // HDFS.  Since loading this tree relies on the root page actually being at that point, no further inserts into
         // that tree are allowed.  Currently, this is not enforced.
         protected boolean releasedLatches;
-        protected final IFIFOPageQueue queue;
+        private final IFIFOPageQueue queue;
         protected List<ICachedPage> pagesToWrite;
+        private final ICompressedPageWriter compressedPageWriter;
 
         public AbstractTreeIndexBulkLoader(float fillFactor) throws HyracksDataException {
             leafFrame = leafFrameFactory.createFrame();
@@ -278,10 +280,12 @@ public abstract class AbstractTreeIndex implements ITreeIndex {
 
             nodeFrontiers.add(leafFrontier);
             pagesToWrite = new ArrayList<>();
+            compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
         }
 
-        protected void handleException() throws HyracksDataException {
+        protected void handleException() {
             // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
+            compressedPageWriter.abort();
             for (NodeFrontier nodeFrontier : nodeFrontiers) {
                 ICachedPage frontierPage = nodeFrontier.page;
                 if (frontierPage.confiscated()) {
@@ -296,10 +300,10 @@ public abstract class AbstractTreeIndex implements ITreeIndex {
 
         @Override
         public void end() throws HyracksDataException {
-            bufferCache.finishQueue();
             if (hasFailed()) {
                 throw HyracksDataException.create(getFailure());
             }
+            bufferCache.finishQueue();
             freePageManager.setRootPageId(rootPage);
         }
 
@@ -320,6 +324,12 @@ public abstract class AbstractTreeIndex implements ITreeIndex {
         public void setLeafFrame(ITreeIndexFrame leafFrame) {
             this.leafFrame = leafFrame;
         }
+
+        public void putInQueue(ICachedPage cPage) throws HyracksDataException {
+            compressedPageWriter.prepareWrite(cPage);
+            queue.put(cPage, this);
+        }
+
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
index 76f7e61..89ccfed 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
@@ -36,6 +36,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
@@ -55,12 +56,13 @@ public class ExternalBTreeLocalResource extends LSMBTreeLocalResource {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
                 storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, null,
-                ioSchedulerProvider, durable);
+                ioSchedulerProvider, durable, NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
             double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) throws HyracksDataException {
-        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields);
+        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
+                NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
index a4c24c9..555f641 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
@@ -30,6 +30,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 public class ExternalBTreeLocalResourceFactory extends LSMBTreeLocalResourceFactory {
 
@@ -46,7 +47,7 @@ public class ExternalBTreeLocalResourceFactory extends LSMBTreeLocalResourceFact
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, null, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                isPrimary, btreeFields);
+                isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
index 2a57e74..ddf0955 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
@@ -36,6 +36,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
@@ -58,13 +59,14 @@ public class ExternalBTreeWithBuddyLocalResource extends LSMBTreeLocalResource {
         super(typeTraits, cmpFactories, buddyBtreeFields, bloomFilterFalsePositiveRate, isPrimary, path, storageManager,
                 mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
                 filterFields, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, null,
-                ioSchedulerProvider, durable);
+                ioSchedulerProvider, durable, NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
             int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields)
             throws HyracksDataException {
-        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields);
+        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
+                NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
index 2aff61a..89e5154 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
@@ -30,6 +30,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 public class ExternalBTreeWithBuddyLocalResourceFactory extends LSMBTreeLocalResourceFactory {
 
@@ -46,7 +47,7 @@ public class ExternalBTreeWithBuddyLocalResourceFactory extends LSMBTreeLocalRes
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, null, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields, bloomFilterFalsePositiveRate,
-                isPrimary, btreeFields);
+                isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index 40278d0..7d5beff 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -40,6 +41,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
@@ -52,6 +54,7 @@ public class LSMBTreeLocalResource extends LsmResource {
     protected final double bloomFilterFalsePositiveRate;
     protected final boolean isPrimary;
     protected final int[] btreeFields;
+    protected final ICompressorDecompressorFactory compressorDecompressorFactory;
 
     public LSMBTreeLocalResource(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, String path,
@@ -60,7 +63,8 @@ public class LSMBTreeLocalResource extends LsmResource {
             IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable) {
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable,
+            ICompressorDecompressorFactory compressorDecompressorFactory) {
         super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable);
@@ -68,15 +72,18 @@ public class LSMBTreeLocalResource extends LsmResource {
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.isPrimary = isPrimary;
         this.btreeFields = btreeFields;
+        this.compressorDecompressorFactory = compressorDecompressorFactory;
     }
 
     protected LSMBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) throws HyracksDataException {
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
+            ICompressorDecompressorFactory compressorDecompressorFactory) throws HyracksDataException {
         super(registry, json);
         this.bloomFilterKeyFields = bloomFilterKeyFields;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.isPrimary = isPrimary;
         this.btreeFields = btreeFields;
+        this.compressorDecompressorFactory = compressorDecompressorFactory;
     }
 
     @Override
@@ -92,7 +99,8 @@ public class LSMBTreeLocalResource extends LsmResource {
                 mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
-                durable, metadataPageManagerFactory, updateAware, serviceCtx.getTracer());
+                durable, metadataPageManagerFactory, updateAware, serviceCtx.getTracer(),
+                compressorDecompressorFactory);
     }
 
     @Override
@@ -108,8 +116,11 @@ public class LSMBTreeLocalResource extends LsmResource {
         final double bloomFilterFalsePositiveRate = json.get("bloomFilterFalsePositiveRate").asDouble();
         final boolean isPrimary = json.get("isPrimary").asBoolean();
         final int[] btreeFields = OBJECT_MAPPER.convertValue(json.get("btreeFields"), int[].class);
+        final JsonNode compressorDecompressorNode = json.get("compressorDecompressorFactory");
+        final ICompressorDecompressorFactory compDecompFactory = (ICompressorDecompressorFactory) registry
+                .deserializeOrDefault(compressorDecompressorNode, NoOpCompressorDecompressorFactory.class);
         return new LSMBTreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary,
-                btreeFields);
+                btreeFields, compDecompFactory);
     }
 
     @Override
@@ -120,5 +131,6 @@ public class LSMBTreeLocalResource extends LsmResource {
         json.put("bloomFilterFalsePositiveRate", bloomFilterFalsePositiveRate);
         json.put("isPrimary", isPrimary);
         json.putPOJO("btreeFields", btreeFields);
+        json.putPOJO("compressorDecompressorFactory", compressorDecompressorFactory.toJson(registry));
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
index 5fae5b9..ea41c3d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
@@ -20,6 +20,7 @@ package org.apache.hyracks.storage.am.lsm.btree.dataflow;
 
 import java.util.Map;
 
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
@@ -40,6 +41,7 @@ public class LSMBTreeLocalResourceFactory extends LsmResourceFactory {
     protected final double bloomFilterFalsePositiveRate;
     protected final boolean isPrimary;
     protected final int[] btreeFields;
+    protected final ICompressorDecompressorFactory compressorDecompressorFactory;
 
     public LSMBTreeLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
@@ -48,7 +50,8 @@ public class LSMBTreeLocalResourceFactory extends LsmResourceFactory {
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) {
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
+            ICompressorDecompressorFactory compressorDecompressorFactory) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable);
@@ -56,6 +59,7 @@ public class LSMBTreeLocalResourceFactory extends LsmResourceFactory {
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.isPrimary = isPrimary;
         this.btreeFields = btreeFields;
+        this.compressorDecompressorFactory = compressorDecompressorFactory;
     }
 
     @Override
@@ -63,6 +67,6 @@ public class LSMBTreeLocalResourceFactory extends LsmResourceFactory {
         return new LSMBTreeLocalResource(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
                 isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory, mergePolicyProperties,
                 filterTypeTraits, filterCmpFactories, btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory,
-                metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable);
+                metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable, compressorDecompressorFactory);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
index 0fc79eb..9169cbf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
@@ -21,12 +21,14 @@ package org.apache.hyracks.storage.am.lsm.btree.impls;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.common.compression.file.CompressedFileReference;
 
 public class LSMBTreeDiskComponent extends AbstractLSMDiskComponent {
     protected final DiskBTree btree;
@@ -80,7 +82,12 @@ public class LSMBTreeDiskComponent extends AbstractLSMDiskComponent {
 
     static Set<String> getFiles(BTree btree) {
         Set<String> files = new HashSet<>();
-        files.add(btree.getFileReference().getFile().getAbsolutePath());
+        final FileReference fileRef = btree.getFileReference();
+        files.add(fileRef.getAbsolutePath());
+        if (fileRef.isCompressed()) {
+            final CompressedFileReference cFileRef = (CompressedFileReference) fileRef;
+            files.add(cFileRef.getLAFAbsolutePath());
+        }
         return files;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
index 2240fd9..ca5f968 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
@@ -27,6 +27,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
@@ -37,6 +38,7 @@ import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManage
 import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
 
 public class LSMBTreeFileManager extends AbstractLSMIndexFileManager {
 
@@ -46,24 +48,30 @@ public class LSMBTreeFileManager extends AbstractLSMIndexFileManager {
     private final boolean hasBloomFilter;
 
     public LSMBTreeFileManager(IIOManager ioManager, FileReference file,
-            TreeIndexFactory<? extends ITreeIndex> btreeFactory, boolean hasBloomFilter) {
-        super(ioManager, file, null);
+            TreeIndexFactory<? extends ITreeIndex> btreeFactory, boolean hasBloomFilter,
+            ICompressorDecompressorFactory compressorDecompressorFactory) {
+        super(ioManager, file, null, compressorDecompressorFactory);
         this.btreeFactory = btreeFactory;
         this.hasBloomFilter = hasBloomFilter;
     }
 
+    public LSMBTreeFileManager(IIOManager ioManager, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> btreeFactory, boolean hasBloomFilter) {
+        this(ioManager, file, btreeFactory, hasBloomFilter, NoOpCompressorDecompressorFactory.INSTANCE);
+    }
+
     @Override
     public LSMComponentFileReferences getRelFlushFileReference() throws HyracksDataException {
         String baseName = getNextComponentSequence(btreeFilter);
-        return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX), null,
-                hasBloomFilter ? baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX) : null);
+        return new LSMComponentFileReferences(getFileReference(baseName + DELIMITER + BTREE_SUFFIX), null,
+                hasBloomFilter ? getFileReference(baseName + DELIMITER + BLOOM_FILTER_SUFFIX) : null);
     }
 
     @Override
     public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName) {
         final String baseName = IndexComponentFileReference.getMergeSequence(firstFileName, lastFileName);
-        return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX), null,
-                hasBloomFilter ? baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX) : null);
+        return new LSMComponentFileReferences(getFileReference(baseName + DELIMITER + BTREE_SUFFIX), null,
+                hasBloomFilter ? getFileReference(baseName + DELIMITER + BLOOM_FILTER_SUFFIX) : null);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index d071bac..70ac3f6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -21,6 +21,7 @@ package org.apache.hyracks.storage.am.lsm.btree.utils;
 
 import java.util.List;
 
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -72,8 +73,8 @@ public class LSMBTreeUtil {
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields, boolean durable,
-            IMetadataPageManagerFactory freePageManagerFactory, boolean updateAware, ITracer tracer)
-            throws HyracksDataException {
+            IMetadataPageManagerFactory freePageManagerFactory, boolean updateAware, ITracer tracer,
+            ICompressorDecompressorFactory compressorDecompressorFactory) throws HyracksDataException {
         LSMBTreeTupleWriterFactory insertTupleWriterFactory =
                 new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, updateAware);
         LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
@@ -106,10 +107,10 @@ public class LSMBTreeUtil {
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
         }
 
-        //Primary LSMBTree index has a BloomFilter.
-        ILSMIndexFileManager fileNameManager =
-                new LSMBTreeFileManager(ioManager, file, diskBTreeFactory, needKeyDupCheck);
+        ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, file, diskBTreeFactory,
+                needKeyDupCheck, compressorDecompressorFactory);
 
+        //Primary LSMBTree index has a BloomFilter.
         ILSMDiskComponentFactory componentFactory;
         ILSMDiskComponentFactory bulkLoadComponentFactory;
         if (needKeyDupCheck) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
index 904029b..7618264 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
@@ -27,6 +27,8 @@ import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
 
+import org.apache.hyracks.api.compression.ICompressorDecompressor;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
@@ -38,6 +40,9 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressor;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
+import org.apache.hyracks.storage.common.compression.file.CompressedFileReference;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 import org.apache.hyracks.util.annotations.NotThreadSafe;
 
@@ -71,6 +76,10 @@ public abstract class AbstractLSMIndexFileManager implements ILSMIndexFileManage
      */
     public static final String DELETE_TREE_SUFFIX = "d";
     /**
+     * Indicates Look Aside File (LAF) for compressed indexes
+     */
+    public static final String LAF_SUFFIX = ".dic";
+    /**
      * Hides transaction components until they are either committed by removing this file or deleted along with the file
      */
     public static final String TXN_PREFIX = ".T";
@@ -88,12 +97,20 @@ public abstract class AbstractLSMIndexFileManager implements ILSMIndexFileManage
     protected final Comparator<IndexComponentFileReference> recencyCmp = new RecencyComparator();
     protected final TreeIndexFactory<? extends ITreeIndex> treeFactory;
     private long lastUsedComponentSeq = UNINITALIZED_COMPONENT_SEQ;
+    private final ICompressorDecompressorFactory compressorDecompressorFactory;
 
     public AbstractLSMIndexFileManager(IIOManager ioManager, FileReference file,
             TreeIndexFactory<? extends ITreeIndex> treeFactory) {
+        this(ioManager, file, treeFactory, NoOpCompressorDecompressorFactory.INSTANCE);
+    }
+
+    public AbstractLSMIndexFileManager(IIOManager ioManager, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> treeFactory,
+            ICompressorDecompressorFactory compressorDecompressorFactory) {
         this.ioManager = ioManager;
         this.baseDir = file;
         this.treeFactory = treeFactory;
+        this.compressorDecompressorFactory = compressorDecompressorFactory;
     }
 
     protected TreeIndexState isValidTreeIndex(ITreeIndex treeIndex) throws HyracksDataException {
@@ -131,7 +148,7 @@ public abstract class AbstractLSMIndexFileManager implements ILSMIndexFileManage
             IBufferCache bufferCache) throws HyracksDataException {
         String[] files = listDirFiles(baseDir, filter);
         for (String fileName : files) {
-            FileReference fileRef = baseDir.getChild(fileName);
+            FileReference fileRef = getFileReference(fileName);
             if (treeFactory == null) {
                 allFiles.add(IndexComponentFileReference.of(fileRef));
                 continue;
@@ -362,6 +379,21 @@ public abstract class AbstractLSMIndexFileManager implements ILSMIndexFileManage
         return IndexComponentFileReference.getFlushSequence(++lastUsedComponentSeq);
     }
 
+    protected FileReference getFileReference(String name) {
+        final ICompressorDecompressor compDecomp = compressorDecompressorFactory.createInstance();
+        //Avoid creating LAF file for NoOpCompressorDecompressor
+        if (compDecomp != NoOpCompressorDecompressor.INSTANCE && isCompressible(name)) {
+            final String path = baseDir.getChildPath(name);
+            return new CompressedFileReference(baseDir.getDeviceHandle(), compDecomp, path, path + LAF_SUFFIX);
+        }
+
+        return baseDir.getChild(name);
+    }
+
+    private boolean isCompressible(String fileName) {
+        return !fileName.endsWith(BLOOM_FILTER_SUFFIX) && !fileName.endsWith(DELETE_TREE_SUFFIX);
+    }
+
     private long getOnDiskLastUsedComponentSequence(FilenameFilter filenameFilter) throws HyracksDataException {
         long maxComponentSeq = -1;
         final String[] files = listDirFiles(baseDir, filenameFilter);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index 635fe7a..b34a13c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -941,9 +941,10 @@ public class RTree extends AbstractTreeIndex {
                     propagateBulk(1, false, pagesToWrite);
 
                     leafFrontier.pageId = freePageManager.takePage(metaFrame);
-                    queue.put(leafFrontier.page, this);
+
+                    putInQueue(leafFrontier.page);
                     for (ICachedPage c : pagesToWrite) {
-                        queue.put(c, this);
+                        putInQueue(c);
                     }
                     pagesToWrite.clear();
                     leafFrontier.page = bufferCache
@@ -974,7 +975,7 @@ public class RTree extends AbstractTreeIndex {
             }
 
             for (ICachedPage c : pagesToWrite) {
-                queue.put(c, this);
+                putInQueue(c);
             }
             finish();
             super.end();
@@ -1011,7 +1012,7 @@ public class RTree extends AbstractTreeIndex {
                     ((RTreeNSMFrame) lowerFrame).adjustMBR();
                     interiorFrameTupleWriter.writeTupleFields(((RTreeNSMFrame) lowerFrame).getMBRTuples(), 0, mbr, 0);
                 }
-                queue.put(n.page, this);
+                putInQueue(n.page);
                 n.page = null;
                 prevPageId = n.pageId;
             }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
index 9c50f2d..423925b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
@@ -66,5 +66,10 @@
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.xerial.snappy</groupId>
+      <artifactId>snappy-java</artifactId>
+      <version>1.1.7.1</version>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AbstractBufferedFileIOManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AbstractBufferedFileIOManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AbstractBufferedFileIOManager.java
new file mode 100644
index 0000000..47e7534
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AbstractBufferedFileIOManager.java
@@ -0,0 +1,287 @@
+/*
+ * 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.hyracks.storage.common.buffercache;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.BlockingQueue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IFileHandle;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.util.IoUtil;
+import org.apache.hyracks.storage.common.compression.file.CompressedFileReference;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+import org.apache.hyracks.util.annotations.NotThreadSafe;
+
+/**
+ * Handles all IO operations for a specified file.
+ */
+@NotThreadSafe
+public abstract class AbstractBufferedFileIOManager {
+    private static final String ERROR_MESSAGE = "%s unexpected number of bytes: [expected: %d, actual: %d, file: %s]";
+    private static final String READ = "Read";
+    private static final String WRITE = "Written";
+
+    protected final BufferCache bufferCache;
+    protected final IPageReplacementStrategy pageReplacementStrategy;
+    private final BlockingQueue<BufferCacheHeaderHelper> headerPageCache;
+    private final IIOManager ioManager;
+
+    private IFileHandle fileHandle;
+    private volatile boolean hasOpen;
+
+    protected AbstractBufferedFileIOManager(BufferCache bufferCache, IIOManager ioManager,
+            BlockingQueue<BufferCacheHeaderHelper> headerPageCache, IPageReplacementStrategy pageReplacementStrategy) {
+        this.bufferCache = bufferCache;
+        this.ioManager = ioManager;
+        this.headerPageCache = headerPageCache;
+        this.pageReplacementStrategy = pageReplacementStrategy;
+        hasOpen = false;
+    }
+
+    /* ********************************
+     * Read/Write page methods
+     * ********************************
+     */
+
+    /**
+     * Read the CachedPage from disk
+     *
+     * @param cPage
+     *            CachedPage in {@link BufferCache}
+     * @throws HyracksDataException
+     */
+    public abstract void read(CachedPage cPage) throws HyracksDataException;
+
+    /**
+     * Write the CachedPage into disk
+     *
+     * @param cPage
+     *            CachedPage in {@link BufferCache}
+     * @throws HyracksDataException
+     */
+    public void write(CachedPage cPage) throws HyracksDataException {
+        final int totalPages = cPage.getFrameSizeMultiplier();
+        final int extraBlockPageId = cPage.getExtraBlockPageId();
+        final BufferCacheHeaderHelper header = checkoutHeaderHelper();
+        write(cPage, header, totalPages, extraBlockPageId);
+    }
+
+    /**
+     * Write the CachedPage into disk called by {@link AbstractBufferedFileIOManager#write(CachedPage)}
+     * Note: It is the responsibility of the caller to return {@link BufferCacheHeaderHelper}
+     *
+     * @param cPage
+     *            CachedPage that will be written
+     * @param header
+     *            HeaderHelper to add into the written page
+     * @param totalPages
+     *            Number of pages to be written
+     * @param extraBlockPageId
+     *            Extra page ID in case it has more than one page
+     * @throws HyracksDataException
+     */
+    protected abstract void write(CachedPage cPage, BufferCacheHeaderHelper header, int totalPages,
+            int extraBlockPageId) throws HyracksDataException;
+
+    /* ********************************
+     * File operations' methods
+     * ********************************
+     */
+
+    /**
+     * Open the file
+     *
+     * @throws HyracksDataException
+     */
+    public void open(FileReference fileRef) throws HyracksDataException {
+        fileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        hasOpen = true;
+    }
+
+    /**
+     * Close the file
+     *
+     * @throws HyracksDataException
+     */
+    public void close() throws HyracksDataException {
+        if (hasOpen) {
+            ioManager.close(fileHandle);
+        }
+    }
+
+    public void purge() throws HyracksDataException {
+        ioManager.close(fileHandle);
+    }
+
+    /**
+     * Force the file into disk
+     *
+     * @param metadata
+     *            see {@link java.nio.channels.FileChannel#force(boolean)}
+     * @throws HyracksDataException
+     */
+    public void force(boolean metadata) throws HyracksDataException {
+        ioManager.sync(fileHandle, metadata);
+    }
+
+    /**
+     * Get the number of pages in the file
+     *
+     * @throws HyracksDataException
+     */
+    public abstract int getNumberOfPages() throws HyracksDataException;
+
+    public void markAsDeleted() throws HyracksDataException {
+        fileHandle = null;
+    }
+
+    /**
+     * Check whether the file has been deleted
+     *
+     * @return
+     *         true if has been deleted, false o.w
+     */
+    public boolean hasBeenDeleted() {
+        return fileHandle == null;
+    }
+
+    /**
+     * Check whether the file has ever been opened
+     *
+     * @return
+     *         true if has ever been open, false o.w
+     */
+    public final boolean hasBeenOpened() {
+        return hasOpen;
+    }
+
+    public final FileReference getFileReference() {
+        return fileHandle.getFileReference();
+    }
+
+    public static void createFile(BufferCache bufferCache, FileReference fileRef) throws HyracksDataException {
+        IoUtil.create(fileRef);
+        if (fileRef.isCompressed()) {
+            final CompressedFileReference cFileRef = (CompressedFileReference) fileRef;
+            try {
+                bufferCache.createFile(cFileRef.getLAFFileReference());
+            } catch (HyracksDataException e) {
+                //In case of creating the LAF file failed, delete fileRef
+                IoUtil.delete(fileRef);
+                throw e;
+            }
+        }
+    }
+
+    public static void deleteFile(FileReference fileRef) throws HyracksDataException {
+        IoUtil.delete(fileRef);
+        if (fileRef.isCompressed()) {
+            final CompressedFileReference cFileRef = (CompressedFileReference) fileRef;
+            if (cFileRef.getFile().exists()) {
+                IoUtil.delete(cFileRef.getLAFFileReference());
+            }
+        }
+    }
+
+    /* ********************************
+     * Compressed file methods
+     * ********************************
+     */
+
+    public abstract ICompressedPageWriter getCompressedPageWriter();
+
+    /* ********************************
+     * Common helper methods
+     * ********************************
+     */
+
+    /**
+     * Get the offset for the first page
+     *
+     * @param cPage
+     *            CachedPage for which the offset is needed
+     * @return
+     *         page offset in the file
+     */
+    protected abstract long getFirstPageOffset(CachedPage cPage);
+
+    /**
+     * Get the offset for the extra page
+     *
+     * @param cPage
+     *            CachedPage for which the offset is needed
+     * @return
+     *         page offset in the file
+     */
+    protected abstract long getExtraPageOffset(CachedPage cPage);
+
+    protected final BufferCacheHeaderHelper checkoutHeaderHelper() {
+        BufferCacheHeaderHelper helper = headerPageCache.poll();
+        if (helper == null) {
+            helper = new BufferCacheHeaderHelper(bufferCache.getPageSize());
+        }
+        return helper;
+    }
+
+    protected final void returnHeaderHelper(BufferCacheHeaderHelper buffer) {
+        headerPageCache.offer(buffer); //NOSONAR
+    }
+
+    protected final long readToBuffer(ByteBuffer buf, long offset) throws HyracksDataException {
+        return ioManager.syncRead(fileHandle, offset, buf);
+    }
+
+    protected final long writeToFile(ByteBuffer buf, long offset) throws HyracksDataException {
+        return ioManager.syncWrite(fileHandle, offset, buf);
+    }
+
+    protected final long writeToFile(ByteBuffer[] buf, long offset) throws HyracksDataException {
+        return ioManager.syncWrite(fileHandle, offset, buf);
+    }
+
+    protected final long getFileSize() {
+        return ioManager.getSize(fileHandle);
+    }
+
+    protected final void verifyBytesWritten(long expected, long actual) {
+        if (expected != actual) {
+            throwException(WRITE, expected, actual);
+        }
+    }
+
+    protected final boolean verifyBytesRead(long expected, long actual) {
+        if (expected != actual) {
+            if (actual == -1) {
+                // disk order scan code seems to rely on this behavior, so silently return
+                return false;
+            } else {
+                throwException(READ, expected, actual);
+            }
+        }
+        return true;
+    }
+
+    protected void throwException(String op, long expected, long actual) {
+        final String path = fileHandle.getFileReference().getAbsolutePath();
+        throw new IllegalStateException(String.format(ERROR_MESSAGE, op, expected, actual, path));
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 1e3f85b..7441395 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -22,7 +22,6 @@ import static org.apache.hyracks.control.nc.io.IOManager.IO_REQUEST_QUEUE_SIZE;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -42,12 +41,12 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IFileHandle;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 import org.apache.hyracks.api.replication.IIOReplicationManager;
 import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.api.util.IoUtil;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 import org.apache.hyracks.storage.common.file.IFileMapManager;
 import org.apache.logging.log4j.Level;
@@ -80,18 +79,18 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
     private final BlockingQueue<BufferCacheHeaderHelper> headerPageCache =
             new ArrayBlockingQueue<>(IO_REQUEST_QUEUE_SIZE);
 
+    private IIOReplicationManager ioReplicationManager;
+    private final List<ICachedPageInternal> cachedPages = new ArrayList<>();
+    private final AtomicLong masterPinCount = new AtomicLong();
+
+    private boolean closed;
+
     //DEBUG
     private static final Level fileOpsLevel = Level.TRACE;
     private ArrayList<CachedPage> confiscatedPages;
     private Lock confiscateLock;
     private HashMap<CachedPage, StackTraceElement[]> confiscatedPagesOwner;
     private ConcurrentHashMap<CachedPage, StackTraceElement[]> pinnedPageOwner;
-    //!DEBUG
-    private IIOReplicationManager ioReplicationManager;
-    private final List<ICachedPageInternal> cachedPages = new ArrayList<>();
-    private final AtomicLong masterPinCount = new AtomicLong();
-
-    private boolean closed;
 
     public BufferCache(IIOManager ioManager, IPageReplacementStrategy pageReplacementStrategy,
             IPageCleanerPolicy pageCleanerPolicy, IFileMapManager fileMapManager, int maxOpenFiles,
@@ -158,7 +157,7 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         synchronized (fileInfoMap) {
             fInfo = fileInfoMap.get(fileId);
         }
-        if (fInfo == null) {
+        if (fInfo == null || fInfo.hasBeenDeleted() || !fInfo.hasBeenOpened()) {
             throw new HyracksDataException("pin called on a fileId " + fileId + " that has not been created.");
         } else if (fInfo.getReferenceCount() <= 0) {
             throw new HyracksDataException("pin called on a fileId " + fileId + " that has not been opened.");
@@ -546,35 +545,9 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
     }
 
     private void read(CachedPage cPage) throws HyracksDataException {
-        BufferedFileHandle fInfo = getFileInfo(cPage);
+        BufferedFileHandle fInfo = getFileHandle(cPage);
         cPage.buffer.clear();
-        BufferCacheHeaderHelper header = checkoutHeaderHelper();
-        try {
-            long bytesRead = ioManager.syncRead(fInfo.getFileHandle(),
-                    getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)), header.prepareRead());
-
-            if (bytesRead != getPageSizeWithHeader()) {
-                if (bytesRead == -1) {
-                    // disk order scan code seems to rely on this behavior, so silently return
-                    return;
-                }
-                throw new HyracksDataException("Failed to read a complete page: " + bytesRead);
-            }
-            int totalPages = header.processRead(cPage);
-
-            if (totalPages > 1) {
-                pageReplacementStrategy.fixupCapacityOnLargeRead(cPage);
-                cPage.buffer.position(pageSize);
-                cPage.buffer.limit(totalPages * pageSize);
-                ioManager.syncRead(fInfo.getFileHandle(), getOffsetForPage(cPage.getExtraBlockPageId()), cPage.buffer);
-            }
-        } finally {
-            returnHeaderHelper(header);
-        }
-    }
-
-    private long getOffsetForPage(long pageId) {
-        return pageId * getPageSizeWithHeader();
+        fInfo.read(cPage);
     }
 
     @Override
@@ -583,67 +556,16 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         pageReplacementStrategy.resizePage((ICachedPageInternal) cPage, totalPages, extraPageBlockHelper);
     }
 
-    BufferedFileHandle getFileInfo(CachedPage cPage) throws HyracksDataException {
-        return getFileInfo(BufferedFileHandle.getFileId(cPage.dpid));
-    }
-
-    BufferedFileHandle getFileInfo(int fileId) throws HyracksDataException {
-        BufferedFileHandle fInfo;
-        synchronized (fileInfoMap) {
-            fInfo = fileInfoMap.get(fileId);
-        }
-        if (fInfo == null) {
-            throw HyracksDataException.create(ErrorCode.FILE_DOES_NOT_EXIST, fileId);
-        }
-        return fInfo;
-    }
-
-    private BufferCacheHeaderHelper checkoutHeaderHelper() {
-        BufferCacheHeaderHelper helper = headerPageCache.poll();
-        if (helper == null) {
-            helper = new BufferCacheHeaderHelper(pageSize);
-        }
-        return helper;
-    }
-
-    private void returnHeaderHelper(BufferCacheHeaderHelper buffer) {
-        headerPageCache.offer(buffer);
-    }
-
     void write(CachedPage cPage) throws HyracksDataException {
-        BufferedFileHandle fInfo = getFileInfo(cPage);
+        BufferedFileHandle fInfo = getFileHandle(cPage);
         // synchronize on fInfo to prevent the file handle from being deleted until the page is written.
         synchronized (fInfo) {
-            if (fInfo.fileHasBeenDeleted()) {
+            if (fInfo.hasBeenDeleted()) {
                 return;
             }
-            ByteBuffer buf = cPage.buffer.duplicate();
-            final int totalPages = cPage.getFrameSizeMultiplier();
-            final int extraBlockPageId = cPage.getExtraBlockPageId();
-            final boolean contiguousLargePages = (BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
-            BufferCacheHeaderHelper header = checkoutHeaderHelper();
-            try {
-                buf.limit(contiguousLargePages ? pageSize * totalPages : pageSize);
-                buf.position(0);
-                long bytesWritten = ioManager.syncWrite(fInfo.getFileHandle(),
-                        getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)), header.prepareWrite(cPage, buf));
-
-                if (bytesWritten != (contiguousLargePages ? pageSize * (totalPages - 1) : 0)
-                        + getPageSizeWithHeader()) {
-                    throw new HyracksDataException("Failed to write completely: " + bytesWritten);
-                }
-            } finally {
-                returnHeaderHelper(header);
-            }
-            if (totalPages > 1 && !contiguousLargePages) {
-                buf.limit(totalPages * pageSize);
-                ioManager.syncWrite(fInfo.getFileHandle(), getOffsetForPage(extraBlockPageId), buf);
-            }
-            if (buf.capacity() != pageSize * totalPages) {
-                throw new IllegalStateException("Illegal number of bytes written, expected bytes written: "
-                        + pageSize * totalPages + " actual bytes writte: " + buf.capacity());
-            }
+            fInfo.write(cPage);
         }
+
     }
 
     @Override
@@ -794,8 +716,8 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         synchronized (fileInfoMap) {
             fileInfoMap.forEach((key, value) -> {
                 try {
-                    sweepAndFlush(key, true);
-                    ioManager.close(value.getFileHandle());
+                    sweepAndFlush(value, true);
+                    value.close();
                 } catch (HyracksDataException e) {
                     if (LOGGER.isWarnEnabled()) {
                         LOGGER.log(Level.WARN, "Error flushing file id: " + key, e);
@@ -811,11 +733,14 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Creating file: " + fileRef + " in cache: " + this);
         }
-        IoUtil.create(fileRef);
+        BufferedFileHandle.createFile(this, fileRef);
+        int fileId;
         try {
             synchronized (fileInfoMap) {
-                return fileMapManager.registerFile(fileRef);
+                fileId = fileMapManager.registerFile(fileRef);
+                getOrCreateFileHandle(fileId);
             }
+            return fileId;
         } catch (Exception e) {
             // If file registration failed for any reason, we need to undo the file creation
             try {
@@ -851,11 +776,14 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         }
         try {
             final BufferedFileHandle fInfo = getOrCreateFileHandle(fileId);
-            if (fInfo.getFileHandle() == null) {
+            //CompressedFileReference may open another file which may sweep and close out this fInfo
+            fInfo.incReferenceCount();
+
+            if (!fInfo.hasBeenOpened()) {
                 // a new file
                 synchronized (fInfo) {
                     // prevent concurrent opening of the same file
-                    if (fInfo.getFileHandle() == null) {
+                    if (!fInfo.hasBeenOpened()) {
                         if (fileInfoMap.size() > maxOpenFiles) {
                             closeOpeningFiles(fInfo);
                         }
@@ -864,15 +792,12 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
                         synchronized (fileInfoMap) {
                             fileRef = fileMapManager.lookupFileName(fileId);
                         }
-                        IFileHandle fh = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-                        fInfo.setFileHandle(fh);
+                        fInfo.open(fileRef);
                     }
                 }
             }
-            fInfo.incReferenceCount();
         } catch (Exception e) {
-            removeFileInfo(fileId);
+            removeFileHandle(fileId);
             throw HyracksDataException.create(e);
         }
     }
@@ -888,11 +813,11 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
                     if (fh != newFileHandle && fh.getReferenceCount() <= 0) {
                         if (fh.getReferenceCount() < 0) {
                             throw new IllegalStateException("Illegal reference count " + fh.getReferenceCount()
-                                    + " of file " + fh.getFileHandle().getFileReference());
+                                    + " of file " + fh.getFileReference());
                         }
                         int entryFileId = entry.getKey();
-                        sweepAndFlush(entryFileId, true);
-                        ioManager.close(entry.getValue().getFileHandle());
+                        sweepAndFlush(fh, true);
+                        entry.getValue().close();
                         fileInfoMap.remove(entryFileId);
                         unreferencedFileFound = true;
                         // for-each iterator is invalid because we changed
@@ -908,7 +833,12 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         }
     }
 
-    private void sweepAndFlush(int fileId, boolean flushDirtyPages) throws HyracksDataException {
+    private void sweepAndFlush(BufferedFileHandle fInfo, boolean flushDirtyPages) throws HyracksDataException {
+        if (!fInfo.hasBeenOpened()) {
+            //Skip flushing as the file has not been open
+            return;
+        }
+        final int fileId = fInfo.getFileId();
         for (final CacheBucket bucket : pageMap) {
             bucket.bucketLock.lock();
             try {
@@ -973,7 +903,7 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
 
         synchronized (fileInfoMap) {
             BufferedFileHandle fInfo = fileInfoMap.get(fileId);
-            if (fInfo == null) {
+            if (fInfo == null || !fInfo.hasBeenOpened()) {
                 throw new HyracksDataException("Closing unopened file");
             }
             if (fInfo.decReferenceCount() < 0) {
@@ -997,7 +927,7 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         synchronized (fileInfoMap) {
             fInfo = fileInfoMap.get(fileId);
         }
-        ioManager.sync(fInfo.getFileHandle(), metadata);
+        fInfo.force(metadata);
     }
 
     @Override
@@ -1013,7 +943,7 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         if (mapped) {
             deleteFile(fileId);
         } else {
-            IoUtil.delete(fileRef);
+            BufferedFileHandle.deleteFile(fileRef);
         }
     }
 
@@ -1022,11 +952,11 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Deleting file: " + fileId + " in cache: " + this);
         }
-        BufferedFileHandle fInfo = removeFileInfo(fileId);
+        BufferedFileHandle fInfo = removeFileHandle(fileId);
         if (fInfo == null) {
             return;
         }
-        sweepAndFlush(fileId, false);
+        sweepAndFlush(fInfo, false);
         try {
             if (fInfo.getReferenceCount() > 0) {
                 throw new HyracksDataException("Deleting open file");
@@ -1040,11 +970,11 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
             } finally {
                 try {
                     synchronized (fInfo) {
-                        ioManager.close(fInfo.getFileHandle());
+                        fInfo.close();
                         fInfo.markAsDeleted();
                     }
                 } finally {
-                    IoUtil.delete(fileRef);
+                    BufferedFileHandle.deleteFile(fileRef);
                 }
             }
         }
@@ -1178,10 +1108,7 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
             if (fInfo == null) {
                 throw new HyracksDataException("No such file mapped for fileId:" + fileId);
             }
-            if (DEBUG) {
-                assert ioManager.getSize(fInfo.getFileHandle()) % getPageSizeWithHeader() == 0;
-            }
-            return (int) (ioManager.getSize(fInfo.getFileHandle()) / getPageSizeWithHeader());
+            return fInfo.getNumberOfPages();
         }
     }
 
@@ -1290,18 +1217,35 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         return null;
     }
 
-    private BufferedFileHandle getOrCreateFileHandle(int fileId) {
+    private BufferedFileHandle getOrCreateFileHandle(int fileId) throws HyracksDataException {
         synchronized (fileInfoMap) {
-            return fileInfoMap.computeIfAbsent(fileId, id -> new BufferedFileHandle(fileId, null));
+            final FileReference fileRef = fileMapManager.lookupFileName(fileId);
+            return fileInfoMap.computeIfAbsent(fileId, id -> BufferedFileHandle.create(fileRef, fileId, this, ioManager,
+                    headerPageCache, pageReplacementStrategy));
         }
     }
 
-    private BufferedFileHandle removeFileInfo(int fileId) {
+    private BufferedFileHandle removeFileHandle(int fileId) {
         synchronized (fileInfoMap) {
             return fileInfoMap.remove(fileId);
         }
     }
 
+    private BufferedFileHandle getFileHandle(CachedPage cPage) throws HyracksDataException {
+        return getFileHandle(BufferedFileHandle.getFileId(cPage.dpid));
+    }
+
+    private BufferedFileHandle getFileHandle(int fileId) throws HyracksDataException {
+        BufferedFileHandle fInfo;
+        synchronized (fileInfoMap) {
+            fInfo = fileInfoMap.get(fileId);
+        }
+        if (fInfo == null) {
+            throw HyracksDataException.create(ErrorCode.FILE_DOES_NOT_EXIST, fileId);
+        }
+        return fInfo;
+    }
+
     private ICachedPage getPageLoop(long dpid, int multiplier, boolean confiscate) throws HyracksDataException {
         final long startingPinCount = DEBUG ? masterPinCount.get() : -1;
         int cycleCount = 0;
@@ -1442,54 +1386,16 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
 
     @Override
     public void purgeHandle(int fileId) throws HyracksDataException {
-        BufferedFileHandle fh = removeFileInfo(fileId);
+        BufferedFileHandle fh = removeFileHandle(fileId);
         if (fh != null) {
             synchronized (fileInfoMap) {
                 fileMapManager.unregisterFile(fileId);
+                fh.purge();
             }
-            ioManager.close(fh.getFileHandle());
         }
 
     }
 
-    static class BufferCacheHeaderHelper {
-        private static final int FRAME_MULTIPLIER_OFF = 0;
-        private static final int EXTRA_BLOCK_PAGE_ID_OFF = FRAME_MULTIPLIER_OFF + 4; // 4
-
-        private final ByteBuffer buf;
-        private final ByteBuffer[] array;
-
-        private BufferCacheHeaderHelper(int pageSize) {
-            buf = ByteBuffer.allocate(RESERVED_HEADER_BYTES + pageSize);
-            array = new ByteBuffer[] { buf, null };
-        }
-
-        private ByteBuffer[] prepareWrite(CachedPage cPage, ByteBuffer pageBuffer) {
-            buf.position(0);
-            buf.limit(RESERVED_HEADER_BYTES);
-            buf.putInt(FRAME_MULTIPLIER_OFF, cPage.getFrameSizeMultiplier());
-            buf.putInt(EXTRA_BLOCK_PAGE_ID_OFF, cPage.getExtraBlockPageId());
-            array[1] = pageBuffer;
-            return array;
-        }
-
-        private ByteBuffer prepareRead() {
-            buf.position(0);
-            buf.limit(buf.capacity());
-            return buf;
-        }
-
-        private int processRead(CachedPage cPage) {
-            buf.position(RESERVED_HEADER_BYTES);
-            cPage.buffer.position(0);
-            cPage.buffer.put(buf);
-            int multiplier = buf.getInt(FRAME_MULTIPLIER_OFF);
-            cPage.setFrameSizeMultiplier(multiplier);
-            cPage.setExtraBlockPageId(buf.getInt(EXTRA_BLOCK_PAGE_ID_OFF));
-            return multiplier;
-        }
-    }
-
     @Override
     public void closeFileIfOpen(FileReference fileRef) {
         synchronized (fileInfoMap) {
@@ -1508,4 +1414,14 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
         }
     }
 
+    @Override
+    public ICompressedPageWriter getCompressedPageWriter(int fileId) {
+        final BufferedFileHandle fInfo;
+        synchronized (fileInfoMap) {
+            fInfo = fileInfoMap.get(fileId);
+        }
+
+        return fInfo.getCompressedPageWriter();
+    }
+
 }
\ No newline at end of file