You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by om...@apache.org on 2016/08/04 21:51:09 UTC

[3/4] orc git commit: ORC-85. Update the C++ library with the newer WriterVersion values. (omalley)

http://git-wip-us.apache.org/repos/asf/orc/blob/0481613d/tools/test/TestMatch.cc
----------------------------------------------------------------------
diff --git a/tools/test/TestMatch.cc b/tools/test/TestMatch.cc
new file mode 100644
index 0000000..40efe53
--- /dev/null
+++ b/tools/test/TestMatch.cc
@@ -0,0 +1,3148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "orc/ColumnPrinter.hh"
+#include "orc/OrcFile.hh"
+
+#include "Adaptor.hh"
+#include "gzip.hh"
+#include "ToolTest.hh"
+
+#include "wrap/gmock.h"
+#include "wrap/gtest-wrapper.h"
+
+#ifdef __clang__
+  DIAGNOSTIC_IGNORE("-Wmissing-variable-declarations")
+#endif
+
+namespace orc {
+
+  class OrcFileDescription {
+  public:
+    std::string filename;
+    std::string json;
+    std::string typeString;
+    std::string formatVersion;
+    uint64_t rowCount;
+    uint64_t contentLength;
+    uint64_t stripeCount;
+    CompressionKind compression;
+    size_t compressionSize;
+    uint64_t rowIndexStride;
+    std::map<std::string, std::string> userMeta;
+
+    OrcFileDescription(const std::string& _filename,
+                       const std::string& _json,
+                       const std::string& _typeString,
+                       const std::string& _version,
+                       uint64_t _rowCount,
+                       uint64_t _contentLength,
+                       uint64_t _stripeCount,
+                       CompressionKind _compression,
+                       size_t _compressionSize,
+                       uint64_t _rowIndexStride,
+                       const std::map<std::string, std::string>& _meta
+                       ): filename(_filename),
+                          json(_json),
+                          typeString(_typeString),
+                          formatVersion(_version),
+                          rowCount(_rowCount),
+                          contentLength(_contentLength),
+                          stripeCount(_stripeCount),
+                          compression(_compression),
+                          compressionSize(_compressionSize),
+                          rowIndexStride(_rowIndexStride),
+                          userMeta(_meta) {
+      // PASS
+    }
+
+    friend std::ostream& operator<< (std::ostream& stream,
+                                     OrcFileDescription const& obj);
+  };
+
+  std::ostream& operator<< (std::ostream& stream,
+                            OrcFileDescription const& obj) {
+    stream << obj.filename;
+    return stream;
+  }
+
+  class FileParam: public testing::TestWithParam<OrcFileDescription> {
+  public:
+    virtual ~FileParam();
+
+    std::string getFilename() {
+      return findExample(GetParam().filename);
+    }
+
+    std::string getJsonFilename() {
+      return findExample("expected/" + GetParam().json);
+    }
+  };
+
+  FileParam::~FileParam() {
+    // PASS
+  }
+
+  TEST_P(FileParam, Metadata) {
+    orc::ReaderOptions opts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(getFilename()), opts);
+
+    EXPECT_EQ(GetParam().compression, reader->getCompression());
+    EXPECT_EQ(GetParam().compressionSize, reader->getCompressionSize());
+    EXPECT_EQ(GetParam().stripeCount, reader->getNumberOfStripes());
+    EXPECT_EQ(GetParam().rowCount, reader->getNumberOfRows());
+    EXPECT_EQ(GetParam().rowIndexStride, reader->getRowIndexStride());
+    EXPECT_EQ(GetParam().contentLength, reader->getContentLength());
+    EXPECT_EQ(GetParam().formatVersion, reader->getFormatVersion());
+    EXPECT_EQ(getFilename(), reader->getStreamName());
+    EXPECT_EQ(GetParam().userMeta.size(), reader->getMetadataKeys().size());
+    for(std::map<std::string, std::string>::const_iterator itr =
+          GetParam().userMeta.begin();
+        itr != GetParam().userMeta.end();
+        ++itr) {
+      ASSERT_EQ(true, reader->hasMetadataValue(itr->first));
+      std::string val = reader->getMetadataValue(itr->first);
+      EXPECT_EQ(itr->second, val);
+    }
+    EXPECT_EQ(true, !reader->hasMetadataValue("foo"));
+    EXPECT_EQ(18446744073709551615UL, reader->getRowNumber());
+
+    EXPECT_EQ(GetParam().typeString, reader->getType().toString());
+  }
+
+  TEST_P(FileParam, Contents) {
+    orc::ReaderOptions opts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(getFilename()), opts);
+    unsigned long rowCount = 0;
+    std::unique_ptr<ColumnVectorBatch> batch = reader->createRowBatch(1024);
+    std::string line;
+    std::unique_ptr<orc::ColumnPrinter> printer =
+      orc::createColumnPrinter(line, &reader->getSelectedType());
+    GzipTextReader expected(getJsonFilename());
+    std::string expectedLine;
+    while (reader->next(*batch)) {
+      EXPECT_EQ(rowCount, reader->getRowNumber());
+      printer->reset(*batch);
+      for(size_t i=0; i < batch->numElements; ++i) {
+        ASSERT_EQ(true, expected.nextLine(expectedLine));
+        line.clear();
+        printer->printRow(i);
+        EXPECT_EQ(expectedLine, line)
+          << "wrong output at row " << (rowCount + i);
+      }
+      rowCount += batch->numElements;
+    }
+    EXPECT_EQ(GetParam().rowCount, rowCount);
+    EXPECT_EQ(GetParam().rowCount, reader->getRowNumber());
+  }
+
+  std::map<std::string, std::string> makeMetadata();
+
+  INSTANTIATE_TEST_CASE_P(TestMatchParam, FileParam,
+    testing::Values(
+                    OrcFileDescription("TestOrcFile.columnProjection.orc",
+                                       "TestOrcFile.columnProjection.jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       21000,
+                                       428406,
+                                       5,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       1000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.emptyFile.orc",
+                                       "TestOrcFile.emptyFile.jsn.gz",
+                                       "struct<boolean1:boolean,byte1:tinyint,"
+                                       "short1:smallint,int1:int,long1:bigint,"
+                                       "float1:float,double1:double,"
+                                       "bytes1:binary,string1:string,"
+                                       "middle:struct<list:array<struct<"
+                                       "int1:int,string1:string>>>,"
+                                       "list:array<struct<int1:int,string1:"
+                                       "string>>,map:map<string,struct<int1:"
+                                       "int,string1:string>>>",
+                                       "0.12",
+                                       0,
+                                       3,
+                                       0,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.metaData.orc",
+                                       "TestOrcFile.metaData.jsn.gz",
+                                       "struct<boolean1:boolean,byte1:tinyint,"
+                                       "short1:smallint,int1:int,long1:bigint,"
+                                       "float1:float,double1:double,"
+                                       "bytes1:binary,string1:string,"
+                                       "middle:struct<list:array<struct<"
+                                       "int1:int,string1:string>>>,"
+                                       "list:array<struct<int1:int,string1:"
+                                       "string>>,map:map<string,struct<int1:"
+                                       "int,string1:string>>>",
+                                       "0.12",
+                                       1,
+                                       980,
+                                       1,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       makeMetadata()),
+                    OrcFileDescription("TestOrcFile.test1.orc",
+                                       "TestOrcFile.test1.jsn.gz",
+                                       "struct<boolean1:boolean,byte1:tinyint,"
+                                       "short1:smallint,int1:int,long1:bigint,"
+                                       "float1:float,double1:double,"
+                                       "bytes1:binary,string1:string,"
+                                       "middle:struct<list:array<struct<"
+                                       "int1:int,string1:string>>>,"
+                                       "list:array<struct<int1:int,string1:"
+                                       "string>>,map:map<string,struct<int1:"
+                                       "int,string1:string>>>",
+                                       "0.12",
+                                       2,
+                                       1015,
+                                       1,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testMemoryManagementV11"
+                                       ".orc",
+                                       "TestOrcFile.testMemoryManagementV11"
+                                       ".jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.11",
+                                       2500,
+                                       18779,
+                                       25,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       0,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testMemoryManagementV12"
+                                       ".orc",
+                                       "TestOrcFile.testMemoryManagementV12"
+                                       ".jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       2500,
+                                       10618,
+                                       4,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       0,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testPredicatePushdown.orc",
+                                       "TestOrcFile.testPredicatePushdown"
+                                       ".jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       3500,
+                                       15529,
+                                       1,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       1000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testSeek.orc",
+                                       "TestOrcFile.testSeek.jsn.gz",
+                                       "struct<boolean1:boolean,byte1:tinyint,"
+                                       "short1:smallint,int1:int,long1:bigint,"
+                                       "float1:float,double1:double,bytes1:"
+                                       "binary,string1:string,middle:struct<"
+                                       "list:array<struct<int1:int,string1:"
+                                       "string>>>,list:array<struct<int1:int,"
+                                       "string1:string>>,map:map<string,"
+                                       "struct<int1:int,string1:string>>>",
+                                       "0.12",
+                                       32768,
+                                       1896379,
+                                       7,
+                                       CompressionKind_ZLIB,
+                                       65536,
+                                       1000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testSnappy.orc",
+                                       "TestOrcFile.testSnappy.jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       10000,
+                                       126061,
+                                       2,
+                                       CompressionKind_SNAPPY,
+                                       100,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testStringAndBinaryStat"
+                                       "istics.orc",
+                                       "TestOrcFile.testStringAndBinaryStat"
+                                       "istics.jsn.gz",
+                                       "struct<bytes1:binary,string1:string>",
+                                       "0.12",
+                                       4,
+                                       185,
+                                       1,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testStripeLevelStats.orc",
+                                       "TestOrcFile.testStripeLevelStats"
+                                       ".jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       11000,
+                                       597,
+                                       3,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testTimestamp.orc",
+                                       "TestOrcFile.testTimestamp.jsn.gz",
+                                       "timestamp",
+                                       "0.11",
+                                       12,
+                                       188,
+                                       1,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testUnionAndTimestamp.orc",
+                                       "TestOrcFile.testUnionAndTimestamp"
+                                       ".jsn.gz",
+                                       "struct<time:timestamp,union:uniontype"
+                                       "<int,string>,decimal:decimal(38,18)>",
+                                       "0.12",
+                                       5077,
+                                       20906,
+                                       2,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testWithoutIndex.orc",
+                                       "TestOrcFile.testWithoutIndex.jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       50000,
+                                       214643,
+                                       10,
+                                       CompressionKind_SNAPPY,
+                                       1000,
+                                       0,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("decimal.orc",
+                                       "decimal.jsn.gz",
+                                       "struct<_col0:decimal(10,5)>",
+                                       "0.12",
+                                       6000,
+                                       16186,
+                                       1,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("demo-11-none.orc",
+                                       "demo-12-zlib.jsn.gz",
+                                       ("struct<_col0:int,_col1:string,"
+                                        "_col2:string,_col3:string,_col4:int,"
+                                        "_col5:string,_col6:int,_col7:int,"
+                                        "_col8:int>"),
+                                       "0.11",
+                                       1920800,
+                                       5069718,
+                                       385,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("demo-11-zlib.orc",
+                                       "demo-12-zlib.jsn.gz",
+                                       ("struct<_col0:int,_col1:string,"
+                                        "_col2:string,_col3:string,_col4:int,"
+                                        "_col5:string,_col6:int,_col7:int,"
+                                        "_col8:int>"),
+                                       "0.11",
+                                       1920800,
+                                       396823,
+                                       385,
+                                       CompressionKind_ZLIB,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("demo-12-zlib.orc",
+                                       "demo-12-zlib.jsn.gz",
+                                       ("struct<_col0:int,_col1:string,"
+                                        "_col2:string,_col3:string,_col4:int,"
+                                        "_col5:string,_col6:int,_col7:int,"
+                                        "_col8:int>"),
+                                       "0.12",
+                                       1920800,
+                                       45592,
+                                       1,
+                                       CompressionKind_ZLIB,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("nulls-at-end-snappy.orc",
+                                       "nulls-at-end-snappy.jsn.gz",
+                                       ("struct<_col0:tinyint,_col1:smallint,"
+                                        "_col2:int,_col3:bigint,_col4:float,"
+                                        "_col5:double,_col6:boolean>"),
+                                       "0.12",
+                                       70000,
+                                       366347,
+                                       1,
+                                       CompressionKind_SNAPPY,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("orc-file-11-format.orc",
+                                       "orc-file-11-format.jsn.gz",
+                                       ("struct<boolean1:boolean,"
+                                        "byte1:tinyint,short1:smallint,"
+                                        "int1:int,long1:bigint,float1:float,"
+                                        "double1:double,bytes1:binary,"
+                                        "string1:string,middle:struct<list:"
+                                        "array<struct<int1:int,"
+                                        "string1:string>>>,list:array<struct"
+                                        "<int1:int,string1:string>>,map:map"
+                                        "<string,struct<int1:int,string1:"
+                                        "string>>,ts:timestamp,"
+                                        "decimal1:decimal(0,0)>"),
+                                       "0.11",
+                                       7500,
+                                       372542,
+                                       2,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("orc_split_elim.orc",
+                                       "orc_split_elim.jsn.gz",
+                                       ("struct<userid:bigint,string1:string,"
+                                        "subtype:double,decimal1:decimal(0,0),"
+                                        "ts:timestamp>"),
+                                       "0.12",
+                                       25000,
+                                       245568,
+                                       5,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("over1k_bloom.orc",
+                                       "over1k_bloom.jsn.gz",
+                                       "struct<_col0:tinyint,_col1:smallint,"
+                                       "_col2:int,_col3:bigint,_col4:float,"
+                                       "_col5:double,_col6:boolean,"
+                                       "_col7:string,_col8:timestamp,"
+                                       "_col9:decimal(4,2),_col10:binary>",
+                                       "0.12",
+                                       2098,
+                                       41780,
+                                       2,
+                                       CompressionKind_ZLIB,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>())
+                    ));
+
+#ifdef HAS_PRE_1970
+INSTANTIATE_TEST_CASE_P(TestMatch1900, FileParam,
+    testing::Values(
+                    OrcFileDescription("TestOrcFile.testDate1900.orc",
+                                       "TestOrcFile.testDate1900.jsn.gz",
+                                       "struct<time:timestamp,date:date>",
+                                       "0.12",
+                                       70000,
+                                       30478,
+                                       8,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>())
+		    ));
+#endif
+
+#ifdef HAS_POST_2038
+  INSTANTIATE_TEST_CASE_P(TestMatch2038, FileParam,
+    testing::Values(
+                    OrcFileDescription("TestOrcFile.testDate2038.orc",
+                                       "TestOrcFile.testDate2038.jsn.gz",
+                                       "struct<time:timestamp,date:date>",
+                                       "0.12",
+                                       212000,
+                                       94762,
+                                       28,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>())
+		    ));
+#endif
+
+  TEST(TestMatch, columnSelectionTest) {
+    ReaderOptions opts;
+    std::list<uint64_t> includes;
+    for(uint64_t i=0; i < 9; i += 2) {
+      includes.push_back(i);
+    }
+    opts.include(includes);
+    std::string filename = findExample("demo-11-none.orc");
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(filename), opts);
+
+    EXPECT_EQ(CompressionKind_NONE, reader->getCompression());
+    EXPECT_EQ(256 * 1024, reader->getCompressionSize());
+    EXPECT_EQ(385, reader->getNumberOfStripes());
+    EXPECT_EQ(1920800, reader->getNumberOfRows());
+    EXPECT_EQ(10000, reader->getRowIndexStride());
+    EXPECT_EQ(5069718, reader->getContentLength());
+    EXPECT_EQ(filename, reader->getStreamName());
+    EXPECT_THAT(reader->getMetadataKeys(), testing::IsEmpty());
+    EXPECT_FALSE(reader->hasMetadataValue("foo"));
+    EXPECT_EQ(18446744073709551615UL, reader->getRowNumber());
+
+    const Type& rootType = reader->getType();
+    EXPECT_EQ(0, rootType.getColumnId());
+    EXPECT_EQ(STRUCT, rootType.getKind());
+    ASSERT_EQ(9, rootType.getSubtypeCount());
+    EXPECT_EQ("_col0", rootType.getFieldName(0));
+    EXPECT_EQ("_col1", rootType.getFieldName(1));
+    EXPECT_EQ("_col2", rootType.getFieldName(2));
+    EXPECT_EQ("_col3", rootType.getFieldName(3));
+    EXPECT_EQ("_col4", rootType.getFieldName(4));
+    EXPECT_EQ("_col5", rootType.getFieldName(5));
+    EXPECT_EQ("_col6", rootType.getFieldName(6));
+    EXPECT_EQ("_col7", rootType.getFieldName(7));
+    EXPECT_EQ("_col8", rootType.getFieldName(8));
+    EXPECT_EQ(INT, rootType.getSubtype(0)->getKind());
+    EXPECT_EQ(STRING, rootType.getSubtype(1)->getKind());
+    EXPECT_EQ(STRING, rootType.getSubtype(2)->getKind());
+    EXPECT_EQ(STRING, rootType.getSubtype(3)->getKind());
+    EXPECT_EQ(INT, rootType.getSubtype(4)->getKind());
+    EXPECT_EQ(STRING, rootType.getSubtype(5)->getKind());
+    EXPECT_EQ(INT, rootType.getSubtype(6)->getKind());
+    EXPECT_EQ(INT, rootType.getSubtype(7)->getKind());
+    EXPECT_EQ(INT, rootType.getSubtype(8)->getKind());
+    for(unsigned int i=0; i < 9; ++i) {
+      EXPECT_EQ(i + 1, rootType.getSubtype(i)->getColumnId())
+        << "fail on " << i;
+    }
+
+    const std::vector<bool> selected = reader->getSelectedColumns();
+    EXPECT_EQ(true, selected[0]) << "fail on " << 0;
+    for (size_t i = 1; i < 10; ++i) {
+      EXPECT_EQ(i%2==1?true:false, selected[i]) << "fail on " << i;
+    }
+
+    unsigned long rowCount = 0;
+    std::unique_ptr<ColumnVectorBatch> batch = reader->createRowBatch(1024);
+    StructVectorBatch* structBatch =
+      dynamic_cast<StructVectorBatch*>(batch.get());
+    ASSERT_TRUE(structBatch != nullptr);
+    LongVectorBatch* longVector = dynamic_cast<LongVectorBatch*>
+      (structBatch->fields[0]);
+    ASSERT_TRUE(longVector != nullptr);
+    int64_t* idCol = longVector->data.data();
+    while (reader->next(*batch)) {
+      EXPECT_EQ(rowCount, reader->getRowNumber());
+      for(unsigned int i=0; i < batch->numElements; ++i) {
+        EXPECT_EQ(rowCount + i + 1, idCol[i]) << "Bad id for " << i;
+      }
+      rowCount += batch->numElements;
+    }
+    EXPECT_EQ(1920800, rowCount);
+    EXPECT_EQ(1920800, reader->getRowNumber());
+  }
+
+  TEST(TestMatch, stripeInformationTest) {
+    ReaderOptions opts;
+    std::string filename = findExample("demo-11-none.orc");
+    std::unique_ptr<Reader> reader = createReader(readLocalFile(filename), opts);
+
+    EXPECT_EQ(385, reader->getNumberOfStripes());
+
+    std::unique_ptr<StripeInformation> stripeInfo = reader->getStripe(7);
+    EXPECT_EQ(92143, stripeInfo->getOffset());
+    EXPECT_EQ(13176, stripeInfo->getLength());
+    EXPECT_EQ(234, stripeInfo->getIndexLength());
+    EXPECT_EQ(12673, stripeInfo->getDataLength());
+    EXPECT_EQ(269, stripeInfo->getFooterLength());
+    EXPECT_EQ(5000, stripeInfo->getNumberOfRows());
+  }
+
+  TEST(TestMatch, readRangeTest) {
+    ReaderOptions fullOpts, lastOpts, oobOpts, offsetOpts;
+    // stripes[N-1]
+    lastOpts.range(5067085, 1);
+    // stripes[N]
+    oobOpts.range(5067086, 4096);
+    // stripes[7, 16]
+    offsetOpts.range(80000, 130722);
+    std::string filename = findExample("demo-11-none.orc");
+    std::unique_ptr<Reader> fullReader =
+      createReader(readLocalFile(filename), fullOpts);
+    std::unique_ptr<Reader> lastReader =
+      createReader(readLocalFile(filename), lastOpts);
+    std::unique_ptr<Reader> oobReader =
+      createReader(readLocalFile(filename), oobOpts);
+    std::unique_ptr<Reader> offsetReader =
+      createReader(readLocalFile(filename), offsetOpts);
+
+    std::unique_ptr<ColumnVectorBatch> oobBatch =
+      oobReader->createRowBatch(5000);
+    EXPECT_FALSE(oobReader->next(*oobBatch));
+
+    // advance fullReader to align with offsetReader
+    std::unique_ptr<ColumnVectorBatch> fullBatch =
+      fullReader->createRowBatch(5000);
+    for (int i=0; i < 7; ++i) {
+      EXPECT_TRUE(fullReader->next(*fullBatch));
+      EXPECT_EQ(5000, fullBatch->numElements);
+    }
+
+    StructVectorBatch *fullStructBatch =
+      dynamic_cast<StructVectorBatch*>(fullBatch.get());
+    ASSERT_TRUE(fullStructBatch != nullptr);
+    LongVectorBatch* fullLongVector =
+      dynamic_cast<LongVectorBatch*>(fullStructBatch->fields[0]);
+    ASSERT_TRUE(fullLongVector != nullptr);
+    int64_t* fullId = fullLongVector->data.data();
+
+    std::unique_ptr<ColumnVectorBatch> offsetBatch =
+      offsetReader->createRowBatch(5000);
+    StructVectorBatch* offsetStructBatch =
+      dynamic_cast<StructVectorBatch*>(offsetBatch.get());
+    ASSERT_TRUE(offsetStructBatch != nullptr);
+    LongVectorBatch* offsetLongVector =
+      dynamic_cast<LongVectorBatch*>(offsetStructBatch->fields[0]);
+    ASSERT_TRUE(offsetLongVector != nullptr);
+    int64_t* offsetId = offsetLongVector->data.data();
+
+    for (int i=7; i < 17; ++i) {
+      EXPECT_TRUE(fullReader->next(*fullBatch));
+      EXPECT_TRUE(offsetReader->next(*offsetBatch));
+      EXPECT_EQ(fullBatch->numElements, offsetBatch->numElements);
+      for (unsigned j=0; j < fullBatch->numElements; ++j) {
+        EXPECT_EQ(fullId[j], offsetId[j]);
+      }
+    }
+    EXPECT_FALSE(offsetReader->next(*offsetBatch));
+
+    // advance fullReader to align with lastReader
+    for (int i=17; i < 384; ++i) {
+      EXPECT_TRUE(fullReader->next(*fullBatch));
+      EXPECT_EQ(5000, fullBatch->numElements);
+    }
+
+    std::unique_ptr<ColumnVectorBatch> lastBatch =
+      lastReader->createRowBatch(5000);
+    StructVectorBatch* lastStructBatch =
+      dynamic_cast<StructVectorBatch*>(lastBatch.get());
+    ASSERT_TRUE(lastStructBatch != nullptr);
+    LongVectorBatch* lastLongVector =
+      dynamic_cast<LongVectorBatch*>(lastStructBatch->fields[0]);
+    ASSERT_TRUE(lastLongVector != nullptr);
+    int64_t* lastId = lastLongVector->data.data();
+
+    EXPECT_TRUE(fullReader->next(*fullBatch));
+    EXPECT_TRUE(lastReader->next(*lastBatch));
+    EXPECT_EQ(fullBatch->numElements, lastBatch->numElements);
+    for (unsigned i=0; i < fullBatch->numElements; ++i) {
+      EXPECT_EQ(fullId[i], lastId[i]);
+    }
+    EXPECT_FALSE(fullReader->next(*fullBatch));
+    EXPECT_FALSE(lastReader->next(*lastBatch));
+  }
+
+TEST(TestMatch, columnStatistics) {
+  orc::ReaderOptions opts;
+  std::string filename = findExample("demo-11-none.orc");
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename), opts);
+
+  // corrupt stats test
+  EXPECT_EQ(true, reader->hasCorrectStatistics());
+
+  // test column statistics
+  std::unique_ptr<orc::Statistics> stats = reader->getStatistics();
+  EXPECT_EQ(10, stats->getNumberOfColumns());
+
+  // 6th real column, start from 1
+  std::unique_ptr<orc::ColumnStatistics> col_6 =
+    reader->getColumnStatistics(6);
+  const orc::StringColumnStatistics* strStats =
+    dynamic_cast<const orc::StringColumnStatistics*> (col_6.get());
+  ASSERT_TRUE(strStats != nullptr);
+
+  EXPECT_EQ("Good", strStats->getMinimum());
+  EXPECT_EQ("Unknown", strStats->getMaximum());
+
+  // 7th real column
+  std::unique_ptr<orc::ColumnStatistics> col_7 =
+    reader->getColumnStatistics(7);
+  const orc::IntegerColumnStatistics* intStats =
+    dynamic_cast<const orc::IntegerColumnStatistics*> (col_7.get());
+  ASSERT_TRUE(intStats != nullptr);
+  EXPECT_EQ(0, intStats->getMinimum());
+  EXPECT_EQ(6, intStats->getMaximum());
+  EXPECT_EQ(5762400, intStats->getSum());
+}
+
+TEST(TestMatch, stripeStatistics) {
+  orc::ReaderOptions opts;
+  std::string filename = findExample("demo-11-none.orc");
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename), opts);
+
+  // test stripe statistics
+  EXPECT_EQ(385, reader->getNumberOfStripeStatistics());
+
+  // stripe[384]: 385th stripe, last stripe
+  unsigned long stripeIdx = 384;
+  std::unique_ptr<orc::Statistics> stripeStats =
+    reader->getStripeStatistics(stripeIdx);
+  EXPECT_EQ(10, stripeStats->getNumberOfColumns());
+
+  // 6th real column
+  const orc::StringColumnStatistics* col_6 =
+    dynamic_cast<const orc::StringColumnStatistics*>
+    (stripeStats->getColumnStatistics(6));
+  ASSERT_TRUE(col_6 != nullptr);
+  EXPECT_EQ("Unknown", col_6->getMinimum());
+  EXPECT_EQ("Unknown", col_6->getMaximum());
+
+  // 7th real column
+  const orc::IntegerColumnStatistics* col_7 =
+    dynamic_cast<const orc::IntegerColumnStatistics*>
+    (stripeStats->getColumnStatistics(7));
+  ASSERT_TRUE(col_7 != nullptr);
+  EXPECT_EQ(6, col_7->getMinimum());
+  EXPECT_EQ(6, col_7->getMaximum());
+  EXPECT_EQ(4800, col_7->getSum());
+}
+
+TEST(TestMatch, corruptStatistics) {
+  orc::ReaderOptions opts;
+  // read the file has corrupt statistics
+  std::string filename = findExample("orc_split_elim.orc");
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename), opts);
+
+  EXPECT_EQ(true, !reader->hasCorrectStatistics());
+
+  // 2nd real column, string
+  std::unique_ptr<orc::ColumnStatistics> col_2 =
+    reader->getColumnStatistics(2);
+  const orc::StringColumnStatistics& strStats =
+    dynamic_cast<const orc::StringColumnStatistics&> (*(col_2.get()));
+  EXPECT_EQ(true, !strStats.hasMinimum());
+  EXPECT_EQ(true, !strStats.hasMaximum());
+
+  // stripe statistics
+  unsigned long stripeIdx = 1;
+  std::unique_ptr<orc::Statistics> stripeStats =
+    reader->getStripeStatistics(stripeIdx);
+
+  // 4th real column, Decimal
+  const orc::DecimalColumnStatistics* col_4 =
+    dynamic_cast<const orc::DecimalColumnStatistics*>
+    (stripeStats->getColumnStatistics(4));
+  ASSERT_TRUE(col_4 != nullptr);
+  EXPECT_EQ(true, !col_4->hasMinimum());
+  EXPECT_EQ(true, !col_4->hasMaximum());
+}
+
+TEST(TestMatch, noStripeStatistics) {
+  orc::ReaderOptions opts;
+  // read the file has no stripe statistics
+  std::string filename = findExample("orc-file-11-format.orc");
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename), opts);
+
+  EXPECT_EQ(0, reader->getNumberOfStripeStatistics());
+}
+
+TEST(TestMatch, seekToRow) {
+  /* Test with a regular file */
+  {
+    orc::ReaderOptions opts;
+    std::string filename = findExample("demo-11-none.orc");
+    std::unique_ptr<orc::Reader> reader =
+        orc::createReader(orc::readLocalFile(filename), opts);
+    EXPECT_EQ(1920800, reader->getNumberOfRows());
+
+    std::unique_ptr<orc::ColumnVectorBatch> batch =
+        reader->createRowBatch(5000); // Stripe size
+    reader->next(*batch);
+    EXPECT_EQ(5000, batch->numElements);
+    EXPECT_EQ(0, reader->getRowNumber());
+
+    // We only load data till the end of the current stripe
+    reader->seekToRow(11000);
+    reader->next(*batch);
+    EXPECT_EQ(4000, batch->numElements);
+    EXPECT_EQ(11000, reader->getRowNumber());
+
+    // We only load data till the end of the current stripe
+    reader->seekToRow(99999);
+    reader->next(*batch);
+    EXPECT_EQ(1, batch->numElements);
+    EXPECT_EQ(99999, reader->getRowNumber());
+
+    // Skip more rows than available
+    reader->seekToRow(1920800);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(1920800, reader->getRowNumber());
+  }
+
+  /* Test with a portion of the file */
+  {
+    orc::ReaderOptions opts;
+    std::string filename = findExample("demo-11-none.orc");
+    opts.range(13126, 13145);   // Read only the second stripe (rows 5000..9999)
+
+    std::unique_ptr<orc::Reader> reader =
+        orc::createReader(orc::readLocalFile(filename), opts);
+    EXPECT_EQ(1920800, reader->getNumberOfRows());
+
+    std::unique_ptr<orc::ColumnVectorBatch> batch =
+        reader->createRowBatch(5000); // Stripe size
+    reader->next(*batch);
+    EXPECT_EQ(5000, batch->numElements);
+
+    reader->seekToRow(7000);
+    reader->next(*batch);
+    EXPECT_EQ(3000, batch->numElements);
+    EXPECT_EQ(7000, reader->getRowNumber());
+
+    reader->seekToRow(1000);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(10000, reader->getRowNumber());
+
+    reader->seekToRow(11000);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(10000, reader->getRowNumber());
+  }
+
+  /* Test with an empty file */
+  {
+    orc::ReaderOptions opts;
+    std::string filename = findExample("TestOrcFile.emptyFile.orc");
+    std::unique_ptr<orc::Reader> reader =
+        orc::createReader(orc::readLocalFile(filename), opts);
+    EXPECT_EQ(0, reader->getNumberOfRows());
+
+    std::unique_ptr<orc::ColumnVectorBatch> batch =
+        reader->createRowBatch(5000);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+
+    reader->seekToRow(0);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(0, reader->getRowNumber());
+
+    reader->seekToRow(1);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(0, reader->getRowNumber());
+  }
+}
+
+TEST(TestMatch, futureFormatVersion) {
+  std::string filename = findExample("version1999.orc");
+  orc::ReaderOptions opts;
+  std::ostringstream errorMsg;
+  opts.setErrorStream(errorMsg);
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename), opts);
+  EXPECT_EQ(("Warning: ORC file " + filename +
+             " was written in an unknown format version 19.99\n"),
+            errorMsg.str());
+  EXPECT_EQ("19.99", reader->getFormatVersion());
+}
+
+TEST(TestMatch, selectColumns) {
+    orc::ReaderOptions opts;
+    std::string filename = findExample("TestOrcFile.testSeek.orc");
+
+    // All columns
+    std::unique_ptr<orc::Reader> reader =
+        orc::createReader(orc::readLocalFile(filename), opts);
+    std::vector<bool> c = reader->getSelectedColumns();
+    EXPECT_EQ(24, c.size());
+    for (unsigned int i=0; i < c.size(); i++) {
+      EXPECT_TRUE(c[i]);
+    }
+    std::unique_ptr<orc::ColumnVectorBatch> batch = reader->createRowBatch(1);
+    std::string line;
+    std::unique_ptr<orc::ColumnPrinter> printer =
+        createColumnPrinter(line, &reader->getSelectedType());
+    reader->next(*batch);
+    printer->reset(*batch);
+    printer->printRow(0);
+    std::ostringstream expected;
+    expected << "{\"boolean1\": true, \"byte1\": -76, "
+        << "\"short1\": 21684, \"int1\": -941468492, "
+        << "\"long1\": -6863419716327549772, \"float1\": 0.7762409, "
+        << "\"double1\": 0.77624090391187, \"bytes1\": [123, 108, 207, 27, 93, "
+        << "157, 139, 233, 181, 90, 14, 60, 34, 120, 26, 119, 231, 50, 155, 121], "
+        << "\"string1\": \"887336a7\", \"middle\": {\"list\": [{\"int1\": "
+        << "-941468492, \"string1\": \"887336a7\"}, {\"int1\": -1598014431, "
+        << "\"string1\": \"ba419d35-x\"}]}, \"list\": [], \"map\": [{\"key\": "
+        << "\"ba419d35-x\", \"value\": {\"int1\": -1598014431, \"string1\": "
+        << "\"ba419d35-x\"}}, {\"key\": \"887336a7\", \"value\": {\"int1\": "
+        << "-941468492, \"string1\": \"887336a7\"}}]}";
+    EXPECT_EQ(expected.str(), line);
+
+    // Int column #2
+    std::list<uint64_t> cols;
+    cols.push_back(1);
+    opts.include(cols);
+    reader = orc::createReader(orc::readLocalFile(filename), opts);
+    c = reader->getSelectedColumns();
+    for (unsigned int i=1; i < c.size(); i++) {
+      if (i==2)
+        EXPECT_TRUE(c[i]);
+      else
+        EXPECT_TRUE(!c[i]);
+    }
+    batch = reader->createRowBatch(1);
+    line.clear();
+    printer = createColumnPrinter(line, &reader->getSelectedType());
+    reader->next(*batch);
+    printer->reset(*batch);
+    printer->printRow(0);
+    std::string expectedInt("{\"byte1\": -76}");
+    EXPECT_EQ(expectedInt, line);
+
+
+    // Struct column #10
+    cols.clear();
+    cols.push_back(9);
+    opts.include(cols);
+    reader = orc::createReader(orc::readLocalFile(filename), opts);
+    c = reader->getSelectedColumns();
+    for (unsigned int i=1; i < c.size(); i++) {
+      if (i>=10 && i<=14)
+        EXPECT_TRUE(c[i]);
+      else
+        EXPECT_TRUE(!c[i]);
+    }
+    batch = reader->createRowBatch(1);
+    line.clear();
+    printer = createColumnPrinter(line, &reader->getSelectedType());
+    reader->next(*batch);
+    printer->reset(*batch);
+    printer->printRow(0);
+    std::ostringstream expectedStruct;
+    expectedStruct << "{\"middle\": {\"list\": "
+        << "[{\"int1\": -941468492, \"string1\": \"887336a7\"}, "
+        << "{\"int1\": -1598014431, \"string1\": \"ba419d35-x\"}]}}";
+    EXPECT_EQ(expectedStruct.str(), line);
+
+    // Array column #11
+    cols.clear();
+    cols.push_back(10);
+    opts.include(cols);
+    reader = orc::createReader(orc::readLocalFile(filename), opts);
+    c = reader->getSelectedColumns();
+    for (unsigned int i=1; i < c.size(); i++) {
+      if (i>=15 && i<=18)
+        EXPECT_TRUE(c[i]);
+      else
+        EXPECT_TRUE(!c[i]);
+    }
+    batch = reader->createRowBatch(1);
+    line.clear();
+    printer = createColumnPrinter(line, &reader->getSelectedType());
+    reader->next(*batch);
+    printer->reset(*batch);
+    printer->printRow(0);
+    std::string expectedArray("{\"list\": []}");
+    EXPECT_EQ(expectedArray, line);
+
+    // Map column #12
+    cols.clear();
+    cols.push_back(11);
+    opts.include(cols);
+    reader = orc::createReader(orc::readLocalFile(filename), opts);
+    c = reader->getSelectedColumns();
+    for (unsigned int i=1; i < c.size(); i++) {
+      if (i>=19 && i<=23)
+        EXPECT_TRUE(c[i]);
+      else
+        EXPECT_TRUE(!c[i]);
+    }
+    batch = reader->createRowBatch(1);
+    line.clear();
+    printer = createColumnPrinter(line, &reader->getSelectedType());
+    reader->next(*batch);
+    printer->reset(*batch);
+    printer->printRow(0);
+    std::ostringstream expectedMap;
+    expectedMap << "{\"map\": [{\"key\": \"ba419d35-x\", \"value\": {\"int1\":"
+        << " -1598014431, \"string1\": \"ba419d35-x\"}}, {\"key\": "
+        << "\"887336a7\", \"value\": {\"int1\": -941468492, \"string1\": "
+        << "\"887336a7\"}}]}";
+    EXPECT_EQ(expectedMap.str(), line);
+}
+
+TEST(TestMatch, memoryUse) {
+  std::string filename = findExample("TestOrcFile.testSeek.orc");
+  std::unique_ptr<orc::Reader> reader;
+  std::unique_ptr<orc::ColumnVectorBatch> batch;
+  orc::ReaderOptions opts;
+  std::list<uint64_t> cols;
+
+  // Int column
+  cols.push_back(1);
+  opts.include(cols);
+  reader = orc::createReader(orc::readLocalFile(filename), opts);
+  EXPECT_EQ(483517, reader->getMemoryUse());
+  batch = reader->createRowBatch(1);
+  EXPECT_EQ(10, batch->getMemoryUsage());
+  batch = reader->createRowBatch(1000);
+  EXPECT_EQ(10000, batch->getMemoryUsage());
+  EXPECT_FALSE(batch->hasVariableLength());
+
+  // Binary column
+  cols.clear();
+  cols.push_back(7);
+  opts.include(cols);
+  reader = orc::createReader(orc::readLocalFile(filename), opts);
+  EXPECT_EQ(835906, reader->getMemoryUse());
+  batch = reader->createRowBatch(1);
+  EXPECT_EQ(18, batch->getMemoryUsage());
+  EXPECT_FALSE(batch->hasVariableLength());
+
+  // String column
+  cols.clear();
+  cols.push_back(8);
+  opts.include(cols);
+  reader = orc::createReader(orc::readLocalFile(filename), opts);
+  EXPECT_EQ(901442, reader->getMemoryUse());
+  batch = reader->createRowBatch(1);
+  EXPECT_EQ(18, batch->getMemoryUsage());
+  EXPECT_FALSE(batch->hasVariableLength());
+
+  // Struct column (with a List subcolumn)
+  cols.clear();
+  cols.push_back(9);
+  opts.include(cols);
+  reader = orc::createReader(orc::readLocalFile(filename), opts);
+  EXPECT_EQ(1294658, reader->getMemoryUse());
+  batch = reader->createRowBatch(1);
+  EXPECT_EQ(46, batch->getMemoryUsage());
+  EXPECT_TRUE(batch->hasVariableLength());
+
+  // List column
+   cols.clear();
+   cols.push_back(10);
+   opts.include(cols);
+   reader = orc::createReader(orc::readLocalFile(filename), opts);
+   EXPECT_EQ(1229122, reader->getMemoryUse());
+   batch = reader->createRowBatch(1);
+   EXPECT_EQ(45, batch->getMemoryUsage());
+   EXPECT_TRUE(batch->hasVariableLength());
+
+  // Map column
+  cols.clear();
+  cols.push_back(11);
+  opts.include(cols);
+  reader = orc::createReader(orc::readLocalFile(filename), opts);
+  EXPECT_EQ(1491266, reader->getMemoryUse());
+  batch = reader->createRowBatch(1);
+  EXPECT_EQ(62, batch->getMemoryUsage());
+  EXPECT_TRUE(batch->hasVariableLength());
+
+  // All columns
+  cols.clear();
+  for(uint64_t c=0; c < 12; ++c) {
+    cols.push_back(c);
+  }
+  opts.include(cols);
+  reader = orc::createReader(orc::readLocalFile(filename), opts);
+  EXPECT_EQ(4112706, reader->getMemoryUse());
+  batch = reader->createRowBatch(1);
+  EXPECT_EQ(248, batch->getMemoryUsage());
+  EXPECT_TRUE(batch->hasVariableLength());
+}
+
+  std::map<std::string, std::string> makeMetadata() {
+    std::map<std::string, std::string> result;
+    result["my.meta"] = "\x01\x02\x03\x04\x05\x06\x07\xff\xfe\x7f\x80";
+    result["clobber"] = "\x05\x07\x0b\x0d\x11\x13";
+    const unsigned char buffer[] = {
+96,180,32,187,56,81,217,212,122,203,147,61,190,112,57,155,246,201,45,163,
+58,240,29,79,183,112,233,140,3,37,244,29,62,186,248,152,109,167,18,200,
+43,205,77,85,75,240,181,64,35,194,155,98,77,233,239,156,47,147,30,252,
+88,15,154,251,8,27,18,225,7,177,232,5,242,180,245,240,241,208,12,45,
+15,98,99,70,112,146,28,80,88,103,255,32,246,168,51,94,152,175,135,37,
+56,85,134,180,31,239,242,5,180,224,90,0,8,35,247,139,95,143,92,2,
+67,156,232,246,122,120,29,144,203,230,191,26,231,242,188,64,164,151,9,160,
+108,14,49,73,155,240,41,105,202,66,210,3,229,102,188,198,150,222,8,250,
+1,2,160,253,46,35,48,176,150,74,187,124,68,48,32,222,28,173,9,191,
+214,56,31,251,148,218,175,187,144,196,237,145,160,97,58,209,220,75,71,3,
+175,132,193,214,59,26,135,105,33,198,213,134,157,97,204,185,142,209,58,230,
+192,154,19,252,145,225,73,34,243,1,207,139,207,147,67,21,166,4,157,47,
+7,217,131,250,169,27,143,78,114,101,236,184,21,167,203,171,193,69,12,183,
+43,60,116,16,119,23,170,36,172,66,242,91,108,103,132,118,125,14,53,70,
+196,247,37,1,145,163,182,170,162,182,77,18,110,85,131,176,76,17,50,89,
+201,72,225,208,179,155,185,86,12,213,64,155,110,202,254,219,200,172,175,238,
+167,77,183,248,90,223,148,190,154,133,161,221,75,3,170,136,131,29,210,156,
+64,120,129,11,58,40,210,45,102,128,182,79,203,177,178,55,194,68,18,52,
+206,171,191,218,216,124,49,21,72,246,121,2,116,185,46,106,89,29,58,177,
+166,11,115,64,11,196,116,197,45,60,188,242,251,174,114,182,230,212,159,176,
+177,133,19,54,250,44,84,12,223,191,120,200,219,73,44,101,231,91,1,242,
+86,10,157,196,86,254,164,3,66,134,86,158,48,134,234,100,151,36,149,156,
+68,8,146,218,235,114,76,6,229,19,58,201,170,148,16,238,186,45,84,254,
+138,253,248,80,125,33,19,226,2,106,147,122,228,57,152,43,206,121,204,36,
+14,38,74,246,205,67,204,48,37,102,107,185,179,127,129,231,20,21,103,173,
+104,201,130,73,128,219,236,205,20,219,32,168,112,88,1,113,21,182,16,82,
+57,102,75,118,131,96,156,24,178,230,82,218,235,107,179,120,187,63,120,128,
+92,187,129,182,154,249,239,253,13,66,111,145,188,18,34,248,23,137,195,144,
+219,40,101,90,30,2,29,96,158,157,25,12,222,85,168,201,35,237,85,18,
+244,72,205,0,24,151,115,7,95,78,212,28,87,207,209,123,139,190,52,2,
+21,185,209,248,226,186,91,123,35,200,54,253,59,89,143,89,220,21,119,196,
+157,151,25,184,177,147,91,128,45,32,152,163,112,27,106,65,71,222,102,86,
+183,253,36,79,11,33,22,190,57,233,40,217,234,84,4,186,183,105,85,13,
+246,140,86,48,227,219,154,2,253,245,209,49,130,27,150,90,189,48,247,209,
+133,115,52,22,177,69,12,237,253,3,174,217,74,188,227,213,6,248,240,0,
+139,40,121,189,39,22,210,78,95,141,155,182,188,127,23,136,50,15,115,77,
+90,104,55,235,130,241,252,79,85,60,247,70,138,139,90,240,208,165,207,223,
+54,19,46,197,250,49,33,156,0,163,227,139,104,148,156,232,107,6,11,38,
+177,188,99,11,39,114,53,250,170,178,143,165,54,168,3,82,2,136,46,127,
+132,245,39,53,8,50,183,129,196,69,203,125,221,220,75,133,165,11,85,51,
+102,209,201,93,140,62,231,96,186,105,70,122,120,4,111,141,220,91,55,180,
+196,21,73,55,1,233,160,82,217,81,160,70,186,207,251,2,21,167,243,244,
+173,73,119,179,108,45,221,194,204,113,207,190,145,114,114,189,43,62,119,155,
+172,71,73,123,222,80,46,60,228,51,229,145,135,85,152,99,68,212,96,24,
+130,253,85,233,201,56,209,85,202,2,218,42,157,135,181,148,16,122,0,1,
+192,42,13,152,157,54,208,67,183,89,5,15,237,166,143,228,253,63,51,193,
+49,111,45,76,60,34,122,56,16,175,204,109,163,7,81,95,98,98,3,176,
+210,97,62,97,194,56,147,104,49,69,75,174,26,166,97,90,176,204,132,43,
+164,94,248,171,60,143,223,88,243,250,139,189,116,106,229,216,246,180,249,228,
+94,165,148,214,32,29,120,148,50,95,204,0,21,223,192,130,110,177,133,10,
+141,63,221,79,208,177,227,165,69,121,76,138,241,231,3,157,67,149,29,249,
+144,163,34,5,177,90,23,157,207,59,205,105,17,141,244,6,237,108,194,224,
+175,115,99,176,73,25,78,173,104,163,90,161,171,115,1,77,71,204,93,209,
+42,92,79,248,134,238,185,189,41,78,155,81,245,102,165,161,60,245,208,105,
+215,200,156,80,249,39,109,202,174,11,23,192,253,242,228,5,151,61,178,48,
+178,91,35,105,32,63,92,31,146,225,101,237,187,27,154,182,4,86,70,58,
+62,234,219,238,252,181,158,168,17,211,164,79,12,22,202,150,251,117,143,135,
+137,184,169,5,166,127,209,45,232,222,164,137,84,95,39,29,140,34,175,77,
+103,199,22,175,142,238,38,204,148,135,22,97,80,99,131,209,10,110,169,151,
+217,77,22,13,211,196,203,240,73,64,176,65,46,195,189,136,228,13,47,11,
+191,118,213,54,140,68,243,158,192,78,111,85,155,134,217,132,158,35,35,64,
+128,51,239,49,161,95,76,188,142,106,237,81,147,97,85,23,213,114,117,58,
+133,96,185,67,8,196,113,114,31,144,76,48,181,159,167,115,30,23,58,76,
+96,47,183,19,234,37,43,194,58,195,128,33,120,49,237,11,142,230,42,181,
+195,150,8,22,31,218,88,209,166,197,104,228,0,114,22,181,229,21,222,136,
+185,163,236,240,158,167,236,21,174,18,105,188,124,184,6,136,39,236,158,185,
+97,185,201,18,108,57,229,44,103,188,64,45,200,197,71,247,94,153,43,226,
+126,159,221,223,62,247,181,89,237,101,57,238,24,83,100,252,113,212,82,2,
+149,177,109,147,207,152,105,10,6,246,175,154,40,85,251,150,130,114,234,69,
+195,0,42,61,185,54,96,131,52,128,205,92,92,127,218,241,171,148,200,158,
+68,126,190,55,105,251,67,90,197,19,234,232,175,17,21,97,215,11,245,4,
+173,94,10,192,235,149,50,70,45,84,95,166,173,12,54,171,19,56,73,242,
+10,75,178,73,237,203,77,225,40,206,97,16,39,189,165,91,52,80,236,57,
+153,127,42,236,57,110,219,35,139,189,122,217,84,219,136,154,107,83,56,173,
+5,174,77,186,194,170,21,139,112,8,202,77,40,135,137,120,197,202,23,160,
+75,201,27,31,45,183,41,9,76,159,235,57,237,128,52,122,241,222,232,63,
+152,60,185,23,134,45,12,10,144,157,235,181,97,242,249,234,35,237,35,111,
+102,44,61,28,2,194,192,209,253,239,139,149,236,194,193,154,84,226,118,213,
+190,61,24,172,239,71,191,90,69,164,55,115,196,127,160,116,138,34,53,88,
+127,217,14,187,112,14,247,68,167,236,7,143,216,246,193,190,169,191,249,242,
+170,40,199,52,36,121,132,5,111,170,71,38,234,210,164,180,106,131,157,235,
+135,101,71,54,74,177,12,176,90,244,49,71,234,7,173,234,121,117,24,120,
+192,104,16,9,255,117,216,51,230,219,245,49,113,2,236,60,228,42,117,147,
+95,52,171,205,163,61,9,247,106,65,163,38,46,180,237,84,86,53,174,234,
+50,156,96,35,214,175,158,104,90,191,232,24,42,224,166,12,245,111,215,209,
+210,219,213,190,144,251,127,171,220,34,47,75,98,151,203,109,154,251,166,62,
+196,192,221,122,192,24,69,112,6,60,96,212,62,8,196,49,95,38,31,138,
+79,6,22,154,205,131,155,149,149,79,202,223,30,96,238,152,60,190,92,33,
+128,146,215,95,114,177,108,178,58,133,107,0,196,195,152,152,142,159,131,176,
+21,252,231,249,201,37,184,182,74,190,228,38,14,36,186,17,228,27,252,246,
+100,82,220,128,34,18,136,206,4,101,253,176,91,18,28,220,8,250,1,205,
+172,178,200,244,226,221,187,184,63,232,49,140,65,194,109,87,165,129,63,171,
+82,79,82,16,25,15,115,201,132,189,106,89,185,207,66,76,50,118,89,133,
+226,229,148,205,220,163,208,244,91,210,31,73,224,174,105,177,250,84,120,93,
+201,113,34,31,217,34,21,251,182,8,95,129,95,181,94,0,100,145,189,230,
+170,154,141,156,216,141,204,42,26,119,41,53,199,241,111,89,10,33,60,152,
+44,195,245,177,252,58,73,34,171,176,77,8,200,61,174,60,169,164,145,66,
+138,83,24,22,81,58,5,119,94,133,244,213,213,161,10,104,53,225,167,56,
+166,3,123,47,66,50,93,193,136,94,35,75,206,253,197,124,161,66,100,147,
+123,127,46,98,245,59,32,43,56,171,118,79,240,72,42,95,118,152,19,231,
+234,237,17,60,94,121,128,210,210,100,214,137,25,50,200,151,86,160,221,90,
+103,23,73,227,70,108,96,79,24,33,188,70,12,113,99,214,40,68,77,138,
+101,86,184,171,136,129,41,116,198,65,236,27,218,209,66,36,12,135,133,239,
+177,67,173,116,107,38,20,224,12,177,170,11,189,176,28,218,39,50,27,94,
+148,243,246,100,218,54,132,91,198,112,215,96,27,197,67,199,76,177,184,134,
+95,18,50,161,163,9,7,1,238,86,243,75,23,246,219,8,103,165,180,191,
+160,223,109,201,6,142,215,218,53,110,116,69,105,180,100,152,194,155,193,184,
+229,15,113,192,39,19,100,107,179,161,148,48,231,24,114,74,119,209,4,38,
+114,215,181,62,231,167,45,95,44,127,15,32,170,152,225,230,162,202,16,38,
+165,199,193,164,142,49,108,86,68,131,131,150,145,249,106,214,137,92,226,178,
+211,113,61,216,240,166,104,208,233,142,211,66,88,141,22,144,170,222,199,158,
+153,30,7,60,230,247,159,125,117,11,204,227,17,48,83,106,250,24,134,117,
+96,17,192,218,153,213,79,44,246,213,242,209,117,188,224,123,8,161,185,12,
+194,241,131,199,33,93,231,66,36,244,50,168,241,158,11,215,108,82,226,88,
+80,190,182,109,46,122,93,37,82,19,190,189,254,90,172,51,163,251,87,224,
+91,179,47,148,137,140,235,74,135,183,41,186,58,176,176,251,233,47,27,4,
+99,113,93,212,169,123,208,193,28,118,105,197,159,229,93,196,154,179,36,93,
+213,188,91,219,90,70,8,40,35,119,205,58,158,166,172,69,184,27,117,44,
+115,55,142,64,136,28,63,229,152,254,93,0,244,240,232,89,250,61,200,68,
+43,1,45,198,171,59,196,61,19,17,164,204,118,211,88,30,190,42,243,165,
+124,5,136,201,184,91,53,140,136,167,122,246,165,61,11,5,244,58,181,53,
+22,138,124,158,83,191,218,235,128,178,79,42,68,186,235,11,166,240,210,168,
+23,167,234,248,8,81,196,7,41,37,134,104,20,40,146,186,31,154,241,131,
+30,4,235,121,113,37,178,2,121,209,167,46,221,196,45,37,33,91,137,182,
+189,245,41,218,207,233,36,97,82,5,197,64,127,76,52,57,135,50,247,55,
+161,96,170,49,179,205,86,36,142,13,61,147,102,55,163,207,2,230,123,139,
+73,13,247,80,156,19,243,194,144,140,185,137,191,35,29,59,152,67,116,132,
+68,146,220,248,160,40,197,139,215,213,236,118,195,33,73,94,3,11,200,105,
+154,138,57,37,43,118,116,159,46,94,188,55,10,194,174,63,43,240,68,96,
+113,111,90,196,101,158,183,233,85,44,137,66,52,44,153,145,110,11,80,135,
+60,155,8,224,251,170,179,26,137,225,12,167,100,144,51,150,54,227,77,127,
+200,39,147,89,245,37,207,106,200,65,50,108,42,223,2,171,8,103,14,216,
+129,209,3,4,56,56,61,142,35,253,52,42,165,34,106,158,245,253,62,190,
+171,68,223,116,136,37,166,237,116,66,99,235,159,122,186,99,233,82,177,171,
+124,222,190,95,203,197,67,34,82,56,136,18,62,255,141,240,135,193,244,31,
+86,50,100,78,177,241,176,135,106,83,124,209,117,39,112,238,42,156,84,127,
+173,147,94,4,219,222,84,43,126,46,6,84,26,155,5,209,75,42,38,149,
+29,158,163,43,166,126,74,92,73,193,250,95,170,85,125,10,145,101,124,249,
+102,112,206,197,0,103,21,104,68,139,163,7,92,195,30,194,140,121,217,133,
+194,223,31,38,116,167,7,99,162,140,161,162,95,4,65,171,156,108,46,230,
+55,147,182,156,244,137,173,70,6,21,196,61,192,127,108,164,54,2,232,100,
+6,247,209,72,41,97,177,254,84,37,66,80,3,141,138,74,165,201,116,146,
+216,189,228,34,135,165,38,180,163,171,0,112,231,243,11,131,172,7,147,151,
+230,179,50,44,76,84,84,23,163,21,108,87,43,233,41,225,236,191,213,51,
+111,66,63,127,35,81,204,165,112,35,251,135,223,58,55,189,247,41,194,179,
+24,55,230,64,11,61,193,151,217,230,5,164,211,90,229,239,205,253,147,31,
+37,187,133,95,36,131,4,31,178,152,225,14,159,119,190,29,232,72,79,45,
+203,105,196,40,13,43,152,105,172,124,143,98,29,113,128,166,109,242,243,149,
+106,105,45,182,203,187,244,181,22,206,53,241,84,174,175,47,221,71,9,243,
+187,38,202,15,205,211,124,158,197,209,162,55,210,167,181,184,158,109,194,156,
+77,152,42,7,153,94,180,91,10,113,74,39,167,61,151,123,80,62,172,23,
+12,228,195,68,227,182,147,240,18,55,114,167,41,230,160,192,212,231,209,160,
+219,109,30,48,84,206,26,46,228,72,167,244,117,76,29,42,86,117,28,10,
+148,140,132,176,171,97,199,53,211,72,128,171,159,249,216,161,89,99,28,182,
+106,201,130,200,8,197,236,35,44,114,130,92,131,7,235,245,182,168,28,38,
+119,48,145,200,37,192,159,149,127,135,141,255,234,99,84,213,33,37,44,236,
+39,49,218,171,182,144,100,176,147,101,220,102,55,108,166,131,68,106,30,103,
+69,24,110,98,13,171,136,147,239,55,112,177,78,127,178,56,243,90,68,121,
+109,247,198,235,154,166,151,27,85,186,255,74,95,104,248,201,161,208,172,212,
+246,226,91,209,109,215,178,21,121,173,45,179,214,170,5,197,192,117,97,189,
+180,184,189,139,51,50,63,167,25,230,141,134,141,156,225,118,146,214,241,151,
+119,126,142,113,64,130,45,67,218,185,119,47,66,240,154,154,91,90,167,221,
+11,173,207,21,97,50,205,73,145,192,245,218,146,221,226,73,133,142,162,208,
+174,75,149,56,230,186,214,74,37,130,231,122,176,247,92,113,240,223,79,210,
+129,20,145,189,99,46,11,17,96,215,221,245,20,251,98,150,7,158,110,125,
+22,106,119,215,103,153,83,186,15,181,136,16,61,49,13,113,247,50,63,198,
+242,211,87,108,123,221,143,201,20,183,254,59,98,36,153,175,50,231,77,190,
+101,76,67,13,229,9,85,176,77,230,103,212,62,115,25,26,100,236,52,86,
+252,55,147,193,118,23,102,224,55,200,43,116,238,209,102,214,90,234,17,249,
+219,38,234,97,172,180,53,78,230,243,157,88,182,243,124,56,111,208,137,183,
+64,86,189,213,120,89,207,110,197,75,236,172,10,49,44,158,138,208,173,156,
+41,209,209,119,187,250,122,4,160,165,148,47,60,14,241,154,41,133,139,182,
+16,185,98,213,181,233,95,219,134,240,129,70,235,0,116,80,135,71,203,72,
+16,41,16,6,188,144,103,141,247,123,184,53,107,195,232,96,174,248,91,202,
+53,120,240,48,254,125,217,107,27,123,244,249,249,106,191,83,118,242,142,101,
+46,48,118,1,201,6,64,20,58,179,141,93,144,221,37,13,200,239,106,184,
+126,169,90,51,75,12,181,50,198,156,167,100,238,245,173,167,96,62,61,23,
+144,214,77,175,114,17,223,119,193,40,174,65,55,54,241,184,38,33,41,9,
+40,190,84,41,171,105,165,43,45,105,107,248,93,18,34,199,27,120,192,134,
+248,244,144,151,213,192,155,133,100,227,56,67,191,74,154,142,171,166,151,162,
+110,254,62,63,66,99,181,64,176,113,99,111,143,56,207,237,103,33,16,7,
+13,53,250,160,209,140,106,191,139,205,206,236,38,50,203,40,85,61,145,166,
+184,114,92,178,58,94,142,125,4,117,75,48,224,242,140,193,73,69,37,11,
+187,218,247,92,156,73,43,247,156,134,225,201,160,172,187,122,197,21,136,28,
+69,58,236,219,209,22,20,63,52,92,170,113,253,204,170,220,220,255,17,80,
+149,97,205,127,56,70,202,176,139,98,94,78,198,125,3,16,133,170,192,130,
+68,33,226,79,234,27,97,66,52,250,48,131,177,199,46,244,52,36,6,149,
+255,249,242,77,169,22,47,149,216,137,88,115,80,28,221,15,109,126,84,26,
+184,76,106,53,212,243,178,48,173,65,158,150,39,242,219,173,18,243,14,251,
+87,245,226,67,210,109,237,65,70,1,0,25,192,14,254,217,215,156,173,142,
+157,152,241,126,27,192,79,56,172,116,103,148,198,244,100,243,103,191,91,14,
+136,109,61,203,101,255,37,151,6,0,41,137,102,154,210,249,139,134,34,168,
+34,221,130,45,156,137,215,129,83,196,153,129,27,255,56,221,7,58,209,175,
+163,187,252,203,76,56,145,248,55,180,174,173,167,202,163,63,130,177,193,121,
+251,73,20,207,59,107,181,126,201,250,113,80,97,121,245,107,246,123,13,108,
+116,242,194,115,111,196,154,44,216,56,32,49,104,216,6,73,247,54,90,0,
+19,115,15,169,233,104,87,235,241,175,84,88,174,99,239,129,51,215,208,157,
+48,93,133,15,203,226,242,117,179,93,6,195,16,174,13,147,88,138,69,13,
+45,24,139,82,224,49,204,106,177,65,89,122,209,63,154,58,243,135,156,94,
+166,71,97,247,46,133,93,157,191,200,215,129,126,58,36,16,29,109,19,46,
+42,49,220,7,53,40,108,69,216,226,174,233,93,16,36,121,160,164,6,62,
+162,228,233,35,28,183,171,204,11,58,168,151,227,141,108,178,139,150,47,226,
+20,235,105,220,54,29,213,70,223,159,61,20,122,46,237,216,35,177,160,241,
+99,80,229,45,164,215,154,167,54,38,99,33,101,229,251,225,230,3,240,179,
+37,22,120,154,51,17,71,34,20,199,37,154,115,201,204,114,60,18,87,233,
+75,64,124,124,45,100,104,205,244,192,179,120,240,230,48,100,174,108,6,23,
+83,155,135,66,156,62,243,84,250,193,115,189,23,158,135,197,242,215,30,36,
+59,62,17,48,133,118,78,7,64,192,249,203,166,236,219,49,211,55,64,174,
+54,248,219,102,152,155,103,24,222,175,162,225,108,41,116,169,107,2,93,29,
+50,37,112,39,188,62,59,242,36,234,221,151,198,111,206,229,255,89,154,21,
+63,10,231,78,137,26,96,177,236,5,15,93,63,25,47,189,164,18,87,122,
+42,70,242,43,137,185,175,20,52,178,177,1,8,245,63,186,107,10,28,153,
+151,174,181,168,25,25,103,150,82,175,110,129,153,86,251,6,224,120,69,215,
+81,247,230,223,79,62,11,215,62,25,230,210,193,96,163,18,227,10,177,184,
+29,122,228,212,181,238,232,123,47,176,108,58,38,137,116,65,205,212,114,142,
+0,165,105,109,255,75,170,68,184,120,94,214,68,223,245,108,216,168,77,247,
+97,29,157,82,180,54,159,78,97,184,234,20,5,176,113,22,173,15,41,108,
+35,155,166,209,100,202,87,108,240,52,65,127,98,166,203,95,178,112,153,129,
+141,122,76,42,117,151,17,146,106,111,87,200,165,51,122,163,3,175,185,44,
+254,246,205,150,162,213,125,61,12,76,170,166,247,78,178,132,205,25,190,64,
+154,23,197,60,51,55,8,198,9,33,113,248,152,245,3,13,152,102,32,110,
+94,84,66,248,234,171,115,136,90,173,16,100,227,113,60,214,181,72,61,122,
+66,253,207,23,103,24,11,105,61,226,242,32,125,46,232,152,211,122,229,153,
+193,38,47,79,176,137,123,4,136,3,200,30,221,33,231,102,170,182,206,181,
+141,75,26,106,238,10,39,64,196,216,191,135,122,175,17,149,177,0,117,103,
+253,21,108,86,240,182,183,190,178,117,28,105,10,220,172,132,109,232,232,170,
+244,102,75,199,80,12,33,148,89,19,11,115,216,43,174,234,241,6,21,27,
+100,119,80,3,57,208,200,94,104,166,230,66,60,43,184,125,227,51,231,115,
+95,173,118,216,209,168,34,47,62,231,184,140,186,216,232,205,34,68,8,224,
+130,231,23,124,155,0,64,156,38,53,204,198,61,142,204,152,22,51,233,40,
+150,176,76,181,66,128,154,208,212,31,228,82,16,196,76,154,180,151,246,166,
+225,109,25,135,180,125,82,224,200,13,31,229,158,199,104,193,157,60,193,173,
+223,19,216,25,159,76,91,236,75,7,105,24,47,17,189,19,70,144,150,78,
+121,67,82,56,94,181,204,146,1,51,69,152,216,165,89,97,108,220,138,13,
+204,236,110,182,1,254,135,38,187,123,122,27,180,160,63,152,193,207,81,19,
+184,157,79,254,216,76,244,207,234,59,39,105,196,28,207,150,154,229,223,122,
+157,251,164,157,117,126,101,224,24,114,90,119,159,218,104,213,253,157,28,78,
+64,147,101,15,101,227,153,4,18,194,31,1,187,69,11,19,173,92,53,153,
+152,197,191,140,168,201,217,173,232,136,140,223,200,44,5,230,7,213,90,241,
+211,156,32,8,170,142,132,48,192,76,238,200,215,25,128,17,88,123,106,212,
+62,164,58,48,146,7,86,98,161,46,132,31,31,165,164,187,63,158,153,233,
+6,44,106,108,83,205,175,115,73,47,216,210,39,138,62,231,223,57,143,69,
+84,198,181,189,135,150,158,179,22,116,2,22,77,174,103,117,235,197,110,74,
+169,162,156,144,210,68,52,10,166,35,159,71,225,139,193,188,117,111,143,21,
+15,252,131,22,85,56,27,147,100,121,228,77,186,174,224,46,239,202,240,187,
+18,113,223,55,45,153,143,196,164,119,116,246,203,210,31,95,83,138,16,25,
+229,133,132,146,80,55,180,123,51,140,153,105,1,237,86,222,196,210,67,78,
+162,150,255,64,51,26,200,216,111,58,167,15,123,218,29,82,86,105,229,129,
+216,13,113,9,135,35,115,211,84,24,71,20,85,250,19,97,242,93,9,97,
+69,255,179,216,150,111,118,155,73,73,17,76,26,151,234,102,105,212,99,151,
+45,5,22,187,115,13,193,90,246,135,92,74,104,202,16,218,138,143,117,215,
+187,145,199,106,187,237,214,151,163,53,130,109,200,64,247,79,8,190,53,192,
+243,228,18,164,218,178,37,79,15,175,30,49,209,225,219,46,127,34,188,158,
+227,5,110,227,61,95,251,252,244,19,102,47,223,56,14,61,150,175,180,185,
+194,213,222,147,198,59,62,210,177,109,170,238,11,111,202,118,203,81,232,27,
+106,23,93,157,119,227,182,200,7,34,114,105,162,150,238,33,102,211,244,250,
+157,27,32,253,178,184,196,127,84,103,242,163,115,7,117,186,185,20,226,73,
+142,48,52,57,147,152,202,186,148,44,17,29,20,137,206,78,164,202,206,97,
+222,220,100,200,105,224,239,39,104,17,154,235,255,240,194,239,222,174,4,104,
+196,137,186,65,235,47,82,116,94,127,55,7,19,254,155,128,124,50,170,12,
+140,135,250,187,36,99,147,169,175,129,239,76,129,170,167,162,123,195,47,60,
+147,198,138,215,193,213,242,155,48,72,226,211,239,242,146,81,57,59,247,79,
+150,198,230,3,233,217,161,90,35,93,168,247,169,151,227,171,92,110,134,246,
+221,156,77,134,160,17,1,64,110,124,139,105,7,9,54,42,44,77,235,36,
+194,184,54,16,60,180,153,191,119,162,240,93,46,163,246,58,116,104,181,100,
+105,227,239,126,218,2,224,57,16,210,162,198,61,129,199,83,155,58,245,62,
+79,125,92,183,163,219,156,146,128,215,19,129,193,32,17,230,203,132,200,182,
+112,205,126,70,255,172,254,77,70,8,210,224,64,43,150,2,15,59,145,13,
+160,62,98,75,132,227,143,5,251,238,2,35,184,137,121,136,116,234,27,189,
+118,236,92,235,77,122,248,231,26,225,72,81,244,49,55,28,178,179,14,9,
+109,57,71,36,107,213,72,60,141,28,80,177,149,170,84,31,156,234,89,153,
+192,30,32,32,93,190,90,162,191,156,112,142,138,140,207,194,157,241,186,78,
+63,231,149,45,193,204,120,72,248,78,91,240,208,204,7,219,5,62,75,147,
+21,168,82,11,158,105,32,176,122,208,82,55,3,76,95,49,78,70,16,157,
+78,253,185,222,109,242,22,226,212,60,216,34,143,39,74,137,88,99,125,18,
+189,163,189,140,88,86,170,234,65,230,177,169,48,143,135,119,145,187,58,14,
+251,108,133,40,0,208,177,222,78,102,12,104,57,187,251,67,61,216,78,112,
+213,153,184,65,85,50,105,18,133,191,126,203,109,171,167,61,170,163,15,185,
+119,145,233,178,84,179,86,223,183,54,151,204,250,126,13,208,23,137,64,76,
+107,151,167,16,97,20,110,203,199,172,93,165,175,158,173,69,135,243,126,155,
+81,70,118,80,209,198,49,111,243,26,224,242,180,196,25,142,210,39,73,244,
+4,136,191,193,54,219,40,54,223,124,106,90,151,55,236,31,49,104,178,98,
+13,85,173,59,170,231,70,243,255,253,97,128,1,191,110,235,49,176,84,178,
+127,8,207,164,154,147,108,168,37,162,111,231,169,26,82,255,231,69,27,23,
+235,139,19,121,210,242,231,10,40,187,189,49,28,118,81,156,111,72,87,58,
+108,78,98,211,179,229,223,22,220,160,37,37,222,247,151,235,129,214,162,151,
+134,118,232,195,127,138,196,33,176,197,214,78,248,3,201,40,56,119,6,97,
+76,92,115,174,226,253,91,95,137,249,222,139,29,114,114,138,7,254,16,196,
+249,40,181,27,112,9,231,147,235,23,64,104,206,244,90,162,70,100,170,232,
+92,98,79,200,159,27,242,195,36,219,160,69,108,122,250,246,173,23,191,197,
+161,104,21,29,78,198,174,192,31,128,161,100,223,248,174,86,38,23,215,170,
+221,44,104,167,46,249,101,180,64,236,115,193,213,194,218,153,140,34,18,12,
+194,201,164,87,233,192,135,17,172,39,205,209,75,253,228,177,231,119,20,151,
+235,171,56,125,37,41,137,111,155,183,106,32,76,129,188,27,199,228,182,246,
+148,167,99,149,28,164,176,229,70,201,229,169,36,21,251,170,210,199,148,141,
+100,70,168,22,26,99,149,136,187,143,119,209,31,145,251,176,17,101,124,150,
+123,60,115,81,149,130,49,24,68,5,227,112,33,169,92,204,81,108,18,3,
+120,152,254,247,248,22,62,3,128,94,147,180,187,41,169,178,158,189,118,215,
+132,34,66,232,226,41,222,250,87,175,162,238,168,251,240,143,169,204,157,20,
+210,199,59,114,192,142,85,25,210,160,221,81,193,107,4,103,11,41,238,134,
+88,56,205,206,102,38,63,44,35,95,164,0,6,128,145,131,70,187,61,226,
+117,138,38,113,204,248,160,144,212,19,30,215,194,1,165,3,197,16,47,158,
+58,221,14,249,45,236,179,48,143,13,141,140,5,159,64,79,155,253,123,98,
+200,170,108,204,201,248,22,240,71,40,159,130,39,144,250,183,236,37,97,211,
+224,190,174,126,174,219,192,58,201,170,56,35,70,152,213,45,228,17,68,133,
+102,10,204,107,4,100,164,220,253,121,88,162,2,192,228,186,212,248,56,107,
+20,158,54,9,3,87,205,244,78,109,89,162,14,15,35,135,83,224,124,92,
+174,54,30,222,194,165,45,56,141,28,152,115,222,226,28,92,134,104,227,237,
+134,139,152,116,33,163,235,69,120,176,146,232,96,202,250,139,113,165,75,212,
+36,93,255,91,174,130,148,175,171,219,4,142,207,145,129,7,242,114,209,222,
+58,188,149,101,9,50,79,237,100,161,254,219,152,19,180,79,172,165,148,183,
+33,35,25,178,177,158,81,129,202,100,153,117,70,208,244,90,179,183,152,202,
+117,69,195,246,26,254,25,247,40,67,226,32,236,59,243,59,118,46,71,28,
+103,99,139,85,172,214,215,139,251,0,190,122,128,51,169,219,147,50,223,232,
+19,98,165,202,227,35,209,36,173,143,94,201,120,248,90,242,96,159,248,194,
+146,109,127,72,231,63,18,149,27,225,195,24,132,233,53,184,51,160,227,170,
+171,103,120,141,34,183,152,82,186,56,156,56,191,197,36,121,15,190,108,65,
+131,230,167,205,52,1,130,20,218,83,152,180,5,71,28,22,126,118,246,29,
+247,215,46,224,220,56,194,192,225,220,58,47,204,46,200,115,2,2,176,170,
+45,38,160,97,24,78,73,183,24,92,232,200,181,81,68,89,213,171,174,97,
+74,19,249,234,174,231,85,147,113,246,221,73,82,249,235,6,129,225,6,0,
+114,182,108,62,123,34,255,53,172,87,5,142,187,156,181,85,18,131,209,78,
+37,171,32,164,199,176,25,128,165,110,167,87,127,167,4,248,47,7,32,199,
+247,183,76,227,169,5,107,35,152,137,211,15,106,79,36,91,217,135,11,214,
+42,64,239,161,40,159,39,171,158,37,64,164,96,201,156,1,14,200,111,48,
+18,61,45,116,173,165,16,23,46,184,163,183,124,98,145,188,202,165,187,146,
+247,196,247,180,36,152,222,80,230,247,140,245,7,11,139,32,0,196,146,182,
+60,112,222,38,73,152,180,196,182,118,92,226,80,150,243,149,149,30,255,255,
+179,255,141,2,49,207,2,217,154,100,46,225,207,25,196,9,5,95,114,119,
+176,106,123,32,32,141,58,211,152,115,47,43,250,108,91,207,255,229,55,35,
+85,198,56,135,135,240,237,3,103,3,130,172,113,50,162,251,195,71,38,151,
+40,73,140,223,196,204,242,99,20,15,255,192,47,29,31,37,87,232,150,26,
+234,197,129,222,37,97,98,174,250,7,208,27,209,186,198,99,30,152,205,79,
+198,31,172,133,201,146,218,241,7,228,114,42,77,15,144,240,238,198,187,44,
+104,59,244,98,53,108,168,198,153,48,137,41,160,133,115,99,235,214,179,122,
+179,59,198,35,20,181,38,229,95,155,3,37,22,148,142,118,139,247,140,53,
+202,236,225,87,224,152,137,17,254,63,184,144,75,147,119,86,4,78,106,6,
+1,120,150,85,239,176,176,186,27,172,42,6,203,52,83,53,250,190,233,86,
+144,123,130,214,17,1,22,213,171,150,95,20,209,34,223,215,69,86,85,167,
+195,156,68,176,184,174,238,183,19,93,239,216,212,206,56,175,29,132,13,27,
+214,229,184,105,192,40,124,200,74,54,50,151,117,254,31,45,52,123,60,167,
+134,154,42,72,31,231,226,201,220,202,131,175,24,177,45,172,50,72,142,35,
+152,73,202,54,183,214,214,83,225,129,54,18,99,243,26,233,108,58,121,6,
+141,73,151,115,74,33,142,137,128,171,121,195,170,183,66,133,33,250,225,210,
+235,205,176,39,235,154,185,193,217,247,79,252,195,160,250,167,5,183,4,55,
+118,243,163,47,96,79,188,70,226,90,116,88,126,250,192,203,21,174,253,39,
+40,136,191,251,230,83,38,123,42,255,55,228,213,17,249,79,127,143,51,62,
+60,167,129,84,75,175,246,68,13,54,196,8,63,158,44,255,178,207,154,30,
+91,87,62,24,210,243,64,41,47,63,41,131,4,56,138,128,8,45,119,36,
+175,250,197,1,168,215,209,161,181,192,236,246,127,194,149,250,149,252,143,125,
+123,150,149,205,35,79,134,2,237,233,77,28,16,37,133,84,18,237,236,220,
+213,185,149,119,66,205,145,222,187,79,92,139,212,246,231,160,23,166,138,145,
+55,62,132,207,246,177,92,203,206,114,49,254,201,42,195,160,146,130,241,3,
+213,235,1,45,143,51,21,213,114,76,0,124,168,32,210,199,91,180,27,229,
+82,41,163,4,155,166,157,226,35,112,143,99,231,148,191,73,98,106,246,26,
+90,214,137,208,5,243,249,164,238,103,214,254,186,221,57,83,44,122,194,94,
+236,186,93,139,234,142,118,224,235,107,122,191,8,219,239,51,151,94,33,74,
+145,19,215,190,66,218,240,114,176,146,233,29,168,67,32,1,74,40,183,243,
+41,180,107,31,73,195,174,31,171,64,185,69,220,187,216,182,205,230,35,112,
+46,180,241,212,86,219,150,95,104,20,228,69,1,243,204,243,0,232,255,93,
+18,167,198,240,153,165,245,14,97,217,91,57,1,0,205,65,186,135,9,102,
+25,150,118,35,186,36,14,221,162,0,120,153,161,220,158,202,70,168,243,13,
+169,182,236,71,89,203,7,254,217,215,104,126,37,122,173,112,129,58,156,89,
+184,225,148,191,178,133,104,102,150,80,216,126,114,98,7,197,187,197,227,225,
+129,72,116,210,201,225,48,207,230,82,141,162,97,139,212,198,162,204,243,215,
+240,210,238,70,46,224,32,80,125,111,65,184,191,184,152,228,53,221,116,247,
+93,130,107,191,205,135,47,34,81,158,187,237,238,205,169,250,116,35,209,133,
+75,224,209,16,102,238,120,197,15,214,228,80,191,43,96,128,195,153,16,205,
+165,246,33,174,224,220,10,227,125,112,139,104,165,221,74,115,143,42,82,107,
+239,201,33,107,198,232,65,244,28,187,35,100,7,61,57,126,71,250,121,176,
+169,76,210,214,94,65,151,19,19,206,174,100,225,84,221,57,148,2,71,94,
+69,107,150,48,228,176,172,16,90,178,54,11,248,254,200,76,31,176,61,108,
+19,14,25,28,230,139,27,220,45,122,94,165,193,77,163,100,45,230,153,187,
+132,37,205,76,146,160,68,73,226,159,119,209,113,184,191,38,115,167,229,70,
+248,160,113,181,165,24,131,9,215,81,109,28,227,218,56,241,64,189,190,208,
+119,174,178,36,84,8,63,164,240,125,183,98,66,65,16,81,27,211,28,72,
+92,230,19,1,203,98,159,185,240,131,203,178,2,72,118,77,168,138,255,137,
+57,206,49,10,186,86,73,96,223,99,15,199,57,192,19,102,65,201,204,13,
+52,135,245,104,123,157,231,1,155,18,221,208,33,90,219,223,246,86,178,223,
+197,111,238,209,110,226,93,249,129,216,219,228,196,13,33,160,44,235,101,182,
+34,190,7,224,235,208,250,31,237,9,223,48,49,136,204,222,1,3,235,249,
+121,43,180,96,247,242,216,191,55,15,249,64,114,10,81,60,207,137,211,106,
+124,242,203,179,157,169,108,208,123,103,117,124,75,16,72,83,78,23,39,13,
+14,112,140,218,135,248,86,119,123,14,207,2,225,70,253,62,23,10,181,4,
+105,209,225,21,207,58,188,23,234,96,218,88,249,50,48,173,160,65,63,150,
+143,117,93,33,67,121,84,66,192,55,61,216,220,122,41,172,14,71,227,132,
+17,76,92,131,147,194,218,11,192,155,244,171,32,141,150,212,40,80,222,30,
+78,135,17,201,234,34,215,182,200,11,183,65,173,11,120,130,77,165,100,197,
+47,14,24,47,54,213,229,78,252,142,27,132,138,210,5,204,42,42,105,136,
+239,188,185,111,204,57,181,255,234,163,184,193,248,85,24,168,77,74,75,177,
+145,46,134,113,211,134,49,185,60,191,42,82,42,222,104,114,33,24,40,24,
+1,64,114,61,105,90,33,176,216,255,250,31,154,37,85,209,154,3,111,237,
+63,27,86,116,83,5,44,150,129,96,217,64,127,122,100,133,149,177,213,221,
+185,138,101,96,46,193,192,127,228,72,248,95,89,206,145,84,249,108,105,124,
+104,165,39,213,34,234,194,8,213,52,4,41,203,59,243,86,75,141,73,248,
+203,245,122,116,51,28,27,101,223,86,170,125,114,66,196,94,255,33,231,245,
+67,165,203,130,89,42,252,121,81,205,195,208,253,150,61,195,122,104,47,63,
+123,62,249,165,226,220,45,57,252,103,236,90,61,96,222,219,163,14,199,182,
+131,132,41,160,59,69,48,234,95,179,134,179,120,219,227,138,30,116,202,204,
+87,187,155,240,232,35,216,150,96,236,152,84,138,14,117,237,219,147,2,59,
+24,38,241,123,52,244,202,253,48,2,3,64,29,43,98,75,171,227,215,112,
+55,141,31,25,241,71,6,73,71,207,152,195,93,175,105,201,67,0,106,136,
+71,172,2,39,179,124,75,43,176,172,207,62,68,203,192,80,170,70,84,193,
+124,139,122,191,235,175,47,66,234,154,192,180,52,131,251,108,81,34,59,9,
+102,154,60,122,154,171,152,100,135,162,237,121,140,130,20,250,5,183,185,35,
+165,197,199,92,238,211,18,108,131,129,221,204,1,219,56,124,187,72,48,87,
+194,77,9,74,28,195,231,10,238,78,147,36,161,250,156,161,14,8,87,235,
+166,137,11,204,248,195,25,114,34,141,55,216,117,253,41,238,231,46,0,158,
+112,217,119,98,66,15,13,181,83,22,97,132,113,154,154,209,135,246,11,139,
+185,189,203,158,35,34,150,134,57,73,76,44,88,124,60,33,159,230,168,83,
+32,131,131,196,105,186,210,238,43,148,95,154,99,212,20,97,46,112,166,210,
+116,85,176,160,247,29,91,95,1,244,205,168,249,240,168,55,107,29,5,72,
+40,131,118,110,115,170,67,15,94,248,44,215,143,197,238,103,163,113,48,77,
+104,141,76,89,10,65,68,109,38,251,252,225,6,195,134,104,51,243,58,239,
+96,208,118,100,146,248,191,82,52,20,63,132,6,14,150,156,160,67,244,241,
+158,181,68,179,16,209,162,117,95,6,109,213,68,75,170,62,27,73,179,130,
+135,184,9,116,193,200,156,159,173,248,188,42,171,79,121,225,212,28,186,101,
+167,122,184,254,255,193,131,129,220,107,179,83,255,99,86,218,181,202,207,231,
+72,96,248,248,120,51,38,32,165,140,131,193,48,144,77,254,231,235,178,54,
+26,30,72,213,255,186,236,68,176,109,220,15,43,219,96,16,4,73,227,155,
+28,18,132,172,20,107,230,201,58,199,90,252,48,36,239,239,113,93,33,139,
+181,186,31,3,51,27,215,242,164,92,175,185,147,78,163,20,221,29,220,45,
+151,17,118,7,10,152,38,196,66,83,192,229,122,156,241,207,228,163,197,151,
+127,213,192,55,169,71,0,109,14,132,226,217,130,80,34,196,170,135,2,42,
+227,136,40,204,233,225,51,68,227,218,67,223,17,188,70,234,193,43,248,117,
+154,236,116,198,229,105,19,212,167,227,103,121,202,91,170,5,162,16,207,202,
+122,168,90,19,168,16,212,99,130,38,145,32,107,148,61,234,138,204,177,49,
+169,189,70,236,239,54,123,110,120,7,7,108,23,168,12,228,234,113,111,86,
+88,69,187,111,220,240,1,22,162,40,165,204,238,214,109,253,79,63,208,199,
+240,112,209,101,205,232,3,125,77,19,129,95,242,107,25,86,64,202,221,84,
+65,184,205,10,200,126,209,31,109,16,242,138,197,5,152,140,209,61,7,142,
+158,3,25,62,194,88,68,243,89,38,215,93,223,23,238,54,29,193,95,11,
+204,136,91,31,103,159,16,136,254,30,153,97,123,195,71,40,128,185,185,63,
+225,2,21,123,216,174,135,12,230,228,187,165,118,166,53,4,119,26,21,11,
+82,25,98,103,217,43,131,17,167,79,233,42,5,224,66,202,237,167,171,62,
+176,58,123,62,120,115,194,117,76,9,165,88,111,207,72,234,126,164,81,57,
+247,36,18,107,228,26,235,224,193,121,194,13,30,59,189,111,99,2,137,215,
+113,54,123,195,27,138,89,4,135,170,181,70,241,207,225,194,20,24,210,174,
+93,255,139,190,100,245,56,117,205,122,84,180,36,247,224,88,192,88,223,68,
+228,166,19,138,94,200,7,77,30,62,156,239,173,32,218,10,206,205,25,53,
+206,53,88,117,217,77,169,146,236,193,80,231,215,20,140,188,201,203,46,114,
+253,191,72,156,1,11,200,114,55,140,69,135,132,234,10,10,182,220,125,191,
+210,3,242,235,121,248,3,185,0,82,14,138,163,55,75,148,141,249,111,171,
+146,186,229,128,143,234,138,154,20,168,231,203,200,228,78,221,53,165,85,56,
+151,17,134,204,117,16,78,37,233,141,97,70,101,83,78,171,149,110,157,6,
+4,169,189,57,199,19,238,143,75,212,99,143,100,148,245,35,202,193,231,47,
+145,184,90,19,73,192,155,22,191,242,154,99,199,47,227,94,238,24,224,179,
+78,11,9,203,4,220,65,166,111,217,204,102,102,49,5,172,108,40,162,41,
+58,163,218,227,85,22,128,119,255,213,11,162,167,82,29,75,85,22,59,246,
+38,214,72,141,43,128,186,15,14,15,8,234,184,216,110,146,216,194,28,179,
+225,150,90,105,16,60,12,43,250,97,84,147,34,210,7,169,9,149,144,51,
+7,66,166,56,236,164,68,22,46,250,75,39,28,178,37,194,25,57,181,91,
+219,162,223,41,140,253,122,210,11,241,48,241,64,24,135,39,17,111,248,63,
+106,130,204,125,96,185,135,56,166,65,144,79,162,102,63,74,2,233,97,174,
+163,94,60,205,53,0,31,32,100,160,241,125,82,149,76,71,254,57,115,144,
+110,202,80,127,62,217,192,137,18,160,66,162,35,58,61,164,122,94,152,9,
+165,96,151,114,227,34,57,23,109,13,177,32,126,11,129,218,100,201,15,244,
+196,147,218,132,69,74,106,107,51,188,76,0,155,98,176,208,201,227,66,240,
+199,57,40,164,181,150,207,108,9,189,118,224,13,85,210,226,164,40,188,216,
+154,69,197,120,141,192,63,82,232,59,131,20,73,98,17,203,69,62,9,30,
+233,21,93,178,93,31,62,239,64,58,23,244,124,152,220,217,201,232,112,22,
+214,236,106,255,214,199,247,167,56,75,249,119,3,70,181,200,18,51,181,222,
+254,55,140,73,188,226,219,178,32,154,100,243,94,152,228,42,117,217,169,238,
+233,3,170,202,27,20,77,228,2,0,10,187,38,71,241,178,207,80,173,179,
+104,240,240,80,254,18,85,68,207,84,123,221,135,130,191,220,55,240,49,153,
+246,31,237,180,227,161,85,117,63,204,125,118,249,128,166,241,105,134,99,161,
+178,246,122,147,149,219,230,146,35,88,155,243,41,33,5,152,199,207,17,189,
+41,10,104,251,49,135,172,167,7,100,179,23,144,187,125,136,109,148,147,158,
+237,75,13,85,205,32,69,77,68,110,118,89,190,233,2,112,240,32,70,83,
+112,39,250,213,0,176,33,214,184,105,133,81,105,210,164,178,134,53,140,70,
+51,137,91,247,197,34,237,125,28,155,18,177,245,232,26,128,60,81,42,3,
+247,101,208,40,173,141,192,201,237,169,215,69,81,131,18,0,24,68,176,228,
+31,206,90,110,23,192,49,155,162,125,216,12,254,71,97,127,150,95,119,148,
+14,7,146,49,189,142,124,176,34,2,149,148,100,138,253,177,13,246,184,73,
+52,245,31,124,115,57,56,29,66,30,143,67,234,238,200,142,124,181,71,242,
+104,213,66,217,221,20,205,200,67,8,50,115,24,106,220,110,83,213,211,5,
+148,88,229,71,246,198,83,201,245,113,148,255,253,232,68,100,128,179,111,236,
+214,122,252,181,24,33,104,159,110,126,134,253,180,42,124,129,154,50,53,169,
+118,215,133,223,11,89,206,48,197,73,21,44,24,123,68,237,34,34,54,159,
+230,5,176,186,140,34,157,126,180,93,193,51,173,74,126,88,81,193,87,75,
+42,5,148,86,18,82,51,168,201,248,81,171,180,161,21,157,76,52,196,209,
+18,23,83,190,47,202,10,11,160,27,240,128,175,25,190,101,253,166,203,170,
+194,4,231,187,35,51,210,18,230,190,68,233,251,186,153,39,244,198,190,6,
+93,238,90,219,90,142,2,83,200,145,27,116,104,90,49,178,83,221,152,135,
+146,160,218,127,167,237,201,3,217,140,130,80,3,203,85,152,181,25,171,109,
+150,125,174,208,57,51,122,210,183,235,67,216,76,91,153,177,194,37,59,7,
+29,59,22,128,158,28,159,251,75,14,73,67,61,219,104,179,87,238,1,10,
+152,8,234,201,81,73,235,252,237,6,60,34,228,76,203,208,80,70,152,244,
+46,87,35,235,199,163,55,234,8,161,249,218,143,195,243,240,166,225,146,251,
+61,140,39,144,183,177,182,191,207,17,157,17,116,132,183,13,244,170,61,199,
+55,55,238,42,15,104,235,201,145,12,224,107,186,2,5,147,2,118,221,151,
+229,109,106,164,181,231,159,81,183,193,104,130,240,35,71,154,217,35,1,131,
+30,136,139,54,225,211,60,180,149,251,29,27,239,124,46,76,143,192,112,239,
+181,168,152,234,231,161,186,125,133,125,125,26,155,92,234,121,35,32,44,42,
+36,194,215,204,95,243,27,83,34,121,94,14,201,210,193,248,242,131,217,12,
+0,27,156,3,28,97,107,71,100,41,161,113,199,191,147,65,59,198,154,110,
+149,242,190,51,131,247,181,250,139,149,20,219,8,216,136,120,248,207,144,172,
+125,141,138,65,122,73,47,61,155,41,112,122,143,225,183,127,164,117,124,86,
+79,139,221,58,178,44,121,253,74,235,215,212,186,80,99,190,158,139,75,87,
+46,223,119,184,190,58,172,169,163,133,100,90,129,131,197,188,229,227,27,94,
+112,177,64,142,92,184,37,58,118,146,3,84,182,212,200,247,84,175,58,175,
+43,60,59,41,135,22,187,150,220,33,217,155,144,161,103,62,23,245,206,220,
+171,226,42,93,135,18,95,199,133,57,223,240,155,92,79,149,108,247,201,97,
+168,186,156,216,14,144,99,98,150,157,189,72,201,161,150,206,12,250,164,129,
+10,185,137,7,132,251,44,168,93,36,254,70,172,216,36,71,23,139,217,7,
+243,248,116,173,192,78,51,86,66,212,117,143,161,159,187,246,47,241,58,168,
+44,167,76,102,31,243,14,143,26,207,248,73,63,25,143,194,99,10,233,190,
+59,176,115,186,138,173,254,126,21,28,164,168,208,229,163,252,142,20,29,224,
+74,203,80,193,0,198,231,211,236,111,144,219,157,145,202,199,11,190,198,60,
+100,115,54,19,227,130,133,144,87,113,109,37,29,54,179,1,173,54,74,95,
+118,253,121,173,33,239,186,221,238,163,16,240,129,233,33,39,35,206,203,54,
+46,78,158,102,208,49,148,181,52,157,43,195,130,12,74,16,20,129,133,103,
+254,21,80,193,128,30,193,26,241,196,124,211,156,54,24,106,224,128,154,162,
+117,128,60,37,115,90,43,235,217,95,146,68,141,171,126,152,174,226,213,105,
+213,162,204,20,123,131,248,24,146,202,81,231,197,230,218,6,18,33,154,37,
+100,114,237,16,241,232,93,163,8,137,227,220,248,1,178,200,123,49,86,250,
+108,239,48,14,108,80,175,244,10,191,73,100,192,44,114,62,2,92,62,248,
+193,9,145,1,168,139,173,23,161,7,219,88,186,109,102,38,39,130,254,93,
+109,240,230,224,139,158,64,135,158,200,221,218,192,244,60,31,216,56,167,33,
+195,13,235,74,10,214,28,239,166,31,122,231,45,151,184,133,117,209,85,251,
+38,101,46,137,31,52,45,46,157,34,3,116,13,83,212,226,176,79,41,70,
+15,22,154,124,164,19,97,212,197,119,97,140,254,38,25,158,173,158,4,251,
+178,226,79,210,126,4,193,65,67,45,86,119,121,214,200,249,13,45,66,5,
+211,234,50,132,16,31,141,8,109,90,104,107,245,131,48,213,107,211,101,37,
+30,145,187,189,114,96,236,136,8,138,69,59,143,121,87,195,74,230,109,77,
+21,15,255,203,231,2,100,245,252,171,166,205,224,33,216,68,122,28,22,177,
+26,185,11,158,249,6,196,254,27,26,51,75,144,232,51,43,48,94,12,179,
+170,187,171,196,236,206,91,59,134,134,99,169,42,201,195,170,166,248,163,116,
+250,232,74,254,125,105,103,7,25,55,151,171,200,199,32,41,234,164,214,21,
+126,168,89,182,6,114,45,18,66,117,167,185,223,174,145,213,173,13,120,2,
+17,2,234,154,247,211,152,151,68,178,189,243,191,100,69,56,220,204,135,8,
+8,243,106,81,200,161,112,254,168,188,204,223,41,199,179,118,175,24,80,181,
+170,251,119,53,153,144,106,152,193,156,110,2,76,22,210,189,54,142,117,176,
+41,12,75,205,151,114,6,13,189,73,198,171,142,128,101,182,209,222,11,114,
+45,39,232,119,242,223,111,146,173,163,117,232,158,82,227,218,55,181,153,209,
+191,44,235,153,169,79,18,202,33,154,149,43,125,156,37,109,244,43,86,226,
+56,81,141,51,169,137,156,143,5,117,89,207,170,127,215,28,116,171,7,16,
+180,47,117,179,112,16,184,46,223,127,230,15,226,49,143,34,161,0,74,190,
+164,205,33,6,69,47,76,151,176,62,237,109,170,137,103,59,171,2,60,229,
+86,41,126,159,61,9,180,190,18,223,84,23,218,160,202,214,171,221,113,204,
+223,96,236,3,154,57,72,156,53,119,189,98,142,172,23,47,223,207,157,224,
+57,119,67,120,18,183,228,129,205,184,146,183,200,106,135,187,11,115,171,131,
+251,158,86,210,209,130,59,30,225,207,255,154,136,104,232,164,216,111,202,39,
+238,160,200,71,247,26,87,37,249,213,204,171,77,16,203,192,251,40,218,194,
+95,209,125,2,40,146,99,233,97,2,230,42,37,159,217,203,58,10,96,182,
+87,9,157,37,95,143,33,75,0,126,233,64,250,86,148,124,114,213,94,31,
+0,78,79,79,222,171,188,103,37,48,242,25,135,37,248,43,201,102,126,30,
+100,113,223,59,89,234,155,103,117,168,48,180,178,58,35,117,89,197,128,226,
+143,119,68,57,227,238,78,14,8,80,192,90,193,1,100,162,163,120,57,94,
+116,120,206,152,147,169,54,85,42,86,144,120,19,99,62,206,56,224,76,128,
+251,235,161,87,69,139,212,197,6,191,205,144,101,220,184,119,103,245,6,58,
+183,235,138,35,144,193,33,211,70,5,69,3,102,203,97,194,49,102,114,200,
+52,199,181,175,114,151,75,40,18,3,10,73,189,60,47,246,169,114,6,229,
+65,165,111,45,114,7,51,136,91,213,86,46,225,115,110,169,124,107,27,26,
+32,226,191,146,204,127,15,192,70,59,90,187,83,189,66,74,43,106,248,135,
+173,68,134,6,70,29,237,15,147,187,75,94,62,253,39,202,198,17,10,42,
+17,170,214,42,24,75,111,46,1,39,12,20,163,215,188,178,161,61,191

<TRUNCATED>