[CARBONDATA-3271] Integrating deep learning framework TensorFlow

Why is this PR needed?
Nowadays AI model training is getting more and more popular. Currently many AI framework uses raw data files or row format data files for model training, it could not provide projection, filtering, and fast scan capability like in columnar store. So, if CarbonData supports AI framework, it can speed up model training by increase IO throughput, and provide more flexible training set selection ability to AI developers

What changes were proposed in this PR?
Added a basic framework:
- Supports shuffle read, which reads the data in random order when feeding data to training model for each epoch.
- Supports data cache to improve reading speed for multiple epoch, including local-disk and memory-cache.
- Supports parallel reading using thread pool and process pool in python.
- Supports reading data in object storage
- Supports manifest format and CarbonData folder

Support tensorflow to use the framework:
Tensorflow integration: New python API in pycarbon to support TensorFlow to read data from CarbonData files for training model

Important files, please review in details:
reader.py
tensorflow.py
carbon.py
carbon_arrow_reader_worker.py
carbon_py_dict_reader_worker.py
carbon_reader.py
carbon_tf_utils.py
carbon_dataset_metadata.py

Does this PR introduce any user interface change?
Yes. (please explain the change and update document)

Main new interfaces:
```
def make_reader(dataset_url=None,
                workers_count=10,
                results_queue_size=100,
                num_epochs=1,
                obs_client=None,
                shuffle=True,
                schema_fields=None,
                is_batch=True,
                reader_pool_type='thread',
                data_format='carbon',
                cache_properties={'cache_type': None, 'cache_location': None, 'cache_size_limit': None,
                                  'cache_row_size_estimate': None, 'cache_extra_settings': None},
                **properties
                ):

def make_tensor(reader, shuffling_queue_capacity=0, min_after_dequeue=0):

def make_dataset(reader)
```

Is any new testcase added?
Yes

https://issues.apache.org/jira/browse/CARBONDATA-3254

Example:

1. Setup

cd /yourpath/carbondata/python/
PYTHONPATH=/yourpath/carbondata/python/
pip install . --user

2. Generating a Pycarbon Dataset from MNIST Data

User should do some config first:

config pyspark and add carbon assembly jar to pyspark/jars folder, which can be compiled from CarbonData project.

default Java sdk jar is in carbondata/store/sdk/target, user also can specific the jar location like by --carbon-sdk-path in generate_pycarbon_dataset.py.

set JAVA_HOME, PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in you system environment.

This creates both a train and test carbon datasets:

cd pycarbon/tests/mnist/dataset_with_unischema
python generate_pycarbon_mnist.py

if user didn't compile CarbonData, then they can specific CarbonData java SDK jar like:

 python generate_pycarbon_mnist.py --carbon-sdk-path  /your_path/carbondata/store/sdk/target/carbondata-sdk.jar

3. Tensorflow training using the Carbon MNIST Dataset

This will invoke a training run using MNIST carbondata,
for 1 epochs, using a batch size of 100, and log every 10 intervals.

python tf_example_carbon_unified_api.py
if user didn't compile CarbonData, then they can specific CarbonData java SDK jar like:

python  tf_example_carbon_unified_api.py --carbon-sdk-path  /your_path/carbondata/store/sdk/target/carbondata-sdk.jar

This closes #3479
diff --git a/.gitignore b/.gitignore
index d0d1505..0af3918 100644
--- a/.gitignore
+++ b/.gitignore
@@ -24,3 +24,4 @@
 */*/.cache-tests
 */*/*/.cache-main
 */*/*/.cache-tests
+python/pycarbon/.pylintrc
diff --git a/python/README.md b/python/README.md
new file mode 100644
index 0000000..39f6808
--- /dev/null
+++ b/python/README.md
@@ -0,0 +1,94 @@
+# PyCarbon
+
+PyCarbon provides python API for integrating CarbonData with AI framework like  TensorFlow, PyTorch, MXNet. By using PyCarbon, AI framework can read training data faster by leveraging CarbonData's indexing and caching ability. Since CarbonData is a columnar storage, AI developer can also perform projection and filtering to pick required data for training efficiently.
+
+## PyCarbon install
+
+$ git clone https://github.com/apache/carbondata.git
+
+$ cd python/pycarbon
+
+$ pip install . --user
+
+
+## how to use
+
+if you have a CarbonData dataset, you can use PyCarbon to read data. For the generation of CarbonData dataset, you can see the examples:
+`generate_dataset_carbon.py` in test/hello_world and `generate_pycarbon_dataset.py` in test/hello_world.
+But user should do some config first:
+
+ - config pyspark and add carbon assembly jar to pyspark/jars folder, which can be compiled from CarbonData project.
+ - default Java sdk jar is in carbondata/store/sdk/target,  user also can specific the jar location like by --carbon-sdk-path in generate_pycarbon_dataset.py.
+
+ - set JAVA_HOME, PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in you system environment.
+
+#### PySpark and SQL
+
+#### Generating the data:
+```
+
+    python generate_pycarbon_mnist.py --carbon-sdk-path  /your_path/carbondata/store/sdk/target/carbondata-sdk.jar 
+
+```
+##### Part of code:
+```
+    # Create a dataframe object from carbon files
+    spark.sql("create table readcarbon using carbon location '" + str(dataset_path) + "'")
+    dataframe = spark.sql("select * from readcarbon")
+
+    # Show a schema
+    dataframe.printSchema()
+
+    # Count all
+    dataframe.count()
+
+    # Show just some columns
+    dataframe.select('id').show()
+
+    # Also use a standard SQL to query a dataset
+    spark.sql('SELECT count(id) from carbon.`{}` '.format(dataset_url)).collect()
+```
+some details are illustrated in `pyspark_hello_world_carbon.py` in test/hello_world.
+
+#### TensorFlow Dataset API
+
+
+##### Running train and test based on mnist:
+
+```
+    python  tf_example_carbon_unified_api.py --carbon-sdk-path  /your_path/carbondata/store/sdk/target/carbondata-sdk.jar 
+
+```
+##### Part or code:
+```
+    with make_reader('file:///some/localpath/a_dataset') as reader:
+        dataset = make_dataset(reader)
+        iterator = dataset.make_one_shot_iterator()
+        tensor = iterator.get_next()
+        with tf.Session() as sess:
+            sample = sess.run(tensor)
+            print(sample.id)
+
+some details are illustrated in `tf_example_carbon_unified_api.py` in test/mnist. 
+```
+
+#####  Part of result:
+
+```
+2020-01-20 21:12:31 INFO  DictionaryBasedVectorResultCollector:72 - Direct pagewise vector fill collector is used to scan and collect the data
+2020-01-20 21:12:32 INFO  UnsafeMemoryManager:176 - Total offheap working memory used after task 2642c969-6c43-4e31-b8b0-450dff1f7821 is 0. Current running tasks are 
+2020-01-20 21:12:32 INFO  UnsafeMemoryManager:176 - Total offheap working memory used after task 67ecf75e-e097-486d-b787-8b7db5f1d7c1 is 0. Current running tasks are 
+After 0 training iterations, the accuracy of the model is: 0.27
+After 10 training iterations, the accuracy of the model is: 0.48
+After 20 training iterations, the accuracy of the model is: 0.78
+After 30 training iterations, the accuracy of the model is: 0.69
+After 40 training iterations, the accuracy of the model is: 0.73
+After 50 training iterations, the accuracy of the model is: 0.79
+After 60 training iterations, the accuracy of the model is: 0.85
+After 70 training iterations, the accuracy of the model is: 0.73
+After 80 training iterations, the accuracy of the model is: 0.86
+After 90 training iterations, the accuracy of the model is: 0.80
+After 99 training iterations, the accuracy of the model is: 0.79
+all time: 185.28250288963318
+Finish
+```
\ No newline at end of file
diff --git a/python/pycarbon/__init__.py b/python/pycarbon/__init__.py
index e69de29..0b909a7 100644
--- a/python/pycarbon/__init__.py
+++ b/python/pycarbon/__init__.py
@@ -0,0 +1,16 @@
+# 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.
+
+__version__ = '0.1.0'
diff --git a/python/pycarbon/core/Constants.py b/python/pycarbon/core/Constants.py
new file mode 100644
index 0000000..adf0053
--- /dev/null
+++ b/python/pycarbon/core/Constants.py
@@ -0,0 +1,17 @@
+# 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.
+
+
+LOCAL_FILE_PREFIX = "file://"
diff --git a/python/pycarbon/core/__init__.py b/python/pycarbon/core/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/core/__init__.py
diff --git a/python/pycarbon/core/carbon.py b/python/pycarbon/core/carbon.py
new file mode 100644
index 0000000..bfe9c1d
--- /dev/null
+++ b/python/pycarbon/core/carbon.py
@@ -0,0 +1,242 @@
+# 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.
+
+
+import pyarrow as pa
+from modelarts import manifest
+from modelarts.field_name import CARBON
+from pyarrow.filesystem import (_ensure_filesystem)
+from pyarrow.filesystem import (_get_fs_from_path)
+from pyarrow.parquet import ParquetFile
+from six.moves.urllib.parse import urlparse
+
+from pycarbon.core.Constants import LOCAL_FILE_PREFIX
+from pycarbon.sdk.ArrowCarbonReader import ArrowCarbonReader
+from pycarbon.sdk.CarbonSchemaReader import CarbonSchemaReader
+from pycarbon.sdk.Configuration import Configuration
+
+
+class CarbonDataset(object):
+  def __init__(self, path,
+               key=None,
+               secret=None,
+               endpoint=None,
+               proxy=None,
+               proxy_port=None,
+               filesystem=None):
+    self.path = path
+    self.url_path = urlparse(path)
+
+    if str(path).endswith(".manifest"):
+      self.manifest_path = path
+      if str(path).startswith(LOCAL_FILE_PREFIX):
+        self.manifest_path = str(path)[len(LOCAL_FILE_PREFIX):]
+
+    if filesystem is None:
+      a_path = self.path
+      if isinstance(a_path, list):
+        a_path = a_path[0]
+      self.fs = _get_fs_from_path(a_path)
+    else:
+      self.fs = _ensure_filesystem(filesystem)
+
+    self.pieces = list()
+
+    if self.url_path.scheme == 's3a':
+      if key is None or secret is None or endpoint is None:
+        raise ValueError('key, secret, endpoint should not be None')
+
+      if proxy is None and proxy_port is None:
+        carbon_splits = ArrowCarbonReader().builder(self.path) \
+          .withHadoopConf("fs.s3a.access.key", key) \
+          .withHadoopConf("fs.s3a.secret.key", secret) \
+          .withHadoopConf("fs.s3a.endpoint", endpoint) \
+          .getSplits(True)
+
+        configuration = Configuration()
+        configuration.set("fs.s3a.access.key", key)
+        configuration.set("fs.s3a.secret.key", secret)
+        configuration.set("fs.s3a.endpoint", endpoint)
+
+        self.configuration = configuration
+
+      elif proxy is not None and proxy_port is not None:
+        carbon_splits = ArrowCarbonReader().builder(self.path) \
+          .withHadoopConf("fs.s3a.access.key", key) \
+          .withHadoopConf("fs.s3a.secret.key", secret) \
+          .withHadoopConf("fs.s3a.endpoint", endpoint) \
+          .withHadoopConf("fs.s3a.proxy.host", proxy) \
+          .withHadoopConf("fs.s3a.proxy.port", proxy_port) \
+          .getSplits(True)
+
+        configuration = Configuration()
+        configuration.set("fs.s3a.access.key", key)
+        configuration.set("fs.s3a.secret.key", secret)
+        configuration.set("fs.s3a.endpoint", endpoint)
+        configuration.set("fs.s3a.proxy.host", proxy)
+        configuration.set("fs.s3a.proxy.port", proxy_port)
+
+        self.configuration = configuration
+      else:
+        raise ValueError('wrong proxy & proxy_port configuration')
+
+      if str(path).endswith(".manifest"):
+        from obs import ObsClient
+        obsClient = ObsClient(access_key_id=key, secret_access_key=secret,
+                              server=str(endpoint).replace('http://', ''),
+                              long_conn_mode=True)
+        sources = manifest.getSources(self.manifest_path, CARBON, obsClient)
+        if sources:
+          self.file_path = sources[0]
+        else:
+          raise Exception("Manifest source can't be None!")
+        carbon_schema = CarbonSchemaReader().readSchema(self.file_path, self.configuration.conf)
+      else:
+        carbon_schema = CarbonSchemaReader().readSchema(self.path, self.configuration.conf)
+
+      for split in carbon_splits:
+        # split = self.url_path.scheme + "://" + self.url_path.netloc + split
+        folder_path = path
+        if str(path).endswith(".manifest"):
+          folder_path = str(self.file_path)[0:(str(self.file_path).rindex('/'))]
+        self.pieces.append(CarbonDatasetPiece(folder_path, carbon_schema, split,
+                                              key=key, secret=secret, endpoint=endpoint,
+                                              proxy=proxy, proxy_port=proxy_port))
+
+    else:
+      if str(path).endswith(".manifest"):
+        sources = manifest.getSources(self.manifest_path, CARBON)
+        if sources:
+          self.file_path = sources[0]
+        else:
+          raise Exception("Manifest source can't be None!")
+
+        try:
+          carbon_schema = CarbonSchemaReader().readSchema(self.file_path)
+        except:
+          raise Exception("readSchema has some errors: " + self.file_path)
+      else:
+        try:
+          carbon_schema = CarbonSchemaReader().readSchema(self.path)
+        except:
+          raise Exception("readSchema has some errors")
+
+      carbon_splits = ArrowCarbonReader().builder(self.path) \
+        .getSplits(True)
+
+      for split in carbon_splits:
+        # split = self.url_path.scheme + "://" + self.url_path.netloc + split
+        if str(path).endswith(".manifest"):
+          self.pieces.append(
+            CarbonDatasetPiece(str(self.file_path)[0:(str(self.file_path).rindex('/'))], carbon_schema, split))
+        else:
+          self.pieces.append(CarbonDatasetPiece(path, carbon_schema, split))
+
+    self.number_of_splits = len(self.pieces)
+    self.schema = self.getArrowSchema()
+    # TODO add mechanism to get the file path based on file filter
+    self.common_metadata_path = self.url_path.path + '/_common_metadata'
+    self.common_metadata = None
+    try:
+      if self.fs.exists(self.common_metadata_path):
+        with self.fs.open(self.common_metadata_path) as f:
+          self.common_metadata = ParquetFile(f).metadata
+    except:
+      self.common_metadata = None
+
+  def getArrowSchema(self):
+    file_path = self.path
+
+    if str(self.path).endswith(".manifest"):
+      file_path = self.file_path
+    if self.url_path.scheme == 's3a':
+      buf = CarbonSchemaReader().readSchema(file_path, True, self.configuration.conf).tostring()
+    else:
+      buf = CarbonSchemaReader().readSchema(file_path, True).tostring()
+
+    reader = pa.RecordBatchFileReader(pa.BufferReader(bytes(buf)))
+    return reader.read_all().schema
+
+
+class CarbonDatasetPiece(object):
+  def __init__(self, path, carbon_schema, input_split,
+               key=None,
+               secret=None,
+               endpoint=None,
+               proxy=None,
+               proxy_port=None):
+    self.path = path
+    self.url_path = urlparse(path)
+    self.input_split = input_split
+    self.carbon_schema = carbon_schema
+    # TODO get record count from carbonapp based on file
+    self.num_rows = 10000
+    self.use_s3 = False
+
+    if self.url_path.scheme == 's3a':
+      self.use_s3 = True
+
+      if key is None or secret is None or endpoint is None:
+        raise ValueError('key, secret, endpoint should not be None')
+
+      self.key = key
+      self.secret = secret
+      self.endpoint = endpoint
+
+      if proxy is None and proxy_port is None:
+        self.proxy = proxy
+        self.proxy_port = proxy_port
+      elif proxy is not None and proxy_port is not None:
+        self.proxy = proxy
+        self.proxy_port = proxy_port
+      else:
+        raise ValueError('wrong proxy & proxy_port configuration')
+
+  def read_all(self, columns):
+    # rebuilding the reader as need to read specific columns
+    carbon_reader_builder = ArrowCarbonReader().builder(self.input_split)
+    carbon_schema_reader = CarbonSchemaReader()
+    if columns is not None:
+      carbon_reader_builder = carbon_reader_builder.projection(columns)
+      updatedSchema = carbon_schema_reader.reorderSchemaBasedOnProjection(columns, self.carbon_schema)
+    else:
+      # TODO Currently when projection is not added in carbon reader
+      # carbon returns record in dimensions+measures,but here we need based on actual schema order
+      # so for handling this adding projection columns based on schema
+      updatedSchema = self.carbon_schema
+      projection = carbon_schema_reader.getProjectionBasedOnSchema(updatedSchema)
+      carbon_reader_builder = carbon_reader_builder.projection(projection)
+
+    if self.use_s3:
+      if self.proxy is None and self.proxy_port is None:
+        carbon_reader = carbon_reader_builder \
+          .withHadoopConf("fs.s3a.access.key", self.key) \
+          .withHadoopConf("fs.s3a.secret.key", self.secret) \
+          .withHadoopConf("fs.s3a.endpoint", self.endpoint) \
+          .build()
+      else:
+        carbon_reader = carbon_reader_builder \
+          .withHadoopConf("fs.s3a.access.key", self.key) \
+          .withHadoopConf("fs.s3a.secret.key", self.secret) \
+          .withHadoopConf("fs.s3a.endpoint", self.endpoint) \
+          .withHadoopConf("fs.s3a.proxy.host", self.proxy) \
+          .withHadoopConf("fs.s3a.proxy.port", self.proxy_port) \
+          .build()
+    else:
+      carbon_reader = carbon_reader_builder.build()
+
+    data = carbon_reader.read(updatedSchema)
+    carbon_reader.close()
+    return data
diff --git a/python/pycarbon/core/carbon_arrow_reader_worker.py b/python/pycarbon/core/carbon_arrow_reader_worker.py
new file mode 100644
index 0000000..03cfd23
--- /dev/null
+++ b/python/pycarbon/core/carbon_arrow_reader_worker.py
@@ -0,0 +1,188 @@
+# 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.
+
+
+from __future__ import division
+
+import hashlib
+import operator
+
+import numpy as np
+import pandas as pd
+import pyarrow as pa
+
+from petastorm.cache import NullCache
+from petastorm.workers_pool.worker_base import WorkerBase
+from petastorm.arrow_reader_worker import ArrowReaderWorkerResultsQueueReader
+
+
+class ArrowCarbonReaderWorker(WorkerBase):
+  def __init__(self, worker_id, publish_func, args):
+    super(ArrowCarbonReaderWorker, self).__init__(worker_id, publish_func, args)
+
+    self._filesystem = args[0]
+    self._dataset_path = args[1]
+    self._schema = args[2]
+    self._ngram = args[3]
+    self._split_pieces = args[4]
+    self._local_cache = args[5]
+    self._transform_spec = args[6]
+
+    if self._ngram:
+      raise NotImplementedError('ngrams are not supported by ArrowReaderWorker')
+
+    # We create datasets lazily in the first invocation of 'def process'. This speeds up startup time since
+    # all Worker constructors are serialized
+    self._dataset = None
+
+  @staticmethod
+  def new_results_queue_reader():
+    return ArrowReaderWorkerResultsQueueReader()
+
+  # pylint: disable=arguments-differ
+  def process(self, piece_index, worker_predicate, shuffle_row_drop_partition):
+    """Main worker function. Loads and returns all rows matching the predicate from a blocklet
+
+    Looks up the requested piece (a single row-group in a carbon file). If a predicate is specified,
+    columns needed by the predicate are loaded first. If no rows in the blocklet matches the predicate criteria
+    the rest of the columns are not loaded.
+
+    :param piece_index:
+    :param shuffle_row_drop_partition: A tuple 2 of the current row drop partition and the total number
+        of partitions.
+    :return:
+    """
+
+    piece = self._split_pieces[piece_index]
+
+    if not isinstance(self._local_cache, NullCache):
+      if worker_predicate:
+        raise RuntimeError('Local cache is not supported together with predicates, '
+                           'unless the dataset is partitioned by the column the predicate operates on.')
+      if shuffle_row_drop_partition[1] != 1:
+        raise RuntimeError('Local cache is not supported together with shuffle_row_drop_partitions > 1')
+
+    if worker_predicate:
+      all_cols = self._load_rows_with_predicate(piece, worker_predicate, shuffle_row_drop_partition)
+    else:
+      # Using hash of the dataset path with the relative path in order to:
+      #  1. Make sure if a common cache serves multiple processes (e.g. redis), we don't have conflicts
+      #  2. Dataset path is hashed, to make sure we don't create too long keys, which maybe incompatible with
+      #     some cache implementations
+      #  3. Still leave relative path and the piece_index in plain text to make it easier to debug
+      cache_key = '{}:{}:{}'.format(hashlib.md5(self._dataset_path.encode('utf-8')).hexdigest(),
+                                    piece.path, piece_index)
+      all_cols = self._local_cache.get(cache_key,
+                                       lambda: self._load_rows(piece, shuffle_row_drop_partition))
+
+    if all_cols:
+      self.publish_func(all_cols)
+
+  def _load_rows(self, piece, shuffle_row_drop_range):
+    """Loads all rows from a piece"""
+
+    # pyarrow would fail if we request a column names that the dataset is partitioned by, so we strip them from
+    # the `columns` argument.
+    # partitions = self._dataset.partitions
+    column_names_in_schema = list(field.name for field in self._schema.fields.values())
+    # column_names = column_names_in_schema - partitions.partition_names
+
+    result = self._read_with_shuffle_row_drop(piece, column_names_in_schema, shuffle_row_drop_range)
+
+    if self._transform_spec:
+      result = pa.Table.from_pandas(self._transform_spec.func(result.to_pandas()), preserve_index=False)
+
+    return result
+
+  def _load_rows_with_predicate(self, piece, worker_predicate, shuffle_row_drop_partition):
+    """Loads all rows that match a predicate from a piece"""
+
+    # 1. Read all columns needed by predicate
+    # 2. Apply the predicate. If nothing matches, exit early
+    # 3. Read the remaining columns
+
+    # Split all column names into ones that are needed by predicateand the rest.
+    predicate_column_names = set(worker_predicate.get_fields())
+
+    if not predicate_column_names:
+      raise ValueError('At least one field name must be returned by predicate\'s get_field() method')
+
+    all_schema_names = set(field.name for field in self._schema.fields.values())
+
+    invalid_column_names = predicate_column_names - all_schema_names
+    if invalid_column_names:
+      raise ValueError('At least some column names requested by the predicate ({}) '
+                       'are not valid schema names: ({})'.format(', '.join(invalid_column_names),
+                                                                 ', '.join(all_schema_names)))
+
+    # Split into 'columns for predicate evaluation' and 'other columns'. We load 'other columns' only if at
+    # least one row in the blocklet matched the predicate
+    other_column_names = all_schema_names - predicate_column_names
+
+    # Read columns needed for the predicate
+    predicate_column_names_list = list(predicate_column_names)
+    predicates_table = self._read_with_shuffle_row_drop(piece, predicate_column_names_list,
+                                                        shuffle_row_drop_partition)
+
+    predicates_data_frame = predicates_table.to_pandas()
+
+    match_predicate_mask = worker_predicate.do_include(predicates_data_frame)
+    erase_mask = match_predicate_mask.map(operator.not_)
+
+    # Don't have anything left after filtering? Exit early.
+    if erase_mask.all():
+      return []
+
+    predicates_data_frame[erase_mask] = None
+
+    if other_column_names:
+      # Read remaining columns
+      other_column_names_list = list(other_column_names)
+      other_table = self._read_with_shuffle_row_drop(piece, other_column_names_list,
+                                                     shuffle_row_drop_partition)
+      other_data_frame = other_table.to_pandas()
+      other_data_frame[erase_mask] = None
+
+      # Partition-by columns will appear in both other and predicate data frames. Deduplicate.
+      columns_from_predicates = predicates_data_frame.columns.difference(other_data_frame.columns)
+      result_data_frame = pd.merge(predicates_data_frame[columns_from_predicates], other_data_frame,
+                                   copy=False, left_index=True, right_index=True)
+    else:
+      result_data_frame = predicates_data_frame
+
+    result = result_data_frame[match_predicate_mask]
+
+    if self._transform_spec:
+      result = self._transform_spec.func(result)
+
+    return pa.Table.from_pandas(result, preserve_index=False)
+
+  def _read_with_shuffle_row_drop(self, piece, column_names, shuffle_row_drop_partition):
+    table = piece.read_all(
+      columns=column_names,
+    )
+
+    num_rows = len(table)
+    num_partitions = shuffle_row_drop_partition[1]
+    this_partition = shuffle_row_drop_partition[0]
+
+    if num_partitions > 1:
+      data_frame_pandas = table.to_pandas()
+      partition_indexes = np.floor(np.arange(num_rows) / (float(num_rows) / min(num_rows, num_partitions)))
+
+      table = pa.Table.from_pandas(data_frame_pandas.loc[partition_indexes == this_partition],
+                                   preserve_index=False)
+
+    return table
diff --git a/python/pycarbon/core/carbon_dataset_metadata.py b/python/pycarbon/core/carbon_dataset_metadata.py
new file mode 100644
index 0000000..45318ce
--- /dev/null
+++ b/python/pycarbon/core/carbon_dataset_metadata.py
@@ -0,0 +1,235 @@
+# 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.
+
+
+import json
+import logging
+from contextlib import contextmanager
+
+from six.moves import cPickle as pickle
+from six.moves.urllib.parse import urlparse
+
+from petastorm.etl.legacy import depickle_legacy_package_name_compatible
+from petastorm.fs_utils import FilesystemResolver
+from petastorm.unischema import Unischema
+from petastorm.unischema import _numpy_and_codec_from_arrow_type
+from petastorm.unischema import UnischemaField
+from petastorm.etl.dataset_metadata import _init_spark, _cleanup_spark
+
+from pycarbon.core.carbon import CarbonDataset
+from pycarbon.core import carbon_utils
+
+logger = logging.getLogger(__name__)
+
+BLOCKLETS_PER_FILE_KEY = b'dataset-toolkit.num_blocklets_per_file.v1'
+UNISCHEMA_KEY = b'dataset-toolkit.unischema.v1'
+
+
+class PycarbonMetadataError(Exception):
+  """
+  Error to specify when the pycarbon metadata does not exist, does not contain the necessary information,
+  or is corrupt/invalid.
+  """
+
+@contextmanager
+def materialize_dataset_carbon(spark, dataset_url, schema, blocklet_size_mb=None, use_summary_metadata=False,
+                               pyarrow_filesystem=None):
+  """
+  A Context Manager which handles all the initialization and finalization necessary
+  to generate metadata for a pycarbon dataset. This should be used around your
+  spark logic to materialize a dataset (specifically the writing of carbon output).
+
+  Note: Any blocklet indexing should happen outside the materialize_dataset_carbon block
+
+  Example:
+
+  >>> spark = SparkSession.builder...
+  >>> ds_url = 'hdfs:///path/to/my/dataset'
+  >>> with materialize_dataset_carbon(spark, ds_url, MyUnischema, 64):
+  >>>   spark.sparkContext.parallelize(range(0, 10)).
+  >>>     ...
+  >>>     .write.save(path=ds_url, format='carbon')
+
+  A user may provide their own instance of pyarrow filesystem object in ``pyarrow_filesystem`` argument (otherwise,
+  pycarbon will create a default one based on the url).
+
+  The following example shows how a custom pyarrow HDFS filesystem, instantiated using ``libhdfs`` driver can be used
+  during Pycarbon dataset generation:
+
+  >>> resolver=FilesystemResolver(dataset_url, spark.sparkContext._jsc.hadoopConfiguration(),
+  >>>                             hdfs_driver='libhdfs')
+  >>> with materialize_dataset_carbon(..., pyarrow_filesystem=resolver.filesystem()):
+  >>>     ...
+
+
+  :param spark: The spark session you are using
+  :param dataset_url: The dataset url to output your dataset to (e.g. ``hdfs:///path/to/dataset``)
+  :param schema: The :class:`petastorm.unischema.Unischema` definition of your dataset
+  :param blocklet_size_mb: The carbon blocklet size to use for your dataset
+  :param use_summary_metadata: Whether to use the carbon summary metadata for blocklet indexing or a custom
+    indexing method. The custom indexing method is more scalable for very large datasets.
+  :param pyarrow_filesystem: A pyarrow filesystem object to be used when saving Pycarbon specific metadata to the
+    Carbon store.
+
+  """
+
+  # After job completes, add the unischema metadata and check for the metadata summary file
+  spark_config = {}
+  _init_spark(spark, spark_config, blocklet_size_mb, use_summary_metadata)
+  yield
+
+  # After job completes, add the unischema metadata and check for the metadata summary file
+  if pyarrow_filesystem is None:
+    resolver = FilesystemResolver(dataset_url, spark.sparkContext._jsc.hadoopConfiguration())
+    # filesystem = resolver.filesystem()
+    dataset_path = resolver.get_dataset_path()
+  else:
+    # filesystem = pyarrow_filesystem
+    dataset_path = urlparse(dataset_url).path
+
+  carbon_dataset = CarbonDataset(dataset_path)
+  _generate_unischema_metadata_carbon(carbon_dataset, schema)
+  if not use_summary_metadata:
+    _generate_num_blocklets_per_file_carbon(carbon_dataset, spark.sparkContext)
+
+  _cleanup_spark(spark, spark_config, blocklet_size_mb)
+
+
+def _generate_unischema_metadata_carbon(carbon_schema, schema):
+  """
+  Generates the serialized unischema and adds it to the dataset carbon metadata to be used upon reading.
+  :param dataset: (CarbonDataset) Dataset to attach schema
+  :param schema:  (Unischema) Schema to attach to dataset
+  :return: None
+  """
+  # TODO: Simply pickling unischema will break if the UnischemaField class is changed,
+  #  or the codec classes are changed. We likely need something more robust.
+  serialized_schema = pickle.dumps(schema)
+  carbon_utils.add_to_dataset_metadata_carbon(carbon_schema, UNISCHEMA_KEY, serialized_schema)
+
+
+def _generate_num_blocklets_per_file_carbon(carbon_dataset, spark_context):
+  """
+  Generates the metadata file containing the number of blocklets in each file
+  for the carbon dataset located at the dataset_url. It does this in spark by
+  opening all carbon files in the dataset on the executors and collecting the
+  number of blocklets in each file back on the driver.
+  :param dataset: CarbonDataset
+  :param spark_context: spark context to use for retrieving the number of blocklets
+  in each carbon file in parallel
+  :return: None, upon successful completion the metadata file will exist.
+  """
+  # if not isinstance(dataset.paths, str):
+  #     raise ValueError('Expected dataset.paths to be a single path, not a list of paths')
+  pieces = carbon_dataset.pieces
+  # Get the common prefix of all the base path in order to retrieve a relative path
+  paths = [piece.path for piece in pieces]
+
+  # Needed pieces from the dataset must be extracted for spark because the dataset object is not serializable
+
+  # TODO add number of blocklets for each carbonfile
+  def get_carbon_blocklet_info(path):
+    return path, 1
+
+  number_of_blocklets = spark_context.parallelize(paths, len(paths)) \
+    .map(get_carbon_blocklet_info) \
+    .collect()
+  number_of_blocklets_str = json.dumps(dict(number_of_blocklets))
+  # Add the dict for the number of blocklets in each file to the carbon file metadata footer
+  carbon_utils.add_to_dataset_metadata_carbon(carbon_dataset, BLOCKLETS_PER_FILE_KEY, number_of_blocklets_str)
+
+
+def get_schema_carbon(carbon_dataset):
+  """Retrieves schema object stored as part of dataset methadata.
+
+  :param dataset: CarbonDataset
+  :return: A :class:`petastorm.unischema.Unischema` object
+  """
+  if not carbon_dataset.common_metadata:
+    raise PycarbonMetadataError(
+      'Could not find _common_metadata file. Use materialize_dataset(..) in'
+      ' pycarbon.etl.carbon_dataset_metadata.py to generate this file in your ETL code.'
+      ' You can generate it on an existing dataset using pycarbon-generate-metadata.py')
+  # TODO add pycarbon-generate-metadata.py
+
+  dataset_metadata_dict = carbon_dataset.common_metadata.metadata
+
+  # Read schema
+  if UNISCHEMA_KEY not in dataset_metadata_dict:
+    raise PycarbonMetadataError(
+      'Could not find the unischema in the dataset common metadata file.'
+      ' Please provide or generate dataset with the unischema attached.'
+      ' Common Metadata file might not be generated properly.'
+      ' Make sure to use materialize_dataset(..) in pycarbon.etl.carbon_dataset_metadata to'
+      ' properly generate this file in your ETL code.'
+      ' You can generate it on an existing dataset using pycarbon-generate-metadata.py')
+  ser_schema = dataset_metadata_dict[UNISCHEMA_KEY]
+  # Since we have moved the unischema class around few times, unpickling old schemas will not work. In this case we
+  # override the old import path to get backwards compatibility
+
+  schema = depickle_legacy_package_name_compatible(ser_schema)
+
+  return schema
+
+
+def get_schema_from_dataset_url_carbon(dataset_url,
+                                       key=None,
+                                       secret=None,
+                                       endpoint=None,
+                                       proxy=None,
+                                       proxy_port=None,
+                                       filesystem=None):
+  """Returns a :class:`petastorm.unischema.Unischema` object loaded from a dataset specified by a url.
+
+  :param dataset_url: A dataset URL
+  :param key: access key
+  :param secret: secret key
+  :param endpoint: endpoint_url
+  :param proxy: proxy
+  :param proxy_port:  proxy_port
+  :param filesystem: filesystem
+  :return: A :class:`petastorm.unischema.Unischema` object
+  """
+
+  # Get a unischema stored in the dataset metadata.
+  stored_schema = get_schema_carbon(CarbonDataset(dataset_url,
+                                                  key=key,
+                                                  secret=secret,
+                                                  endpoint=endpoint,
+                                                  proxy=proxy,
+                                                  proxy_port=proxy_port,
+                                                  filesystem=filesystem))
+
+  return stored_schema
+
+
+def infer_or_load_unischema_carbon(carbon_dataset):
+  """Try to recover Unischema object stored by ``materialize_dataset`` function. If it can be loaded, infer
+      Unischema from native Carbon schema"""
+  try:
+    return get_schema_carbon(carbon_dataset)
+  except PycarbonMetadataError:
+    logger.info('Failed loading Unischema from metadata in %s. Assuming the dataset was not created with '
+                'Pycarbon. Will try to construct from native Carbon schema.')
+
+    unischema_fields = []
+    arrow_schema = carbon_dataset.schema
+    for column_name in arrow_schema.names:
+      arrow_field = arrow_schema.field_by_name(column_name)
+      field_type = arrow_field.type
+      codec, np_type = _numpy_and_codec_from_arrow_type(field_type)
+
+      unischema_fields.append(UnischemaField(column_name, np_type, (), codec, arrow_field.nullable))
+    return Unischema('inferred_schema', unischema_fields)
diff --git a/python/pycarbon/core/carbon_fs_utils.py b/python/pycarbon/core/carbon_fs_utils.py
new file mode 100644
index 0000000..7824b4b
--- /dev/null
+++ b/python/pycarbon/core/carbon_fs_utils.py
@@ -0,0 +1,165 @@
+# 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.
+
+
+import pyarrow
+import six
+from six.moves.urllib.parse import urlparse
+
+from petastorm.hdfs.namenode import HdfsNamenodeResolver, HdfsConnector
+
+
+class CarbonFilesystemResolver(object):
+  """Resolves a dataset URL, makes a connection via pyarrow, and provides a filesystem object."""
+
+  def __init__(self, dataset_url, key=None, secret=None, endpoint=None, proxy=None, proxy_port=None,
+               hadoop_configuration=None, connector=HdfsConnector, hdfs_driver='libhdfs3'):
+    """
+    Given a dataset URL and an optional hadoop configuration, parse and interpret the URL to
+    instantiate a pyarrow filesystem.
+
+    Interpretation of the URL ``scheme://hostname:port/path`` occurs in the following order:
+
+    1. If no ``scheme``, no longer supported, so raise an exception!
+    2. If ``scheme`` is ``file``, use local filesystem path.
+    3. If ``scheme`` is ``hdfs``:
+       a. Try the ``hostname`` as a namespace and attempt to connect to a name node.
+          1. If that doesn't work, try connecting directly to namenode ``hostname:port``.
+       b. If no host, connect to the default name node.
+    5. If ``scheme`` is ``s3``, use s3fs. The user must manually install s3fs before using s3
+    6. Fail otherwise.
+
+    :param dataset_url: The hdfs URL or absolute path to the dataset
+    :param key: access key of obs
+    :param secret: secret key of obs
+    :param endpoint: endpoint of obs
+    :param proxy: proxy
+    :param proxy_port:  proxy_port
+    :param hadoop_configuration: an optional hadoop configuration
+    :param connector: the HDFS connector object to use (ONLY override for testing purposes)
+    :param hdfs_driver: A string denoting the hdfs driver to use (if using a dataset on hdfs). Current choices are
+    libhdfs (java through JNI) or libhdfs3 (C++)
+    """
+    # Cache both the original URL and the resolved, urlparsed dataset_url
+    self._dataset_url = dataset_url
+    self._parsed_dataset_url = None
+    # Cache the instantiated filesystem object
+    self._filesystem = None
+
+    if isinstance(self._dataset_url, six.string_types):
+      self._parsed_dataset_url = urlparse(self._dataset_url)
+    else:
+      self._parsed_dataset_url = self._dataset_url
+
+    if not self._parsed_dataset_url.scheme:
+      # Case 1
+      raise ValueError('ERROR! A scheme-less dataset url ({}) is no longer supported. '
+                       'Please prepend "file://" for local filesystem.'.format(self._parsed_dataset_url.scheme))
+
+    elif self._parsed_dataset_url.scheme == 'file':
+      # Case 2: definitely local
+      self._filesystem = pyarrow.localfs
+
+    elif self._parsed_dataset_url.scheme == 'hdfs':
+
+      if hdfs_driver == 'libhdfs3':
+        # libhdfs3 does not do any namenode resolution itself so we do it manually. This is not necessary
+        # if using libhdfs
+
+        # Obtain singleton and force hadoop config evaluation
+        namenode_resolver = HdfsNamenodeResolver(hadoop_configuration)
+
+        # Since we can't tell for sure, first treat the URL as though it references a name service
+        if self._parsed_dataset_url.netloc:
+          # Case 3a: Use the portion of netloc before any port, which doesn't get lowercased
+          nameservice = self._parsed_dataset_url.netloc.split(':')[0]
+          namenodes = namenode_resolver.resolve_hdfs_name_service(nameservice)
+          if namenodes:
+            self._filesystem = connector.connect_to_either_namenode(namenodes)
+          if self._filesystem is None:
+            # Case 3a1: That didn't work; try the URL as a namenode host
+            self._filesystem = connector.hdfs_connect_namenode(self._parsed_dataset_url)
+        else:
+          # Case 3b: No netloc, so let's try to connect to default namenode
+          # HdfsNamenodeResolver will raise exception if it fails to connect.
+          nameservice, namenodes = namenode_resolver.resolve_default_hdfs_service()
+          filesystem = connector.connect_to_either_namenode(namenodes)
+          if filesystem is not None:
+            # Properly replace the parsed dataset URL once default namenode is confirmed
+            self._parsed_dataset_url = urlparse(
+              'hdfs://{}{}'.format(nameservice, self._parsed_dataset_url.path))
+            self._filesystem = filesystem
+      else:
+        self._filesystem = connector.hdfs_connect_namenode(self._parsed_dataset_url, hdfs_driver)
+
+    elif self._parsed_dataset_url.scheme == "s3a":
+      # Case 5
+      # S3 support requires s3fs to be installed
+      try:
+        import s3fs
+      except ImportError:
+        raise ValueError('Must have s3fs installed in order to use datasets on s3. '
+                         'Please install s3fs and try again.')
+
+      if not self._parsed_dataset_url.netloc:
+        raise ValueError('URLs must be of the form s3://bucket/path')
+
+      if key is None or secret is None or endpoint is None:
+        raise ValueError('key, secret, endpoint should not be None')
+
+      http_proxy = 'http://' + proxy + ':' + str(proxy_port) if (
+        proxy is not None and proxy_port is not None) else None
+
+      https_proxy = 'https://' + proxy + ':' + str(proxy_port) if (
+        proxy is not None and proxy_port is not None) else None
+
+      config_kwargs = {'proxies': {'http': http_proxy, 'https': https_proxy}} if (
+        http_proxy is not None) else None
+
+      fs = s3fs.S3FileSystem(key=key,
+                             secret=secret,
+                             client_kwargs={'endpoint_url': endpoint},
+                             config_kwargs=config_kwargs)
+
+      self._filesystem = pyarrow.filesystem.S3FSWrapper(fs)
+
+    else:
+      # Case 6
+      raise ValueError('Unsupported scheme in dataset url {}. '
+                       'Currently, only "file" and "hdfs" are supported.'.format(self._parsed_dataset_url.scheme))
+
+  def parsed_dataset_url(self):
+    """
+    :return: The urlparse'd dataset_url
+    """
+    return self._parsed_dataset_url
+
+  def get_dataset_path(self):
+    """
+    The dataset path is different than the one in `_parsed_dataset_url` for some filesystems.
+    For example s3fs expects the bucket name to be included in the path and doesn't support
+    paths that start with a `/`
+    """
+    if isinstance(self._filesystem, pyarrow.filesystem.S3FSWrapper):
+      # s3fs expects paths of the form `bucket/path`
+      return self._parsed_dataset_url.netloc + self._parsed_dataset_url.path
+
+    return self._parsed_dataset_url.path
+
+  def filesystem(self):
+    """
+    :return: The pyarrow filesystem object
+    """
+    return self._filesystem
diff --git a/python/pycarbon/core/carbon_local_memory_cache.py b/python/pycarbon/core/carbon_local_memory_cache.py
new file mode 100644
index 0000000..9795c01
--- /dev/null
+++ b/python/pycarbon/core/carbon_local_memory_cache.py
@@ -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.
+
+
+from __future__ import division
+
+from petastorm.cache import CacheBase
+
+
+class LocalMemoryCache(CacheBase):
+  def __init__(self, size_limit_bytes):
+    """LocalMemoryCache is an adapter to a diskcache implementation.
+
+    LocalMemoryCache can be used by a pycarbon Reader class to temporarily keep parts of the dataset in local memory.
+
+    :param size_limit_bytes: Maximal size of the memory to be used by cache. The size of the cache may actually
+                             grow somewhat above the size_limit_bytes, so the limit is not very strict.
+    """
+
+    self._cache = dict()
+
+  def get(self, key, fill_cache_func):
+    value = self._cache.get(key)
+    if value is None:
+      value = fill_cache_func()
+      self._cache[key] = value
+
+    return value
+
+  def cleanup(self):
+    self._cache.clear()
+
+  def size(self):
+    return len(self._cache)
diff --git a/python/pycarbon/core/carbon_py_dict_reader_worker.py b/python/pycarbon/core/carbon_py_dict_reader_worker.py
new file mode 100644
index 0000000..60ef98e
--- /dev/null
+++ b/python/pycarbon/core/carbon_py_dict_reader_worker.py
@@ -0,0 +1,195 @@
+# 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.
+
+
+from __future__ import division
+
+import hashlib
+
+import numpy as np
+
+from petastorm import utils
+from petastorm.cache import NullCache
+from petastorm.workers_pool.worker_base import WorkerBase
+from petastorm.py_dict_reader_worker import PyDictReaderWorkerResultsQueueReader
+from petastorm.py_dict_reader_worker import _select_cols, _merge_two_dicts
+
+
+class PyDictCarbonReaderWorker(WorkerBase):
+  def __init__(self, worker_id, publish_func, args):
+    super(PyDictCarbonReaderWorker, self).__init__(worker_id, publish_func, args)
+
+    self._filesystem = args[0]
+    self._dataset_path = args[1]
+    self._schema = args[2]
+    self._ngram = args[3]
+    self._split_pieces = args[4]
+    self._local_cache = args[5]
+    self._transform_spec = args[6]
+
+    # We create datasets lazily in the first invocation of 'def process'. This speeds up startup time since
+    # all Worker constructors are serialized
+    self._dataset = None
+
+  @staticmethod
+  def new_results_queue_reader():
+    return PyDictReaderWorkerResultsQueueReader()
+
+  # pylint: disable=arguments-differ
+  def process(self, piece_index, worker_predicate, shuffle_row_drop_partition):
+    """Main worker function. Loads and returns all rows matching the predicate from a blocklet
+
+    Looks up the requested piece (a single row-group in a carbon file). If a predicate is specified,
+    columns needed by the predicate are loaded first. If no rows in the blocklet matches the predicate criteria
+    the rest of the columns are not loaded.
+
+    :param piece_index:
+    :param shuffle_row_drop_partition: A tuple 2 of the current row drop partition and the total number
+        of partitions.
+    :return:
+    """
+    # start = time.time()
+    piece = self._split_pieces[piece_index]
+
+    if not isinstance(self._local_cache, NullCache):
+      if worker_predicate:
+        raise RuntimeError('Local cache is not supported together with predicates, '
+                           'unless the dataset is partitioned by the column the predicate operates on.')
+      if shuffle_row_drop_partition[1] != 1:
+        raise RuntimeError('Local cache is not supported together with shuffle_row_drop_partitions > 1')
+
+    if worker_predicate:
+      all_cols = self._load_rows_with_predicate(piece, worker_predicate, shuffle_row_drop_partition)
+    else:
+      # Using hash of the dataset path with the relative path in order to:
+      #  1. Make sure if a common cache serves multiple processes (e.g. redis), we don't have conflicts
+      #  2. Dataset path is hashed, to make sure we don't create too long keys, which maybe incompatible with
+      #     some cache implementations
+      #  3. Still leave relative path and the piece_index in plain text to make it easier to debug
+      cache_key = '{}:{}:{}'.format(hashlib.md5(self._dataset_path.encode('utf-8')).hexdigest(),
+                                    piece.path, piece_index)
+      # start1 = time.time()
+      all_cols = self._local_cache.get(cache_key,
+                                       lambda: self._load_rows(piece, shuffle_row_drop_partition))
+      # print("cache time is " + str(time.time() - start))
+
+    if self._ngram:
+      all_cols = self._ngram.form_ngram(data=all_cols, schema=self._schema)
+
+    if all_cols:
+      self.publish_func(all_cols)
+    # print("process time is " + str(time.time() - start))
+
+  def _load_rows_with_predicate(self, piece, worker_predicate, shuffle_row_drop_partition):
+    """Loads all rows that match a predicate from a piece"""
+
+    # 1. Read all columns needed by predicate and decode
+    # 2. Apply the predicate. If nothing matches, exit early
+    # 3. Read the remaining columns and decode
+    # 4. Combine with columns already decoded for the predicate.
+
+    # Split all column names into ones that are needed by predicateand the rest.
+    predicate_column_names = set(worker_predicate.get_fields())
+
+    if not predicate_column_names:
+      raise ValueError('At least one field name must be returned by predicate\'s get_field() method')
+
+    all_schema_names = set(field.name for field in self._schema.fields.values())
+
+    invalid_column_names = predicate_column_names - all_schema_names
+    if invalid_column_names:
+      raise ValueError('At least some column names requested by the predicate ({}) '
+                       'are not valid schema names: ({})'.format(', '.join(invalid_column_names),
+                                                                 ', '.join(all_schema_names)))
+
+    other_column_names = all_schema_names - predicate_column_names
+    other_column_names_list = list(other_column_names)
+
+    predicate_column_names_list = list(predicate_column_names)
+    # Read columns needed for the predicate
+    predicate_rows = self._read_with_shuffle_row_drop(piece, predicate_column_names_list,
+                                                      shuffle_row_drop_partition)
+
+    # Decode values
+    transform_func = self._transform_spec.func if self._transform_spec else (lambda x: x)
+    decoded_predicate_rows = [
+      transform_func(utils.decode_row(_select_cols(row, predicate_column_names), self._schema))
+      for row in predicate_rows]
+
+    # Use the predicate to filter
+    match_predicate_mask = [worker_predicate.do_include(row) for row in decoded_predicate_rows]
+
+    # Don't have anything left after filtering? Exit early.
+    if not any(match_predicate_mask):
+      return []
+
+    # Remove rows that were filtered out by the predicate
+    filtered_decoded_predicate_rows = [row for i, row in enumerate(decoded_predicate_rows) if
+                                       match_predicate_mask[i]]
+
+    if other_column_names:
+      # Read remaining columns
+      other_rows = self._read_with_shuffle_row_drop(piece, other_column_names_list,
+                                                    shuffle_row_drop_partition)
+
+      # Remove rows that were filtered out by the predicate
+      filtered_other_rows = [row for i, row in enumerate(other_rows) if match_predicate_mask[i]]
+
+      # Decode remaining columns
+      decoded_other_rows = [utils.decode_row(row, self._schema) for row in filtered_other_rows]
+
+      # Merge predicate needed columns with the remaining
+      all_cols = [_merge_two_dicts(a, b) for a, b in zip(decoded_other_rows, filtered_decoded_predicate_rows)]
+      return all_cols
+    else:
+      return filtered_decoded_predicate_rows
+
+  def _load_rows(self, piece, shuffle_row_drop_range):
+    """Loads all rows from a piece"""
+
+    # pyarrow would fail if we request a column names that the dataset is partitioned by, so we strip them from
+    # the `columns` argument.
+    column_names = list(field.name for field in self._schema.fields.values())
+
+    all_rows = self._read_with_shuffle_row_drop(piece, column_names, shuffle_row_drop_range)
+
+    transform_func = self._transform_spec.func if self._transform_spec else (lambda x: x)
+    return [transform_func(utils.decode_row(row, self._schema)) for row in all_rows]
+
+  def _read_with_shuffle_row_drop(self, piece, column_names, shuffle_row_drop_partition):
+    # start = time.time()
+    data_frame = piece.read_all(
+      columns=column_names,
+    )
+    # print(" total piece time taken is " + str(time.time() - start))
+    # start = time.time()
+    data_frame = data_frame.to_pandas()
+    # print(" panda time is " + str(time.time() - start))
+
+    num_rows = len(data_frame)
+    num_partitions = shuffle_row_drop_partition[1]
+    this_partition = shuffle_row_drop_partition[0]
+
+    partition_indexes = np.floor(np.arange(num_rows) / (float(num_rows) / min(num_rows, num_partitions)))
+
+    if self._ngram:
+      # If we have an ngram we need to take elements from the next partition to build the sequence
+      next_partition_indexes = np.where(partition_indexes >= this_partition + 1)
+      if next_partition_indexes[0].size:
+        next_partition_to_add = next_partition_indexes[0][0:self._ngram.length - 1]
+        partition_indexes[next_partition_to_add] = this_partition
+
+    selected_dataframe = data_frame.loc[partition_indexes == this_partition]
+    return selected_dataframe.to_dict('records')
diff --git a/python/pycarbon/core/carbon_reader.py b/python/pycarbon/core/carbon_reader.py
new file mode 100644
index 0000000..7025895
--- /dev/null
+++ b/python/pycarbon/core/carbon_reader.py
@@ -0,0 +1,569 @@
+# 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.
+
+
+import collections
+import logging
+import warnings
+
+import six
+
+from petastorm.cache import NullCache
+from petastorm.local_disk_cache import LocalDiskCache
+from petastorm.ngram import NGram
+from petastorm.transform import transform_schema
+from petastorm.workers_pool.thread_pool import ThreadPool
+from petastorm.workers_pool.ventilator import ConcurrentVentilator
+
+from pycarbon.core.carbon_arrow_reader_worker import ArrowCarbonReaderWorker
+from pycarbon.core.carbon_py_dict_reader_worker import PyDictCarbonReaderWorker
+from pycarbon.core.carbon import CarbonDataset
+from pycarbon.core import carbon_dataset_metadata
+from pycarbon.core.carbon_dataset_metadata import infer_or_load_unischema_carbon
+from pycarbon.core.carbon_dataset_metadata import PycarbonMetadataError
+from pycarbon.core.carbon_local_memory_cache import LocalMemoryCache
+from pycarbon.core.carbon_fs_utils import CarbonFilesystemResolver
+
+logger = logging.getLogger(__name__)
+
+
+# Ventilator guarantees that no more than workers + _VENTILATE_EXTRA_BLOCKLETS are processed at a moment by a
+# worker pool. This guarantees that we don't run out of memory if data consumer is slower than the Reader.
+_VENTILATE_EXTRA_BLOCKLETS = 2
+
+
+def make_carbon_reader(dataset_url,
+                       key=None,
+                       secret=None,
+                       endpoint=None,
+                       proxy=None,
+                       proxy_port=None,
+                       schema_fields=None,
+                       reader_pool_type='thread', workers_count=10, results_queue_size=100,
+                       shuffle_blocklets=True, shuffle_row_drop_partitions=1,
+                       predicate=None,
+                       blocklet_selector=None,
+                       num_epochs=1,
+                       cur_shard=None, shard_count=None,
+                       cache_type='null', cache_location=None, cache_size_limit=None,
+                       cache_row_size_estimate=None, cache_extra_settings=None,
+                       hdfs_driver='libhdfs3',
+                       reader_engine='reader_v1', reader_engine_params=None,
+                       transform_spec=None):
+  """
+  Creates an instance of Reader for reading Pycarbon datasets. A Pycarbon dataset is a dataset generated using
+  :func:`~pycarbon.etl.carbon_dataset_metadata.materialize_dataset_carbon` context manager as explained
+
+  See :func:`~pycarbon.make_batch_carbon_reader` to read from a Carbon store that was not generated using
+  :func:`~pycarbon.etl.carbon_dataset_metadata.materialize_dataset_carbon`.
+
+  :param dataset_url: an filepath or a url to a carbon directory,
+      e.g. ``'hdfs://some_hdfs_cluster/user/yevgeni/carbon8'``, or ``'file:///tmp/mydataset'``
+      or ``'s3://bucket/mydataset'``.
+  :param key: access key
+  :param secret: secret key
+  :param endpoint: endpoint_url
+  :param proxy: proxy
+  :param proxy_port:  proxy_port
+  :param schema_fields: Can be: a list of unischema fields and/or regex pattern strings; ``None`` to read all fields;
+          an NGram object, then it will return an NGram of the specified fields.
+  :param reader_pool_type: A string denoting the reader pool type. Should be one of ['thread', 'process', 'dummy']
+      denoting a thread pool, process pool, or running everything in the master thread. Defaults to 'thread'
+    TODO: process support
+  :param workers_count: An int for the number of workers to use in the reader pool. This only is used for the
+      thread or process pool. Defaults to 10
+  :param results_queue_size: Size of the results queue to store prefetched rows. Currently only applicable to
+      thread reader pool type.
+  :param shuffle_blocklets: Whether to shuffle blocklets (the order in which full blocklets are read)
+  :param shuffle_row_drop_partitions: This is is a positive integer which determines how many partitions to
+      break up a blocklet into for increased shuffling in exchange for worse performance (extra reads).
+      For example if you specify 2 each blocklet read will drop half of the rows within every blocklet and
+      read the remaining rows in separate reads. It is recommended to keep this number below the regular row
+      group size in order to not waste reads which drop all rows.
+  :param predicate: instance of :class:`.PredicateBase` object to filter rows to be returned by reader. The predicate
+      will be passed a single row and must return a boolean value indicating whether to include it in the results.
+  :param blocklet_selector: instance of blocklet selector object to select blocklet to be read
+    TODO: blocklet_selector
+  :param num_epochs: An epoch is a single pass over all rows in the dataset. Setting ``num_epochs`` to
+      ``None`` will result in an infinite number of epochs.
+  :param cur_shard: An int denoting the current shard number. Each node reading a shard should
+      pass in a unique shard number in the range [0, shard_count). shard_count must be supplied as well.
+      Defaults to None
+  :param shard_count: An int denoting the number of shards to break this dataset into. Defaults to None
+    TODO: cur_shard & shard_count
+  :param cache_type: A string denoting the cache type, if desired. Options are [None, 'null', 'local-disk'] to
+      either have a null/noop cache or a cache implemented using diskcache. Caching is useful when communication
+      to the main data store is either slow or expensive and the local machine has large enough storage
+      to store entire dataset (or a partition of a dataset if shard_count is used). By default will be a null cache.
+  :param cache_location: A string denoting the location or path of the cache.
+  :param cache_size_limit: An int specifying the size limit of the cache in bytes
+  :param cache_row_size_estimate: An int specifying the estimated size of a row in the dataset
+  :param cache_extra_settings: A dictionary of extra settings to pass to the cache implementation,
+  :param hdfs_driver: A string denoting the hdfs driver to use (if using a dataset on hdfs). Current choices are
+      libhdfs (java through JNI) or libhdfs3 (C++)
+  :param reader_engine: Multiple engine implementations exist ('reader_v1' and 'experimental_reader_v2'). 'reader_v1'
+      (the default value) selects a stable reader implementation.
+    TODO: experimental_reader_v2 for carbon
+  :param reader_engine_params: For advanced usage: a dictionary with arguments passed directly to a reader
+      implementation constructor chosen by ``reader_engine`` argument.  You should not use this parameter, unless you
+      fine-tuning of a reader.
+  :param transform_spec: An instance of :class:`~petastorm.transform.TransformSpec` object defining how a record
+      is transformed after it is loaded and decoded. The transformation occurs on a worker thread/process (depends
+      on the ``reader_pool_type`` value).
+  :return: A :class:`Reader` object
+  """
+
+  if dataset_url is None or not isinstance(dataset_url, six.string_types):
+    raise ValueError("""dataset_url must be a string""")
+
+  dataset_url = dataset_url[:-1] if dataset_url[-1] == '/' else dataset_url
+  logger.debug('dataset_url: %s', dataset_url)
+
+  resolver = CarbonFilesystemResolver(dataset_url,
+                                      key=key,
+                                      secret=secret,
+                                      endpoint=endpoint,
+                                      proxy=proxy,
+                                      proxy_port=proxy_port,
+                                      hdfs_driver=hdfs_driver)
+  filesystem = resolver.filesystem()
+
+  if cache_type is None or cache_type == 'null':
+    cache = NullCache()
+  elif cache_type == 'local-disk':
+    cache = LocalDiskCache(cache_location, cache_size_limit, cache_row_size_estimate, **cache_extra_settings or {})
+  elif cache_type == 'memory-cache':
+    cache = LocalMemoryCache(cache_size_limit)
+  else:
+    raise ValueError('Unknown cache_type: {}'.format(cache_type))
+
+  # Fail if this is a non-pycarbon dataset. Typically, a Carbon store will have hundred thousands rows in a single
+  # blocklet. Using PyDictCarbonReaderWorker or ReaderV2 implementation is very inefficient as it processes data on a
+  # row by row basis. ArrowCarbonReaderWorker (used by make_batch_carbon_reader) is much more efficient in these cases.
+  try:
+    carbon_dataset_metadata.get_schema_from_dataset_url_carbon(dataset_url,
+                                                               key=key,
+                                                               secret=secret,
+                                                               endpoint=endpoint,
+                                                               proxy=proxy,
+                                                               proxy_port=proxy_port,
+                                                               filesystem=filesystem)
+  except PycarbonMetadataError:
+    raise RuntimeError('Currently make_carbon_reader supports reading only Pycarbon datasets(has unischema). '
+                       'To read from a non-Pycarbon Carbon store use make_batch_carbon_reader')
+
+  if reader_engine == 'reader_v1':
+    if reader_pool_type == 'thread':
+      reader_pool = ThreadPool(workers_count, results_queue_size)
+    elif reader_pool_type == 'process':
+      raise NotImplementedError('not support process reader_pool_type now.')
+    elif reader_pool_type == 'dummy':
+      raise NotImplementedError('not support dummy reader_pool_type now.')
+    else:
+      raise ValueError('Unknown reader_pool_type: {}'.format(reader_pool_type))
+
+    # Create a dictionary with all ReaderV2 parameters, so we can merge with reader_engine_params if specified
+    kwargs = {
+      'key': key,
+      'secret': secret,
+      'endpoint': endpoint,
+      'proxy': proxy,
+      'proxy_port': proxy_port,
+      'schema_fields': schema_fields,
+      'reader_pool': reader_pool,
+      'shuffle_blocklets': shuffle_blocklets,
+      'shuffle_row_drop_partitions': shuffle_row_drop_partitions,
+      'predicate': predicate,
+      'blocklet_selector': blocklet_selector,
+      'num_epochs': num_epochs,
+      'cur_shard': cur_shard,
+      'shard_count': shard_count,
+      'cache': cache,
+      'transform_spec': transform_spec,
+    }
+
+    if reader_engine_params:
+      kwargs.update(reader_engine_params)
+
+    try:
+      return CarbonDataReader(filesystem, dataset_url,
+                              worker_class=PyDictCarbonReaderWorker,
+                              **kwargs)
+    except PycarbonMetadataError as e:
+      logger.error('Unexpected exception: %s', str(e))
+      raise RuntimeError('make_carbon_reader has failed. If you were trying to open a Carbon store that was not '
+                         'created using Pycarbon materialize_dataset_carbon and it contains only scalar columns, '
+                         'you may use make_batch_reader to read it.\n'
+                         'Inner exception: %s', str(e))
+
+  elif reader_engine == 'experimental_reader_v2':
+    raise NotImplementedError('not support experimental_reader_v2 reader engine now.')
+  else:
+    raise ValueError('Unexpected value of reader_engine argument \'%s\'. '
+                     'Supported reader_engine values are \'reader_v1\' and \'experimental_reader_v2\'',
+                     reader_engine)
+
+
+def make_batch_carbon_reader(dataset_url,
+                             key=None,
+                             secret=None,
+                             endpoint=None,
+                             proxy=None,
+                             proxy_port=None,
+                             schema_fields=None,
+                             reader_pool_type='thread', workers_count=10, results_queue_size=100,
+                             shuffle_blocklets=True, shuffle_row_drop_partitions=1,
+                             predicate=None,
+                             blocklet_selector=None,
+                             num_epochs=1,
+                             cur_shard=None, shard_count=None,
+                             cache_type='null', cache_location=None, cache_size_limit=None,
+                             cache_row_size_estimate=None, cache_extra_settings=None,
+                             hdfs_driver='libhdfs3',
+                             transform_spec=None):
+  """
+  Creates an instance of Reader for reading batches out of a non-Pycarbon Carbon store.
+
+  Currently, only stores having native scalar carbon data types are supported.
+  Use :func:`~pycarbon.make_carbon_reader` to read Pycarbon Carbon stores generated with
+  :func:`~pycarbon.etl.carbon_dataset_metadata.materialize_dataset_carbon`.
+
+  NOTE: only scalar columns are currently supported.
+
+  :param dataset_url: an filepath or a url to a carbon directory,
+      e.g. ``'hdfs://some_hdfs_cluster/user/yevgeni/carbon8'``, or ``'file:///tmp/mydataset'``
+      or ``'s3://bucket/mydataset'``.
+  :param key: access key
+  :param secret: secret key
+  :param endpoint: endpoint_url
+  :param proxy: proxy
+  :param proxy_port:  proxy_port
+  :param schema_fields: A list of regex pattern strings. Only columns matching at least one of the
+      patterns in the list will be loaded.
+  :param reader_pool_type: A string denoting the reader pool type. Should be one of ['thread', 'process', 'dummy']
+      denoting a thread pool, process pool, or running everything in the master thread. Defaults to 'thread'
+  :param workers_count: An int for the number of workers to use in the reader pool. This only is used for the
+      thread or process pool. Defaults to 10
+  :param results_queue_size: Size of the results queue to store prefetched rows. Currently only applicable to
+      thread reader pool type.
+  :param shuffle_blocklets: Whether to shuffle blocklets (the order in which full blocklets are read)
+  :param shuffle_row_drop_partitions: This is is a positive integer which determines how many partitions to
+      break up a blocklet into for increased shuffling in exchange for worse performance (extra reads).
+      For example if you specify 2 each blocklet read will drop half of the rows within every blocklet and
+      read the remaining rows in separate reads. It is recommended to keep this number below the regular row
+      group size in order to not waste reads which drop all rows.
+  :param predicate: instance of :class:`.PredicateBase` object to filter rows to be returned by reader. The predicate
+      will be passed a pandas DataFrame object and must return a pandas Series with boolean values of matching
+      dimensions.
+  :param blocklet_selector: instance of blocklet selector object to select blocklets to be read
+  :param num_epochs: An epoch is a single pass over all rows in the dataset. Setting ``num_epochs`` to
+      ``None`` will result in an infinite number of epochs.
+  :param cur_shard: An int denoting the current shard number. Each node reading a shard should
+      pass in a unique shard number in the range [0, shard_count). shard_count must be supplied as well.
+      Defaults to None
+  :param shard_count: An int denoting the number of shards to break this dataset into. Defaults to None
+  :param cache_type: A string denoting the cache type, if desired. Options are [None, 'null', 'local-disk'] to
+      either have a null/noop cache or a cache implemented using diskcache. Caching is useful when communication
+      to the main data store is either slow or expensive and the local machine has large enough storage
+      to store entire dataset (or a partition of a dataset if shard_count is used). By default will be a null cache.
+  :param cache_location: A string denoting the location or path of the cache.
+  :param cache_size_limit: An int specifying the size limit of the cache in bytes
+  :param cache_row_size_estimate: An int specifying the estimated size of a row in the dataset
+  :param cache_extra_settings: A dictionary of extra settings to pass to the cache implementation,
+  :param hdfs_driver: A string denoting the hdfs driver to use (if using a dataset on hdfs). Current choices are
+      libhdfs (java through JNI) or libhdfs3 (C++)
+  :param transform_spec: An instance of :class:`~petastorm.transform.TransformSpec` object defining how a record
+      is transformed after it is loaded and decoded. The transformation occurs on a worker thread/process (depends
+      on the ``reader_pool_type`` value).
+  :return: A :class:`Reader` object
+  """
+
+  if dataset_url is None or not isinstance(dataset_url, six.string_types):
+    raise ValueError("""dataset_url must be a string""")
+
+  dataset_url = dataset_url[:-1] if dataset_url[-1] == '/' else dataset_url
+  logger.debug('dataset_url: %s', dataset_url)
+
+  resolver = CarbonFilesystemResolver(dataset_url,
+                                      key=key,
+                                      secret=secret,
+                                      endpoint=endpoint,
+                                      proxy=proxy,
+                                      proxy_port=proxy_port,
+                                      hdfs_driver=hdfs_driver)
+  filesystem = resolver.filesystem()
+
+  try:
+    carbon_dataset_metadata.get_schema_from_dataset_url_carbon(dataset_url,
+                                                               key=key,
+                                                               secret=secret,
+                                                               endpoint=endpoint,
+                                                               proxy=proxy,
+                                                               proxy_port=proxy_port,
+                                                               filesystem=filesystem)
+    warnings.warn('Please use make_carbon_reader (instead of \'make_batch_carbon_reader\' function '
+                  'to read this dataset as it contains unischema file.')
+  except PycarbonMetadataError:
+    pass
+
+  if cache_type is None or cache_type == 'null':
+    cache = NullCache()
+  elif cache_type == 'local-disk':
+    cache = LocalDiskCache(cache_location, cache_size_limit, cache_row_size_estimate,
+                           **cache_extra_settings or {})
+  elif cache_type == 'memory-cache':
+    cache = LocalMemoryCache(cache_size_limit)
+  else:
+    raise ValueError('Unknown cache_type: {}'.format(cache_type))
+
+  if reader_pool_type == 'thread':
+    reader_pool = ThreadPool(workers_count, results_queue_size)
+  elif reader_pool_type == 'process':
+    raise NotImplementedError('not support process reader_pool_type now.')
+  elif reader_pool_type == 'dummy':
+    raise NotImplementedError('not support dummy reader_pool_type now.')
+  else:
+    raise ValueError('Unknown reader_pool_type: {}'.format(reader_pool_type))
+
+  return CarbonDataReader(filesystem, dataset_url,
+                          key=key, secret=secret, endpoint=endpoint,
+                          proxy=proxy, proxy_port=proxy_port,
+                          schema_fields=schema_fields,
+                          worker_class=ArrowCarbonReaderWorker,
+                          reader_pool=reader_pool,
+                          shuffle_blocklets=shuffle_blocklets,
+                          shuffle_row_drop_partitions=shuffle_row_drop_partitions,
+                          predicate=predicate,
+                          blocklet_selector=blocklet_selector,
+                          num_epochs=num_epochs,
+                          cur_shard=cur_shard,
+                          shard_count=shard_count,
+                          cache=cache,
+                          transform_spec=transform_spec)
+
+
+class CarbonDataReader(object):
+  """Reads a dataset from a Pycarbon dataset.
+
+  :ivar last_row_consumed: True if the last row was already returned by the Reader.
+  """
+
+  def __init__(self, pyarrow_filesystem, dataset_path,
+               key=None, secret=None, endpoint=None,
+               proxy=None, proxy_port=None,
+               schema_fields=None,
+               shuffle_blocklets=True, shuffle_row_drop_partitions=1,
+               predicate=None, blocklet_selector=None, reader_pool=None, num_epochs=1,
+               cur_shard=None, shard_count=None, cache=None, worker_class=None,
+               transform_spec=None):
+    """Initializes a reader object.
+
+    :param pyarrow_filesystem: An instance of ``pyarrow.FileSystem`` that will be used. If not specified,
+        then a default one will be selected based on the url (only for ``hdfs://`` or ``file://``; for
+        ``s3://`` support, use ``make_reader``). The default hdfs driver is ``libhdfs3``. If you want
+        to to use ``libhdfs``, use
+        ``pyarrow_filesystem=pyarrow.hdfs.connect('hdfs:///some/path', driver='libhdfs')``.
+    :param dataset_path: filepath to a carbon directory on the specified filesystem.
+        e.g. ``'/user/yevgeni/carbon8'``, or ``'/tmp/mydataset'``.
+    :param key: access key
+    :param secret: secret key
+    :param endpoint: endpoint_url
+    :param proxy: proxy
+    :param proxy_port:  proxy_port
+    :param schema_fields: Either list of unischema fields to subset, or ``None`` to read all fields.
+        OR an NGram object, then it will return an NGram of the specified properties.
+    :param shuffle_blocklets: Whether to shuffle blocklets (the order in which full blocklets are read)
+    :param shuffle_row_drop_partitions: This is is a positive integer which determines how many partitions to
+        break up a blocklet into for increased shuffling in exchange for worse performance (extra reads).
+        For example if you specify 2 each blocklet read will drop half of the rows within every blocklet and
+        read the remaining rows in separate reads. It is recommended to keep this number below the regular row
+        group size in order to not waste reads which drop all rows.
+    :param predicate: instance of predicate object to filter rows to be returned by reader.
+    :param blocklet_selector: instance of blocklet selector object to select blocklets to be read
+    :param reader_pool: parallelization pool. ``ThreadPool(10)`` (10 threads) is used by default.
+        This pool is a custom implementation used to parallelize reading data from the dataset.
+        Any object from workers_pool package can be used
+        (e.g. :class:`petastorm.workers_pool.process_pool.ProcessPool`).
+    :param num_epochs: An epoch is a single pass over all rows in the dataset. Setting ``num_epochs`` to
+        ``None`` will result in an infinite number of epochs.
+    :param cur_shard: An int denoting the current shard number used. Each reader instance should
+        pass in a unique shard number in the range ``[0, shard_count)``.
+        ``shard_count`` must be supplied as well. Defaults to None
+    :param shard_count: An int denoting the number of shard partitions there are. Defaults to None
+    :param cache: An object conforming to :class:`.CacheBase` interface. Before loading blocklets from a carbon
+        file the Reader will attempt to load these values from cache. Caching is useful when communication
+        to the main data store is either slow or expensive and the local machine has large enough storage
+        to store entire dataset (or a partition of a dataset if shards are used).
+        By default, use the :class:`.NullCache` implementation.
+
+    :param worker_class: This is the class that will be instantiated on a different thread/process. It's
+        responsibility is to load and filter the data.
+    """
+
+    # 1. Open the carbon storage (dataset) & Get a list of all blocklets
+    # 2. Filter blocklets
+    # 4. Create a blocklet ventilator object
+    # 5. Start workers pool
+    if not (isinstance(schema_fields, collections.Iterable) or isinstance(schema_fields, NGram)
+            or schema_fields is None):
+      raise ValueError("""Fields must be either None, an iterable collection of Unischema fields or an NGram
+            object.""")
+
+    self.ngram = schema_fields if isinstance(schema_fields, NGram) else None
+
+    # By default, use original method of working with list of dictionaries and not arrow tables
+    worker_class = worker_class or PyDictCarbonReaderWorker
+    self._results_queue_reader = worker_class.new_results_queue_reader()
+
+    if self.ngram and not self.ngram.timestamp_overlap and shuffle_row_drop_partitions > 1:
+      raise NotImplementedError('Using timestamp_overlap=False is not implemented with'
+                                ' shuffle_options.shuffle_row_drop_partitions > 1')
+
+    self.cache = cache or NullCache()
+
+    self._workers_pool = reader_pool or ThreadPool(10)
+    # 1. Resolve dataset path (hdfs://, file://) and open the carbon storage (dataset)
+    self.carbon_dataset = CarbonDataset(dataset_path,
+                                        key=key,
+                                        secret=secret,
+                                        endpoint=endpoint,
+                                        proxy=proxy,
+                                        proxy_port=proxy_port,
+                                        filesystem=pyarrow_filesystem)
+    stored_schema = infer_or_load_unischema_carbon(self.carbon_dataset)
+
+    # Make a schema view (a view is a Unischema containing only a subset of fields
+    # Will raise an exception if invalid schema fields are in schema_fields
+    fields = schema_fields if isinstance(schema_fields, collections.Iterable) else None
+    storage_schema = stored_schema.create_schema_view(fields) if fields else stored_schema
+    if transform_spec:
+      self.schema = transform_schema(storage_schema, transform_spec)
+    else:
+      self.schema = storage_schema
+
+    # 2. Filter blocklets
+    filtered_blocklet_indexes = list(range(len(self.carbon_dataset.pieces)))
+    worker_predicate = predicate
+
+    # 3. Create a blocklet ventilator object
+    normalized_shuffle_row_drop_partitions = \
+      self._normalize_shuffle_options(shuffle_row_drop_partitions, self.carbon_dataset)
+    self.ventilator = self._create_ventilator(filtered_blocklet_indexes, shuffle_blocklets,
+                                              normalized_shuffle_row_drop_partitions, num_epochs, worker_predicate,
+                                              self._workers_pool.workers_count + _VENTILATE_EXTRA_BLOCKLETS)
+
+    # 4. Start workers pool
+    self._workers_pool.start(worker_class, (pyarrow_filesystem, dataset_path, storage_schema, self.ngram,
+                                            self.carbon_dataset.pieces, cache, transform_spec),
+                             ventilator=self.ventilator)
+    logger.debug('Workers pool started')
+
+    self.last_row_consumed = False
+
+  def reset(self):
+    """Resets ``Reader`` state and allows to fetch more samples once the ``Reader`` finished reading all epochs,
+    as specified by the ``num_epochs`` parameter.
+
+    Once all samples were read from a reader, an attempt to fetch new sample (e.g. ``next(reader)`` would raise
+    ``StopIterationError``. You can reset the reader to the original state and restart reading samples
+    calling ``reset()``.
+
+    We do not support calling ``reset()`` until all samples were consumed. ``NotImplementedError``
+    will be raised if a user attempt to do so.
+
+    Calling reset after ``stop()`` was called has no effect.
+
+    :return: None
+    """
+    if not self.last_row_consumed:
+      raise NotImplementedError('Currently do not support resetting a reader while in the middle of iteration. '
+                                'You can call reset only after all samples were consumed.')
+    self.last_row_consumed = False
+    self.ventilator.reset()
+
+  @property
+  def batched_output(self):
+    return self._results_queue_reader.batched_output
+
+  @staticmethod
+  def _normalize_shuffle_options(shuffle_row_drop_partitions, carbonSplit):
+    """Checks that shuffle_options doesnt ask for more patitions than rows in a blocklet.
+    This prevents sending partitions to workers which will result in not reading anything."""
+    if shuffle_row_drop_partitions > 1 and carbonSplit.number_of_splits > 0:
+      max_rows_in_blocklet = 1
+      for i in six.moves.xrange(carbonSplit.number_of_splits):
+        max_rows_in_blocklet = max(max_rows_in_blocklet, carbonSplit.pieces.__getitem__(i).num_rows)
+      return min(shuffle_row_drop_partitions, max_rows_in_blocklet)
+    return shuffle_row_drop_partitions
+
+  def _create_ventilator(self, blocklet_indexes, shuffle_blocklets, shuffle_row_drop_partitions,
+                         num_epochs, worker_predicate, max_ventilation_queue_size):
+    items_to_ventilate = []
+    for piece_index in blocklet_indexes:
+      for shuffle_row_drop_partition in range(shuffle_row_drop_partitions):
+        items_to_ventilate.append(
+          {'piece_index': piece_index,
+           'worker_predicate': worker_predicate,
+           'shuffle_row_drop_partition': (shuffle_row_drop_partition,
+                                          shuffle_row_drop_partitions)})
+
+    return ConcurrentVentilator(self._workers_pool.ventilate,
+                                items_to_ventilate,
+                                iterations=num_epochs,
+                                max_ventilation_queue_size=max_ventilation_queue_size,
+                                randomize_item_order=shuffle_blocklets)
+
+  def stop(self):
+    """Stops all worker threads/processes."""
+    self._workers_pool.stop()
+
+  def join(self):
+    """Joins all worker threads/processes. Will block until all worker workers have been fully terminated."""
+    self._workers_pool.join()
+
+  def cleanup(self):
+    if not isinstance(self.cache, NullCache):
+      self.cache.cleanup()
+
+  def exit(self):
+    self.stop()
+    self.join()
+    self.cleanup()
+
+  @property
+  def diagnostics(self):
+    return self._workers_pool.diagnostics
+
+  def __iter__(self):
+    return self
+
+  def __next__(self):
+    try:
+      return self._results_queue_reader.read_next(self._workers_pool, self.schema, self.ngram)
+    except StopIteration:
+      self.last_row_consumed = True
+      raise
+
+  def next(self):
+    return self.__next__()
+
+  # Functions needed to treat reader as a context manager
+  def __enter__(self):
+    return self
+
+  def __exit__(self, exc_type, exc_val, exc_tb):
+    self.stop()
+    self.join()
+    self.cleanup()
diff --git a/python/pycarbon/core/carbon_tf_utils.py b/python/pycarbon/core/carbon_tf_utils.py
new file mode 100644
index 0000000..2775556
--- /dev/null
+++ b/python/pycarbon/core/carbon_tf_utils.py
@@ -0,0 +1,403 @@
+# 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.
+
+
+"""A set of TensorFlow specific helper functions"""
+
+import datetime
+import sys
+import warnings
+from calendar import timegm
+from collections import OrderedDict, namedtuple
+from decimal import Decimal
+
+import numpy as np
+import tensorflow as tf
+
+# Mapping of identical datatypes in numpy-ish and TensorFlow-ish
+_NUMPY_TO_TF_DTYPES_MAPPING = {
+  np.bool: tf.bool,
+  np.int8: tf.int8,
+  np.int16: tf.int16,
+  np.int32: tf.int32,
+  np.int64: tf.int64,
+  np.uint8: tf.uint8,
+  np.uint16: tf.int32,
+  np.float32: tf.float32,
+  np.float64: tf.float64,
+  np.string_: tf.string,
+  np.unicode_: tf.string,
+  np.str_: tf.string,
+  np.bool_: tf.bool,
+  Decimal: tf.string,
+  np.datetime64: tf.int64,
+}
+
+# Name of an op in the TF graph used for the random shuffling queue. This name can be used by diagnostics code that
+# wishes to read-out shuffling queue size
+RANDOM_SHUFFLING_QUEUE_SIZE = 'random_shuffling_queue_size'
+
+
+def date_to_nsec_from_epoch(dt):
+  return timegm(dt.timetuple()) * 1000000000
+
+
+_date_to_nsec_from_epoch_vectorized = np.vectorize(date_to_nsec_from_epoch)
+
+
+def _sanitize_field_tf_types(sample):
+  """Takes a named tuple and casts/promotes types unknown to TF to the types that are known.
+
+  Three casts that are currently implemented
+    - Decimal to string
+    - uint16 to int32
+    - np.datetime64 to int64, as nanoseconds since unix epoch
+
+  :param sample: named tuple or a dictionary
+  :return: same type as the input with values casted to types supported by Tensorflow
+  """
+  next_sample_dict = sample._asdict()
+
+  for k, v in next_sample_dict.items():
+    if v is None:
+      raise RuntimeError('Encountered "{}"=None. Tensorflow does not support None values as a tensor.'
+                         'Consider filtering out these rows using a predicate.'.format(k))
+    # Assuming conversion to the same numpy type is trivial and dirty cheap
+    if isinstance(v, Decimal):
+      # Normalizing decimals only to get rid of the trailing zeros (makes testing easier, assuming has
+      # no other effect)
+      next_sample_dict[k] = str(v.normalize())
+    elif isinstance(v, np.ndarray) and np.issubdtype(v.dtype, np.datetime64):
+      # Convert to nanoseconds from POSIX epoch
+      next_sample_dict[k] = (v - np.datetime64('1970-01-01T00:00:00.0')) \
+        .astype('timedelta64[ns]').astype(np.int64)
+    elif isinstance(v, np.ndarray) and v.dtype == np.uint16:
+      next_sample_dict[k] = v.astype(np.int32)
+    elif isinstance(v, np.ndarray) and v.dtype.type in (np.bytes_, np.unicode_):
+      if v.size != 0:
+        next_sample_dict[k] = v.tolist()
+    elif isinstance(v, np.ndarray) and v.dtype.kind == 'O' and isinstance(v[0], datetime.date):
+      # Pyarrow 0.12.1 started returning python datetime.date when parquet column is a DateType() column.
+      # Convert values in such column into nsec from epoch int64.
+      next_sample_dict[k] = _date_to_nsec_from_epoch_vectorized(v)
+
+  # Construct object of the same type as the input
+  return sample.__class__(**next_sample_dict)
+
+
+def _schema_to_tf_dtypes(schema):
+  """Returns schema as a list of tensorflow dtypes.
+  :param schema: The schema.
+  :return: List of tensorflow dtypes.
+  """
+  return [_numpy_to_tf_dtypes(f.numpy_dtype) for f in schema.fields.values()]
+
+
+def _schema_to_tf_dtypes_ngram(schema, ngram):
+  """Returns schema as a list of tensorflow dtypes for a ngram.
+  :param schema: The schema.
+  :param ngram: The ngram.
+  :return: tensorflow dtypes for a ngram.
+  """
+  result = []
+  # Iterate over each timestep
+  for key in sorted(ngram.fields.keys()):
+    # Get schema at that timestep
+    new_schema = ngram.get_schema_at_timestep(schema=schema, timestep=key)
+    for field in new_schema.fields.values():
+      result.append(_numpy_to_tf_dtypes(field.numpy_dtype))
+  return result
+
+
+def _numpy_to_tf_dtypes(numpy_dtype):
+  """Returns a tensorflow dtype object corresponding to numpy's dtype.
+
+  A :class:`ValueError` is raised if there is no known mapping between the types
+
+  :param numpy_dtype: numpy dtype object
+  :return: tensorflow dtype object
+  """
+  if numpy_dtype in _NUMPY_TO_TF_DTYPES_MAPPING:
+    if numpy_dtype == np.unicode_ and sys.version_info >= (3, 0):
+      warnings.warn("Tensorflow will convert all unicode strings back to bytes type. "
+                    "You may need to decode values.", UnicodeWarning)
+    return _NUMPY_TO_TF_DTYPES_MAPPING[numpy_dtype]
+  else:
+    raise ValueError('Unknown mapping of numpy {} to tensorflow dtype'.format(numpy_dtype))
+
+
+def _flatten(data):
+  """Flattens the data, where it takes a dictionary of timesteps, each value is a dictionary and converts it to
+  one flat dictionary having a key that is the key of the inner dictionary + '_' + timestep.
+
+  For example, ``data`` would be ``{1: {'a': 'avalue', 'b': 'bvalue'}, 2: {'c': 'cvalue', 'd': 'dvalue'}}`` and the
+  output of :func:`._flatten` would be ``{'a_1': 'avalue', 'b_1': 'bvalue', 'c_2': 'cvalue', 'd_2': 'dvalue'}``.
+
+  :param data: The data to flatten.
+  :return: The flattened dictionary.
+  """
+  flattened = OrderedDict()
+  for index, key in enumerate(sorted(data.keys())):
+    data_dict = data[key]._asdict()
+    for subkey in data_dict:
+      encoded_key = subkey + '_' + str(index)
+      flattened[encoded_key] = data_dict[subkey]
+
+  FlattenedTuple = namedtuple('flattened', list(flattened.keys()))
+  return FlattenedTuple(**flattened)
+
+
+def make_namedtuple_tf_ngram(unischema, ngram, *args, **kargs):
+  """Creates a dictionary of timestep keys and namedtuple values from args and kargs.
+
+  :param ngram: The ngram definition.
+  :param args: args.
+  :param kargs: kargs.
+  :return: A dictionary of timestep keys and namedtuple values.
+  """
+
+  ngram_result = {}
+  previous_args_end = 0
+  for timestep in range(min(ngram.fields.keys()), max(ngram.fields.keys()) + 1):
+    # For each timestep iteration, mark the args and kargs for that timestep and create
+    # a namedtuple from them.
+    current_field_names = ngram.get_field_names_at_timestep(timestep)
+    new_schema = ngram.get_schema_at_timestep(schema=unischema, timestep=timestep)
+    new_args_end = previous_args_end + len(current_field_names)
+    args_timestep = args[previous_args_end:new_args_end]
+    previous_args_end = new_args_end
+    kargs_timestep = (kargs[str(timestep)] if str(timestep) in kargs else {})
+    ngram_result[timestep] = new_schema._get_namedtuple()(*args_timestep, **kargs_timestep)
+  return ngram_result
+
+
+def _set_shape(schema, fields_as_dict, batched_output=None):
+  # Assign static shape for all tensors
+  # Workaround of an issue described here:
+  # https://stackoverflow.com/questions/49161316/trailing-x00-characters-in-tensor-when-numpy-string-array-is-returned-from-tf
+  for k in fields_as_dict.keys():
+    unischema_field = schema.fields[k]
+
+    if batched_output:
+      shape = (None,) + unischema_field.shape
+    else:
+      shape = unischema_field.shape
+    # Set static shape
+    fields_as_dict[k].set_shape(shape)
+
+
+def _shuffling_queue(shuffling_queue_capacity, min_after_dequeue, dtypes, fields_as_list):
+  """Creates a shuffling queue with enqueue/dequeue pair. Always a single writing thread."""
+
+  # Named tuples loose the 'named' part when going via queue
+  shuffling_queue = tf.RandomShuffleQueue(shuffling_queue_capacity, min_after_dequeue, dtypes)
+
+  # The following call to .size has a side effect of creating a new node in the TF graph. We are interested
+  # in the side effect so we can read the queue size somewhere else, addressing the node by a 'well-known-name'
+  shuffling_queue.size(name=RANDOM_SHUFFLING_QUEUE_SIZE)
+
+  # We need the queue only for shuffling, so we use only a single enqueuing thread (actually would be happy
+  # not to introduce any threads. Not sure if there is such a mechanism in TF)
+  queue_runner = tf.train.QueueRunner(shuffling_queue, 1 * [shuffling_queue.enqueue(fields_as_list)])
+
+  tf.train.add_queue_runner(queue_runner)
+
+  # Passed through the queue. We got an ordered list. The order matches the order of fields in unischema
+  fields_as_list = shuffling_queue.dequeue()
+  return fields_as_list
+
+
+def _tf_tensors_nonngram(reader, shuffling_queue_capacity, min_after_dequeue):
+  """A tensorflow data adapter for non ngrams. Return value is a named tuple with tensorflow tensors supplying
+  the data directly into a Tensoflow graph. See `tf_tensor` documentation for input/output arguments meaning."""
+
+  # TODO: implement a mechanism for signaling that we have no more data
+  def dequeue_sample_impl(x):
+    next_sample = next(reader)
+    # Decimal is not supported by TF. int8,16,32,64 scalars are all returned as python native int type
+    # (casted to 64 bit by tensorflow). sanitize_field_tf_types will explicitly convert all values
+    # to explicit numpy types making it compatible with return values expected by Tensorflow
+    return _sanitize_field_tf_types(next_sample)
+
+  # fields_as_list is a list with tensors matching the order of the values in the schema. named-tuple semantics is
+  # not preserved across tf.py_func call boundary.
+  fields_as_list = tf.py_func(dequeue_sample_impl, [tf.constant(1)], _schema_to_tf_dtypes(reader.schema))
+
+  if shuffling_queue_capacity > 0:
+    # Pass py_func output via shuffling queue if requested.
+    fields_as_list = _shuffling_queue(shuffling_queue_capacity, min_after_dequeue,
+                                      _schema_to_tf_dtypes(reader.schema), fields_as_list)
+
+  # Going via `make_namedtuple_tf` is a little wasteful, since we are converting directly to dict. However, this
+  # spares the need to implement a function similar to make_namedtuple_tf that returns dict instead of a named tuple
+  fields_as_dict = reader.schema.make_namedtuple_tf(*fields_as_list)._asdict()
+
+  # Force all static shapes to be set in the returned value based on the unischema
+  _set_shape(reader.schema, fields_as_dict, reader.batched_output)
+
+  # Make a row tensor into a nice named tuple
+  return reader.schema.make_namedtuple_tf(**fields_as_dict)
+
+
+def _tf_tensors_ngram(reader, shuffling_queue_capacity, min_after_dequeue):
+  """A tensorflow data adapter for ngrams. Return value is a named tuple with tensorflow tensors supplying
+  the data directly into a Tensoflow graph. See `tf_tensor` documentation for input/output arguments meaning."""
+
+  # TODO: implement a mechanism for signaling that we have no more data
+  def dequeue_sample_impl(x):
+    next_sample = next(reader)
+    assert isinstance(next_sample, dict)
+
+    # Create a dictionary, where each key is a timestep, and value is named tuple or dictionary.
+    ngram = {}
+    for timestep in next_sample:
+      ngram[timestep] = _sanitize_field_tf_types(next_sample[timestep])
+
+    return _flatten(ngram)
+
+  fields_as_list = tf.py_func(dequeue_sample_impl, [tf.constant(1)],
+                              _schema_to_tf_dtypes_ngram(reader.schema, reader.ngram))
+
+  if shuffling_queue_capacity > 0:
+    # Pass py_func output via shuffling queue if requested.
+    fields_as_list = _shuffling_queue(shuffling_queue_capacity, min_after_dequeue,
+                                      _schema_to_tf_dtypes_ngram(reader.schema, reader.ngram), fields_as_list)
+
+  fields_as_namedtuple = make_namedtuple_tf_ngram(reader.schema, reader.ngram, *fields_as_list)
+
+  # We change the key to str format here in order to be able to use ** later to expand the dictionary as kargs.
+  fields_as_dict = {
+    str(timestep): fields_as_namedtuple[timestep]._asdict() for timestep in fields_as_namedtuple}
+  for timestep in fields_as_dict:
+    _set_shape(reader.schema, fields_as_dict[timestep])
+
+  return make_namedtuple_tf_ngram(reader.schema, reader.ngram, **fields_as_dict)
+
+
+def tf_tensors(reader, shuffling_queue_capacity=0, min_after_dequeue=0):
+  """Bridges between python-only interface of the Reader (next(Reader)) and tensorflow world.
+
+  This function returns a named tuple of tensors from the dataset, e.g.,
+
+  >>> row_tensors
+  >>> Out[2]: TestSchema_view(field_1=<tf.Tensor 'PyFunc:0' shape=() dtype=string>,
+  >>>         field_2=<tf.Tensor 'StringSplit:1' shape=(?,) dtype=string>,
+  >>>         field_3=<tf.Tensor 'PyFunc:2' shape=() dtype=int64>, ...)
+
+  If the reader was created with ``ngram=NGram(...)`` parameter, then a dictionary of named tuples is returned
+  (indexed by time):
+
+  >>> row_tensors
+  >>> Out[6]:
+  >>> {0: TestSchema_view(field_1=<tf.Tensor 'PyFunc_4:0' shape=() dtype=string>, field_2=...),
+  >>>  1: TestSchema_view(field_1=<tf.Tensor 'PyFunc_4:11' shape=() dtype=string>, field_2=...),
+  >>>  2: TestSchema_view(field_1=<tf.Tensor 'PyFunc_4:22' shape=() dtype=string>, field_2=...)}
+
+  An optional shuffling queue is created if shuffling_queue_capacity is greater than 0.
+
+  Note that if reading a unischema field that is unicode (``np.unicode_`` or ``np.str_``) tensorflow will
+  represent it as a tf.string which will be an array of bytes. If using python3 you may need to decode
+  it to convert it back to a python str type.
+
+  :param reader: An instance of pycarbon.CarbonDataReader object used as the data source
+  :param shuffling_queue_capacity: Queue capacity is passed to the underlying :class:`tf.RandomShuffleQueue`
+      instance. If set to 0, no suffling will be done.
+  :param min_after_dequeue: If ``shuffling_queue_capacity > 0``, this value is passed to the underlying
+      :class:`tf.RandomShuffleQueue`.
+  :return: If no ngram reading is used, the function will return a named tuple with tensors that are populated
+      from the underlying dataset. If ngram reading is enabled, a dictionary of named tuples of tensors is returned.
+      The dictionary is indexed by time.
+  """
+
+  # NGram enabled and disabled code is quite different. It appears to be cleaner to simply go in orthogonal
+  # execution paths.
+
+  if reader.batched_output:
+    if shuffling_queue_capacity > 0:
+      raise ValueError('shuffling_queue_capacity can not be used with a reader that produces '
+                       'batched_output, since each batch is a carbon read blocklet. Extra '
+                       'shuffling of the batches does not further decrease correlation.')
+
+  if reader.ngram:
+    result = _tf_tensors_ngram(reader, shuffling_queue_capacity, min_after_dequeue)
+  else:
+    result = _tf_tensors_nonngram(reader, shuffling_queue_capacity, min_after_dequeue)
+
+  return result
+
+
+def _set_shape_to_named_tuple(schema, fields, batched_output):
+  """Assign static shape for all tensors"""
+  fields_as_dict = fields._asdict()
+  _set_shape(schema, fields_as_dict, batched_output)
+  return schema.make_namedtuple_tf(**fields_as_dict)
+
+
+def make_pycarbon_dataset(reader):
+  """Creates a `tensorflow.data.Dataset <https://www.tensorflow.org/api_docs/python/tf/data/Dataset>`_ object from
+  a Pycarbon :class:`~pycarbon.carbon_reader.CarbonDataReader`.
+
+  The returned object can be used as any ``tf.data.Dataset`` with some limitations described below.
+
+  * ``repeat``: An error will be raised if you call ``repeat`` on the returned dataset. Please use ``num_epochs``
+    argument of the :meth:`~pycarbon.carbon_reader.CarbonDataReader` constructor.
+  * ``shard``: Consider using ``training_partition`` and ``num_training_partitions`` arguments of the
+    :class:`~pycarbon.carbon_reader.CarbonDataReader` constructor as it will not load any unused shards.
+  * ``filter``: Consider using :class:`~pycarbon.carbon_reader.CarbonDataReader` ``predicate`` constructor argument.
+    It will make use of columnar nature of the underlying Apache Parquet store to load only the columns that the
+    predicate operates on prior to loading and decoding other columns. :class:`~pycarbon.carbon_reader.
+    CarbonDataReader`'s predicate feature will also make use of Parquet partitioning (if the dataset is partitioned).
+
+  The elements produced by the returned dataset object are namedtuples based on the
+  :class:`~petastorm.unischema.Unischema`.
+
+  >>> import tensorflow as tf
+  >>> from pycarbon.core.carbon_reader import CarbonDataReader
+  >>> from pycarbon.core.carbon_tf_utils import make_pycarbon_dataset
+  >>>
+  >>> with CarbonDataReader('file:///some/path') as reader:
+  >>>     dataset = make_pycarbon_dataset(reader)
+  >>>     next_sample = dataset.make_one_shot_iterator().get_next()
+  >>>     with tf.Session() as sess:
+  >>>         x = sess.run(next_sample)
+
+
+  NGrams are not yet supported by this function.
+
+  :param reader: An instance of :class:`~pycarbon.carbon_reader.CarbonDataReader` object that would serve
+    as a data source.
+  :return: A ``tf.data.Dataset`` instance.
+  """
+
+  if not reader.ngram:
+
+    def dequeue_sample_impl():
+      if reader.last_row_consumed:
+        # This means that Dataset is trying to create a new instance of the generator. Can not do that
+        # (nor want to do that) since this is an expensive operation. num_epochs is a more efficient way
+        # to do this.
+        raise RuntimeError('Multiple iterations over make_pycarbon_dataset are not supported. '
+                           'Multiple iterations can be triggered by calling \'repeat\' method of Dataset class.'
+                           'Use Reader\'s num_epochs contructor arguments to set number of iterations.')
+      for row in reader:
+        yield _sanitize_field_tf_types(row)
+
+    flat_dataset = tf.data.Dataset.from_generator(dequeue_sample_impl, tuple(_schema_to_tf_dtypes(reader.schema)))
+    named_tuple_dataset = flat_dataset \
+      .map(reader.schema.make_namedtuple_tf) \
+      .map(lambda row: _set_shape_to_named_tuple(reader.schema, row, reader.batched_output))
+    return named_tuple_dataset
+  else:
+    raise NotImplementedError('make_pycarbon_dataset does not support NGram yet.')
diff --git a/python/pycarbon/core/carbon_utils.py b/python/pycarbon/core/carbon_utils.py
new file mode 100644
index 0000000..b132b3f
--- /dev/null
+++ b/python/pycarbon/core/carbon_utils.py
@@ -0,0 +1,72 @@
+# 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.
+
+
+import logging
+import os
+
+import pyarrow
+
+from pyarrow.filesystem import LocalFileSystem
+
+logger = logging.getLogger(__name__)
+
+
+def add_to_dataset_metadata_carbon(carbon_dataset, key, value):
+  """
+  Adds a key and value to the carbon metadata file of a carbon dataset.
+  :param dataset: (CarbonDataset) carbon dataset
+  :param key:     (str) key of metadata entry
+  :param value:   (str) value of metadata
+  """
+  if not isinstance(carbon_dataset.path, str):
+    raise ValueError('Expected dataset.paths to be a single path, not a list of paths')
+
+  metadata_file_path = carbon_dataset.path.rstrip('/') + '/_metadata'
+  common_metadata_file_path = carbon_dataset.path.rstrip('/') + '/_common_metadata'
+  common_metadata_file_crc_path = carbon_dataset.path.rstrip('/') + '/._common_metadata.crc'
+
+  # TODO currenlty usinf carbon to read and write _common_metadat, need to handle in carbon
+  # If the metadata file already exists, add to it.
+  # Otherwise fetch the schema from one of the existing carbon files in the dataset
+  if carbon_dataset.fs.exists(common_metadata_file_path):
+    with carbon_dataset.fs.open(common_metadata_file_path) as f:
+      arrow_metadata = pyarrow.parquet.read_metadata(f)
+      base_schema = arrow_metadata.schema.to_arrow_schema()
+  elif carbon_dataset.fs.exists(metadata_file_path):
+    # If just the metadata file exists and not the common metadata file, copy the contents of
+    # the metadata file to the common_metadata file for backwards compatibility
+    with carbon_dataset.fs.open(metadata_file_path) as f:
+      arrow_metadata = pyarrow.parquet.read_metadata(f)
+      base_schema = arrow_metadata.schema.to_arrow_schema()
+  else:
+    base_schema = carbon_dataset.schema
+
+  # base_schema.metadata may be None, e.g.
+  metadata_dict = base_schema.metadata or dict()
+  metadata_dict[key] = value
+  schema = base_schema.add_metadata(metadata_dict)
+
+  with carbon_dataset.fs.open(common_metadata_file_path, 'wb') as metadata_file:
+    pyarrow.parquet.write_metadata(schema, metadata_file)
+
+  # We have just modified _common_metadata file, but the filesystem implementation used by pyarrow does not
+  # update the .crc value. We better delete the .crc to make sure there is no mismatch between _common_metadata
+  # content and the checksum.
+  if isinstance(carbon_dataset.fs, LocalFileSystem) and carbon_dataset.fs.exists(common_metadata_file_crc_path):
+    try:
+      carbon_dataset.fs.rm(common_metadata_file_crc_path)
+    except NotImplementedError:
+      os.remove(common_metadata_file_crc_path)
diff --git a/python/pycarbon/integration/__init__.py b/python/pycarbon/integration/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/integration/__init__.py
diff --git a/python/pycarbon/integration/tensorflow.py b/python/pycarbon/integration/tensorflow.py
new file mode 100644
index 0000000..26620cd
--- /dev/null
+++ b/python/pycarbon/integration/tensorflow.py
@@ -0,0 +1,358 @@
+# 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.
+
+
+"""A set of TensorFlow specific helper functions"""
+
+import datetime
+import sys
+import warnings
+from calendar import timegm
+from collections import OrderedDict, namedtuple
+from decimal import Decimal
+
+import numpy as np
+import tensorflow as tf
+
+from pycarbon.core.carbon_tf_utils import RANDOM_SHUFFLING_QUEUE_SIZE, make_namedtuple_tf_ngram
+
+
+class TensorFlow(object):
+  def __init__(self):
+    self
+
+  # Mapping of identical datatypes in numpy-ish and tensorflow-ish
+  _NUMPY_TO_TF_DTYPES_MAPPING = {
+    np.bool: tf.bool,
+    np.int8: tf.int8,
+    np.int16: tf.int16,
+    np.int32: tf.int32,
+    np.int64: tf.int64,
+    np.uint8: tf.uint8,
+    np.uint16: tf.int32,
+    np.float32: tf.float32,
+    np.float64: tf.float64,
+    np.string_: tf.string,
+    np.unicode_: tf.string,
+    np.str_: tf.string,
+    np.bool_: tf.bool,
+    Decimal: tf.string,
+    np.datetime64: tf.int64,
+  }
+
+  # Name of an op in the TF graph used for the random shuffling queue. This name can be used by diagnostics code that
+  # wishes to read-out shuffling queue size
+  RANDOM_SHUFFLING_QUEUE_SIZE = 'random_shuffling_queue_size'
+
+  def date_to_nsec_from_epoch(self, dt):
+    return timegm(dt.timetuple()) * 1000000000
+
+  _date_to_nsec_from_epoch_vectorized = np.vectorize(date_to_nsec_from_epoch)
+
+  def _sanitize_field_tf_types(self, sample):
+    """Takes a named tuple and casts/promotes types unknown to TF to the types that are known.
+
+    Three casts that are currently implemented
+      - Decimal to string
+      - uint16 to int32
+      - np.datetime64 to int64, as nanoseconds since unix epoch
+
+    :param sample: named tuple or a dictionary
+    :return: same type as the input with values casted to types supported by Tensorflow
+    """
+    next_sample_dict = sample._asdict()
+
+    for k, v in next_sample_dict.items():
+      if v is None:
+        raise RuntimeError('Encountered "{}"=None. Tensorflow does not support None values as a tensor.'
+                           'Consider filtering out these rows using a predicate.'.format(k))
+      # Assuming conversion to the same numpy type is trivial and dirty cheap
+      if isinstance(v, Decimal):
+        # Normalizing decimals only to get rid of the trailing zeros (makes testing easier, assuming has
+        # no other effect)
+        next_sample_dict[k] = str(v.normalize())
+      elif isinstance(v, np.ndarray) and np.issubdtype(v.dtype, np.datetime64):
+        # Convert to nanoseconds from POSIX epoch
+        next_sample_dict[k] = (v - np.datetime64('1970-01-01T00:00:00.0')) \
+          .astype('timedelta64[ns]').astype(np.int64)
+      elif isinstance(v, np.ndarray) and v.dtype == np.uint16:
+        next_sample_dict[k] = v.astype(np.int32)
+      elif isinstance(v, np.ndarray) and v.dtype.type in (np.bytes_, np.unicode_):
+        if v.size != 0:
+          next_sample_dict[k] = v.tolist()
+      elif isinstance(v, np.ndarray) and v.dtype.kind == 'O' and isinstance(v[0], datetime.date):
+        # Pyarrow 0.12.1 started returning python datetime.date when parquet column is a DateType() column.
+        # Convert values in such column into nsec from epoch int64.
+        next_sample_dict[k] = self._date_to_nsec_from_epoch_vectorized(v)
+
+    # Construct object of the same type as the input
+    return sample.__class__(**next_sample_dict)
+
+  def _schema_to_tf_dtypes(self, schema):
+    """Returns schema as a list of tensorflow dtypes.
+    :param schema: The schema.
+    :return: List of tensorflow dtypes.
+    """
+    return [self._numpy_to_tf_dtypes(f.numpy_dtype) for f in schema.fields.values()]
+
+  def _schema_to_tf_dtypes_ngram(self, schema, ngram):
+    """Returns schema as a list of tensorflow dtypes for a ngram.
+    :param schema: The schema.
+    :param ngram: The ngram.
+    :return: tensorflow dtypes for a ngram.
+    """
+    result = []
+    # Iterate over each timestep
+    for key in sorted(ngram.fields.keys()):
+      # Get schema at that timestep
+      new_schema = ngram.get_schema_at_timestep(schema=schema, timestep=key)
+      for field in new_schema.fields.values():
+        result.append(self._numpy_to_tf_dtypes(field.numpy_dtype))
+    return result
+
+  def _numpy_to_tf_dtypes(self, numpy_dtype):
+    """Returns a tensorflow dtype object corresponding to numpy's dtype.
+
+    A :class:`ValueError` is raised if there is no known mapping between the types
+
+    :param numpy_dtype: numpy dtype object
+    :return: tensorflow dtype object
+    """
+    if numpy_dtype in self._NUMPY_TO_TF_DTYPES_MAPPING:
+      if numpy_dtype == np.unicode_ and sys.version_info >= (3, 0):
+        warnings.warn("Tensorflow will convert all unicode strings back to bytes type. "
+                      "You may need to decode values.", UnicodeWarning)
+      return self._NUMPY_TO_TF_DTYPES_MAPPING[numpy_dtype]
+    else:
+      raise ValueError('Unknown mapping of numpy {} to tensorflow dtype'.format(numpy_dtype))
+
+  def _flatten(self, data):
+    """Flattens the data, where it takes a dictionary of timesteps, each value is a dictionary and converts it to
+    one flat dictionary having a key that is the key of the inner dictionary + '_' + timestep.
+
+    For example, ``data`` would be ``{1: {'a': 'avalue', 'b': 'bvalue'}, 2: {'c': 'cvalue', 'd': 'dvalue'}}`` and the
+    output of :func:`._flatten` would be ``{'a_1': 'avalue', 'b_1': 'bvalue', 'c_2': 'cvalue', 'd_2': 'dvalue'}``.
+
+    :param data: The data to flatten.
+    :return: The flattened dictionary.
+    """
+    flattened = OrderedDict()
+    for index, key in enumerate(sorted(data.keys())):
+      data_dict = data[key]._asdict()
+      for subkey in data_dict:
+        encoded_key = subkey + '_' + str(index)
+        flattened[encoded_key] = data_dict[subkey]
+
+    FlattenedTuple = namedtuple('flattened', list(flattened.keys()))
+    return FlattenedTuple(**flattened)
+
+  def make_namedtuple_tf_ngram(self, unischema, ngram, *args, **kargs):
+    """Creates a dictionary of timestep keys and namedtuple values from args and kargs.
+
+    :param ngram: The ngram definition.
+    :param args: args.
+    :param kargs: kargs.
+    :return: A dictionary of timestep keys and namedtuple values.
+    """
+
+    ngram_result = {}
+    previous_args_end = 0
+    for timestep in range(min(ngram.fields.keys()), max(ngram.fields.keys()) + 1):
+      # For each timestep iteration, mark the args and kargs for that timestep and create
+      # a namedtuple from them.
+      current_field_names = ngram.get_field_names_at_timestep(timestep)
+      new_schema = ngram.get_schema_at_timestep(schema=unischema, timestep=timestep)
+      new_args_end = previous_args_end + len(current_field_names)
+      args_timestep = args[previous_args_end:new_args_end]
+      previous_args_end = new_args_end
+      kargs_timestep = (kargs[str(timestep)] if str(timestep) in kargs else {})
+      ngram_result[timestep] = new_schema._get_namedtuple()(*args_timestep, **kargs_timestep)
+    return ngram_result
+
+  def _set_shape(self, schema, fields_as_dict, batched_output=None):
+    # Assign static shape for all tensors
+    # Workaround of an issue described here:
+    # https://stackoverflow.com/questions/49161316/trailing-x00-characters-in-tensor-when-numpy-string-array-is-returned-from-tf
+    for k in fields_as_dict.keys():
+      unischema_field = schema.fields[k]
+
+      if batched_output:
+        shape = (None,) + unischema_field.shape
+      else:
+        shape = unischema_field.shape
+      # Set static shape
+      fields_as_dict[k].set_shape(shape)
+
+  def _shuffling_queue(self, shuffling_queue_capacity, min_after_dequeue, dtypes, fields_as_list):
+    """Creates a shuffling queue with enqueue/dequeue pair. Always a single writing thread."""
+
+    # Named tuples loose the 'named' part when going via queue
+    shuffling_queue = tf.RandomShuffleQueue(shuffling_queue_capacity, min_after_dequeue, dtypes)
+
+    # The following call to .size has a side effect of creating a new node in the TF graph. We are interested
+    # in the side effect so we can read the queue size somewhere else, addressing the node by a 'well-known-name'
+    shuffling_queue.size(name=RANDOM_SHUFFLING_QUEUE_SIZE)
+
+    # We need the queue only for shuffling, so we use only a single enqueuing thread (actually would be happy
+    # not to introduce any threads. Not sure if there is such a mechanism in TF)
+    queue_runner = tf.train.QueueRunner(shuffling_queue, 1 * [shuffling_queue.enqueue(fields_as_list)])
+
+    tf.train.add_queue_runner(queue_runner)
+
+    # Passed through the queue. We got an ordered list. The order matches the order of fields in unischema
+    fields_as_list = shuffling_queue.dequeue()
+    return fields_as_list
+
+  def _tf_tensors_nonngram(self, reader, shuffling_queue_capacity, min_after_dequeue):
+    """A tensorflow data adapter for non ngrams. Return value is a named tuple with tensorflow tensors supplying
+    the data directly into a Tensoflow graph. See `tf_tensor` documentation for input/output arguments meaning."""
+
+    # TODO: implement a mechanism for signaling that we have no more data
+    def dequeue_sample_impl(x):
+      next_sample = next(reader)
+      # Decimal is not supported by TF. int8,16,32,64 scalars are all returned as python native int type
+      # (casted to 64 bit by tensorflow). sanitize_field_tf_types will explicitly convert all values
+      # to explicit numpy types making it compatible with return values expected by Tensorflow
+      return self._sanitize_field_tf_types(next_sample)
+
+    # fields_as_list is a list with tensors matching the order of the values in the schema. named-tuple semantics is
+    # not preserved across tf.py_func call boundary.
+    fields_as_list = tf.py_func(dequeue_sample_impl, [tf.constant(1)], self._schema_to_tf_dtypes(reader.schema))
+
+    if shuffling_queue_capacity > 0:
+      # Pass py_func output via shuffling queue if requested.
+      fields_as_list = self._shuffling_queue(shuffling_queue_capacity, min_after_dequeue,
+                                             self._schema_to_tf_dtypes(reader.schema), fields_as_list)
+
+    # Going via `make_namedtuple_tf` is a little wasteful, since we are converting directly to dict. However, this
+    # spares the need to implement a function similar to make_namedtuple_tf that returns dict instead of a named tuple
+    fields_as_dict = reader.schema.make_namedtuple_tf(*fields_as_list)._asdict()
+
+    # Force all static shapes to be set in the returned value based on the unischema
+    self._set_shape(reader.schema, fields_as_dict, reader.batched_output)
+
+    # Make a row tensor into a nice named tuple
+    return reader.schema.make_namedtuple_tf(**fields_as_dict)
+
+  def _tf_tensors_ngram(self, reader, shuffling_queue_capacity, min_after_dequeue):
+    """A tensorflow data adapter for ngrams. Return value is a named tuple with tensorflow tensors supplying
+    the data directly into a Tensoflow graph. See `tf_tensor` documentation for input/output arguments meaning."""
+
+    # TODO: implement a mechanism for signaling that we have no more data
+    def dequeue_sample_impl(x):
+      next_sample = next(reader)
+      assert isinstance(next_sample, dict)
+
+      # Create a dictionary, where each key is a timestep, and value is named tuple or dictionary.
+      ngram = {}
+      for timestep in next_sample:
+        ngram[timestep] = self._sanitize_field_tf_types(next_sample[timestep])
+
+      return self._flatten(ngram)
+
+    fields_as_list = tf.py_func(dequeue_sample_impl, [tf.constant(1)],
+                                self._schema_to_tf_dtypes_ngram(reader.schema, reader.ngram))
+
+    if shuffling_queue_capacity > 0:
+      # Pass py_func output via shuffling queue if requested.
+      fields_as_list = self._shuffling_queue(shuffling_queue_capacity, min_after_dequeue,
+                                             self._schema_to_tf_dtypes_ngram(reader.schema, reader.ngram),
+                                             fields_as_list)
+
+    fields_as_namedtuple = make_namedtuple_tf_ngram(reader.schema, reader.ngram, *fields_as_list)
+
+    # We change the key to str format here in order to be able to use ** later to expand the dictionary as kargs.
+    fields_as_dict = {
+      str(timestep): fields_as_namedtuple[timestep]._asdict() for timestep in fields_as_namedtuple}
+    for timestep in fields_as_dict:
+      self._set_shape(reader.schema, fields_as_dict[timestep])
+
+    return make_namedtuple_tf_ngram(reader.schema, reader.ngram, **fields_as_dict)
+
+  def make_tensor(self, reader, shuffling_queue_capacity=0, min_after_dequeue=0):
+    """Bridges between python-only interface of the Reader (next(Reader)) and tensorflow world.
+
+    This function returns a named tuple of tensors from the dataset, e.g.,
+
+    If the reader was created with ``ngram=NGram(...)`` parameter, then a dictionary of named tuples is returned
+    (indexed by time):
+
+    An optional shuffling queue is created if shuffling_queue_capacity is greater than 0.
+
+    Note that if reading a unischema field that is unicode (``np.unicode_`` or ``np.str_``) tensorflow will
+    represent it as a tf.string which will be an array of bytes. If using python3 you may need to decode
+    it to convert it back to a python str type.
+
+    :param reader: An instance of Reader object used as the data source
+    :param shuffling_queue_capacity: Queue capacity is passed to the underlying :class:`tf.RandomShuffleQueue`
+        instance. If set to 0, no suffling will be done.
+    :param min_after_dequeue: If ``shuffling_queue_capacity > 0``, this value is passed to the underlying
+        :class:`tf.RandomShuffleQueue`.
+    :return: If no ngram reading is used, the function will return a named tuple with tensors that are populated
+        from the underlying dataset. If ngram reading is enabled, a dictionary of named tuples of tensors is returned.
+        The dictionary is indexed by time.
+    """
+
+    # NGram enabled and disabled code is quite different. It appears to be cleaner to simply go in orthogonal
+    # execution paths.
+
+    if reader.batched_output:
+      if shuffling_queue_capacity > 0:
+        raise ValueError('shuffling_queue_capacity can not be used with a reader that produces '
+                         'batched_output. Extra shuffling of the batches does not further '
+                         'decrease correlation.')
+
+    if reader.ngram:
+      result = self._tf_tensors_ngram(reader, shuffling_queue_capacity, min_after_dequeue)
+    else:
+      result = self._tf_tensors_nonngram(reader, shuffling_queue_capacity, min_after_dequeue)
+
+    return result
+
+  def _set_shape_to_named_tuple(self, schema, fields, batched_output):
+    """Assign static shape for all tensors"""
+    fields_as_dict = fields._asdict()
+    self._set_shape(schema, fields_as_dict, batched_output)
+    return schema.make_namedtuple_tf(**fields_as_dict)
+
+  def make_dataset(self, reader):
+    """Creates a `tensorflow.data.Dataset <https://www.tensorflow.org/api_docs/python/tf/data/Dataset>`_ object from
+
+    NGrams are not yet supported by this function.
+
+    :param reader: An instance of :class:`Reader` object that would serve as a data source.
+    :return: A ``tf.data.Dataset`` instance.
+    """
+
+    if not reader.ngram:
+
+      def dequeue_sample_impl():
+        if reader.last_row_consumed:
+          # This means that Dataset is trying to create a new instance of the generator. Can not do that
+          # (nor want to do that) since this is an expensive operation. num_epochs is a more efficient way
+          # to do this.
+          raise RuntimeError('Multiple iterations over make_dataset are not supported. '
+                             'Multiple iterations can be triggered by calling \'repeat\' method of Dataset class.'
+                             'Use Reader\'s num_epochs constructor arguments to set number of iterations.')
+        for row in reader:
+          yield self._sanitize_field_tf_types(row)
+
+      flat_dataset = tf.data.Dataset.from_generator(dequeue_sample_impl, tuple(self._schema_to_tf_dtypes(reader.schema)))
+      named_tuple_dataset = flat_dataset \
+        .map(reader.schema.make_namedtuple_tf) \
+        .map(lambda row: self._set_shape_to_named_tuple(reader.schema, row, reader.batched_output))
+      return named_tuple_dataset
+    else:
+      raise NotImplementedError('make_dataset does not support NGram yet.')
diff --git a/python/pycarbon/reader.py b/python/pycarbon/reader.py
new file mode 100644
index 0000000..8b7fb3e
--- /dev/null
+++ b/python/pycarbon/reader.py
@@ -0,0 +1,202 @@
+# 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.
+
+
+from obs import ObsClient
+
+from pycarbon.core.carbon_reader import make_carbon_reader, make_batch_carbon_reader
+from pycarbon.integration.tensorflow import TensorFlow
+
+
+def make_reader(dataset_url=None,
+                workers_count=10,
+                results_queue_size=100,
+                num_epochs=1,
+                obs_client=None,
+                shuffle=True,
+                schema_fields=None,
+                is_batch=True,
+                reader_pool_type='thread',
+                data_format='carbon',
+                cache_properties={'cache_type': None, 'cache_location': None, 'cache_size_limit': None,
+                                  'cache_row_size_estimate': None, 'cache_extra_settings': None},
+                **properties
+                ):
+  """
+  an unified api for different data format dataset
+
+  :param dataset_url: an filepath or a url to a carbon directory,
+      e.g. ``'hdfs://some_hdfs_cluster/user/yevgeni/carbon8'``, or ``'file:///tmp/mydataset'``
+      or ``'s3a://bucket/mydataset'``.
+  :param data_format: dataset data format (default: carbon)
+  :param is_batch: return single record or batch records (default: True)
+  :param obs_client: obs client object
+    access key
+    secret key
+    endpoint_url
+  :param schema_fields: Can be: a list of unischema fields and/or regex pattern strings; ``None`` to read all fields;
+          an NGram object, then it will return an NGram of the specified fields.
+  :param reader_pool_type: A string denoting the reader pool type. Should be one of ['thread', 'process', 'dummy']
+      denoting a thread pool, process pool, or running everything in the master thread. Defaults to 'thread'
+    TODO: process support
+  :param workers_count: An int for the number of workers to use in the reader pool. This only is used for the
+      thread or process pool. Defaults to 10
+  :param results_queue_size: Size of the results queue to store prefetched rows. Currently only applicable to
+      thread reader pool type.
+  :param shuffle: Whether to shuffle partition (the order in which full partition are read)
+  :param num_epochs: An epoch is a single pass over all rows in the dataset. Setting ``num_epochs`` to
+      ``None`` will result in an infinite number of epochs.
+  :param cache_properties: a dict of cache parameters
+    cache_type: A string denoting the cache type, if desired. Options are [None, 'null', 'local-disk', 'memory-cache']
+      to either have a null/noop cache or a cache implemented using diskcache. Caching is useful when communication
+      to the main data store is either slow or expensive and the local machine has large enough storage
+      to store entire dataset. By default will be a null cache.
+    cache_location: A string denoting the location or path of the cache.
+    cache_size_limit: An int specifying the size limit of the cache in bytes
+    cache_row_size_estimate: An int specifying the estimated size of a row in the dataset
+    cache_extra_settings: A dictionary of extra settings to pass to the cache implementation,
+  :param **properties: other parameters (using dict)
+  :return: A :class:`Reader` object
+  """
+
+  if is_batch is True:
+    if data_format == 'carbon':
+      if isinstance(obs_client, ObsClient):
+        if obs_client.is_secure is True:
+          endpoint = "https://" + obs_client.server
+        else:
+          endpoint = "http://" + obs_client.server
+        return make_batch_carbon_reader(dataset_url,
+                                        key=obs_client.securityProvider.access_key_id,
+                                        secret=obs_client.securityProvider.secret_access_key,
+                                        endpoint=endpoint,
+                                        proxy=obs_client.proxy_host,
+                                        proxy_port=obs_client.proxy_port,
+                                        schema_fields=schema_fields,
+                                        reader_pool_type=reader_pool_type,
+                                        workers_count=workers_count,
+                                        results_queue_size=results_queue_size,
+                                        shuffle_blocklets=shuffle,
+                                        num_epochs=num_epochs,
+                                        cache_type=cache_properties['cache_type'],
+                                        cache_location=cache_properties['cache_location'],
+                                        cache_size_limit=cache_properties['cache_size_limit'],
+                                        cache_row_size_estimate=cache_properties['cache_row_size_estimate'],
+                                        cache_extra_settings=cache_properties['cache_extra_settings'],
+                                        **properties)
+      elif obs_client is None:
+        return make_batch_carbon_reader(dataset_url,
+                                        schema_fields=schema_fields,
+                                        reader_pool_type=reader_pool_type,
+                                        workers_count=workers_count,
+                                        results_queue_size=results_queue_size,
+                                        shuffle_blocklets=shuffle,
+                                        num_epochs=num_epochs,
+                                        cache_type=cache_properties['cache_type'],
+                                        cache_location=cache_properties['cache_location'],
+                                        cache_size_limit=cache_properties['cache_size_limit'],
+                                        cache_row_size_estimate=cache_properties['cache_row_size_estimate'],
+                                        cache_extra_settings=cache_properties['cache_extra_settings'],
+                                        **properties)
+      else:
+        raise ValueError("""obs_client should be a ObsClient object or None""")
+    else:
+      raise NotImplementedError("""not support other data format datset""")
+
+  elif is_batch is False:
+    if data_format == 'carbon':
+      if isinstance(obs_client, ObsClient):
+        if obs_client.is_secure is True:
+          endpoint = "https://" + obs_client.server
+        else:
+          endpoint = "http://" + obs_client.server
+        return make_carbon_reader(dataset_url,
+                                  key=obs_client.securityProvider.access_key_id,
+                                  secret=obs_client.securityProvider.secret_access_key,
+                                  endpoint=endpoint,
+                                  proxy=obs_client.proxy_host,
+                                  proxy_port=obs_client.proxy_port,
+                                  schema_fields=schema_fields,
+                                  reader_pool_type=reader_pool_type,
+                                  workers_count=workers_count,
+                                  results_queue_size=results_queue_size,
+                                  shuffle_blocklets=shuffle,
+                                  num_epochs=num_epochs,
+                                  cache_type=cache_properties['cache_type'],
+                                  cache_location=cache_properties['cache_location'],
+                                  cache_size_limit=cache_properties['cache_size_limit'],
+                                  cache_row_size_estimate=cache_properties['cache_row_size_estimate'],
+                                  cache_extra_settings=cache_properties['cache_extra_settings'],
+                                  **properties)
+      elif obs_client is None:
+        return make_carbon_reader(dataset_url,
+                                  schema_fields=schema_fields,
+                                  reader_pool_type=reader_pool_type,
+                                  workers_count=workers_count,
+                                  results_queue_size=results_queue_size,
+                                  shuffle_blocklets=shuffle,
+                                  num_epochs=num_epochs,
+                                  cache_type=cache_properties['cache_type'],
+                                  cache_location=cache_properties['cache_location'],
+                                  cache_size_limit=cache_properties['cache_size_limit'],
+                                  cache_row_size_estimate=cache_properties['cache_row_size_estimate'],
+                                  cache_extra_settings=cache_properties['cache_extra_settings'],
+                                  **properties)
+      else:
+        raise ValueError("""obs_client should be a ObsClient object or None""")
+    else:
+      raise NotImplementedError("""not support other data format datset""")
+
+  else:
+    raise ValueError("""the value of is_batch is invalid, it should be set True or False""")
+
+
+def make_dataset(reader):
+  """Creates a `tensorflow.data.Dataset <https://www.tensorflow.org/api_docs/python/tf/data/Dataset>`_ object from
+
+  NGrams are not yet supported by this function.
+
+  :param reader: An instance of :class:`Reader` object that would serve as a data source.
+  :return: A ``tf.data.Dataset`` instance.
+  """
+  tensorflow = TensorFlow()
+  return tensorflow.make_dataset(reader)
+
+
+def make_tensor(reader, shuffling_queue_capacity=0, min_after_dequeue=0):
+  """Bridges between python-only interface of the Reader (next(Reader)) and tensorflow world.
+
+  This function returns a named tuple of tensors from the dataset, e.g.,
+
+  If the reader was created with ``ngram=NGram(...)`` parameter, then a dictionary of named tuples is returned
+  (indexed by time):
+
+  An optional shuffling queue is created if shuffling_queue_capacity is greater than 0.
+
+  Note that if reading a unischema field that is unicode (``np.unicode_`` or ``np.str_``) tensorflow will
+  represent it as a tf.string which will be an array of bytes. If using python3 you may need to decode
+  it to convert it back to a python str type.
+
+  :param reader: An instance of Reader object used as the data source
+  :param shuffling_queue_capacity: Queue capacity is passed to the underlying :class:`tf.RandomShuffleQueue`
+      instance. If set to 0, no suffling will be done.
+  :param min_after_dequeue: If ``shuffling_queue_capacity > 0``, this value is passed to the underlying
+      :class:`tf.RandomShuffleQueue`.
+  :return: If no ngram reading is used, the function will return a named tuple with tensors that are populated
+      from the underlying dataset. If ngram reading is enabled, a dictionary of named tuples of tensors is returned.
+      The dictionary is indexed by time.
+  """
+  tensorflow = TensorFlow()
+  return tensorflow.make_tensor(reader, shuffling_queue_capacity, min_after_dequeue)
diff --git a/python/pycarbon/test/.coveragerc b/python/pycarbon/test/.coveragerc
deleted file mode 100644
index 7fc46f5..0000000
--- a/python/pycarbon/test/.coveragerc
+++ /dev/null
@@ -1,4 +0,0 @@
-[run]
-omit =
-    */__init__.py
-
diff --git a/python/pycarbon/tests/.coveragerc b/python/pycarbon/tests/.coveragerc
new file mode 100644
index 0000000..0fba310
--- /dev/null
+++ b/python/pycarbon/tests/.coveragerc
@@ -0,0 +1,6 @@
+[run]
+omit = 
+    pycarbon/tests/*
+    pycarbon/*/__init__.py
+    pycarbon/__init__.py
+
diff --git a/python/pycarbon/test/__init__.py b/python/pycarbon/tests/__init__.py
similarity index 84%
rename from python/pycarbon/test/__init__.py
rename to python/pycarbon/tests/__init__.py
index ccedf21..44e26d0 100644
--- a/python/pycarbon/test/__init__.py
+++ b/python/pycarbon/tests/__init__.py
@@ -13,13 +13,14 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+
 import os
 
 curdir = os.path.dirname(os.path.realpath(__file__))
 
 jardir = os.path.abspath(os.path.join(curdir, os.path.pardir, os.path.pardir))
 
-DEFAULT_CARBONSDK_PATH = os.path.join(jardir, 'jars/carbondata-sdk.jar')
+DEFAULT_CARBONSDK_PATH = os.path.join(jardir, '../store/sdk/target/carbondata-sdk.jar')
 
 S3_DATA_PATH = 's3a://sdk/binary'
 S3_DATA_PATH1 = 's3a://sdk/binary/sub1'
@@ -30,3 +31,8 @@
 LOCAL_DATA_PATH = os.path.join(jardir, 'examples/data/binary')
 
 IMAGE_DATA_PATH = os.path.join(jardir, 'examples/data/image')
+
+access_key = "your access key of S3"
+secret_key = "your secret key of S3"
+endpoint = "your endpoint of S3"
+proxy="your proxy"
diff --git a/python/pycarbon/tests/conftest.py b/python/pycarbon/tests/conftest.py
new file mode 100644
index 0000000..a281e74
--- /dev/null
+++ b/python/pycarbon/tests/conftest.py
@@ -0,0 +1,143 @@
+# 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.
+
+
+import logging
+import os
+import pickle
+from base64 import b64encode, b64decode
+from collections import namedtuple
+
+import pytest
+import six
+
+from pycarbon.tests.core.test_carbon_common import create_test_dataset, create_test_scalar_dataset, \
+  create_many_columns_non_unischema_dataset
+
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+SyntheticDataset = namedtuple('synthetic_dataset', ['url', 'data', 'path'])
+
+ObsDataset = namedtuple('obs_dataset', ['url', 'wrong_url', 'not_exist_url'])
+
+# Number of rows in a fake dataset
+_ROWS_COUNT = 100
+
+_CACHE_FAKE_DATASET_OPTION_SHORT = '-Y'
+_CACHE_FAKE_DATASET_OPTION = '--cache-synthetic-dataset'
+
+logger = logging.getLogger(__name__)
+
+
+def pytest_addoption(parser):
+  parser.addoption(
+    _CACHE_FAKE_DATASET_OPTION_SHORT, _CACHE_FAKE_DATASET_OPTION, action="store_true", default=False,
+    help='Use a cached version of synthetic dataset if available. This helps speedup local tests reruns as '
+         'we don\'t have to rerun spark. CAUTION: you won\'t be exercising dataset generating parts of petastorm '
+         'hence tests results maybe inaccurate'
+  )
+  parser.addoption('--pyspark-python', type=str, default=None,
+                   help='pyspark python env variable')
+  parser.addoption('--pyspark-driver-python', type=str, default=None,
+                   help='pyspark driver python env variable')
+  parser.addoption('--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                   help='carbon sdk path')
+  parser.addoption('--access_key', type=str, default=None, required=False,
+                   help='access_key of obs')
+  parser.addoption('--secret_key', type=str, default=None, required=False,
+                   help='secret_key of obs')
+  parser.addoption('--end_point', type=str, default=None, required=False,
+                   help='end_point of obs')
+  parser.addoption('--obs_path', type=str, default="s3a://modelarts-carbon/test/benchmark_external_dataset/", required=False,
+                   help='path of obs')
+
+
+def maybe_cached_dataset(config, name, generating_func):
+  """Returns cached dataset instance if caching of datasets is enabled and a valid dataset is available.
+
+  We speedup test startup time by caching previously generated synthetic dataset.
+  This is useful while developing for tests reruns, but can be dangerous since we can
+  get stale results when petastorm code participating in dataset generation is used.
+
+  :param config: request.config object.
+  :param name: name of the cached dataset. Used as a cache key.
+  :param generating_func: This function will be called (`generating_func()`) if dataset cache is disabled or
+    no valid dataset is found in cache.
+  :return:
+  """
+  if config.getoption(_CACHE_FAKE_DATASET_OPTION):
+    cache_key = '{}_{}'.format(name, 'PY2' if six.PY2 else 'PY3')
+    serialized = config.cache.get(cache_key, None)
+    dataset = pickle.loads(b64decode(serialized)) if serialized else None
+    if not dataset or not os.path.exists(dataset.path):
+      dataset = generating_func()
+      config.cache.set(cache_key, b64encode(pickle.dumps(dataset)).decode('ascii'))
+    else:
+      logger.warn('CAUTION: %s HAS BEEN USED. USING %s CACHED TEST DATASET! MAYBE STALE!',
+                  _CACHE_FAKE_DATASET_OPTION, name)
+  else:
+    dataset = generating_func()
+
+  return dataset
+
+
+@pytest.fixture(scope="session")
+def carbon_synthetic_dataset(request, tmpdir_factory):
+  def _synthetic_dataset_no_cache():
+    path = tmpdir_factory.mktemp("data").strpath
+    url = 'file://' + path
+    data = create_test_dataset(url, range(_ROWS_COUNT))
+    dataset = SyntheticDataset(url=url, path=path, data=data)
+    return dataset
+
+  return maybe_cached_dataset(request.config, 'synthetic_dataset', _synthetic_dataset_no_cache)
+
+
+@pytest.fixture(scope="session")
+def carbon_scalar_dataset(request, tmpdir_factory):
+  def _pure_carbon_dataset_no_cache():
+    path = tmpdir_factory.mktemp("data").strpath
+    url = 'file://' + path
+    data = create_test_scalar_dataset(url, 100)
+    dataset = SyntheticDataset(url=url, path=path, data=data)
+    return dataset
+
+  return maybe_cached_dataset(request.config, 'scalar', _pure_carbon_dataset_no_cache)
+
+
+@pytest.fixture(scope="session")
+def carbon_many_columns_non_unischema_dataset(request, tmpdir_factory):
+  """This dataset has 1000 columns. All of the same int32 type."""
+
+  def _dataset_no_cache():
+    path = tmpdir_factory.mktemp("data").strpath
+    url = 'file://' + path
+    data = create_many_columns_non_unischema_dataset(url, 10)
+    dataset = SyntheticDataset(url=url, path=path, data=data)
+    return dataset
+
+  return maybe_cached_dataset(request.config, 'many_column_non_unischema', _dataset_no_cache)
+
+
+@pytest.fixture(scope="session")
+def carbon_obs_dataset(request):
+  def _obs_dataset_no_cache():
+    url = pytest.config.getoption("--obs_path")
+    wrong_url = 's3a:////modelarts-carbon/test/benchmark_external_dataset/'
+    not_exist_url = 's3a://modelarts-carbon/test/not_exist_dir/'
+    dataset = ObsDataset(url=url, wrong_url=wrong_url, not_exist_url=not_exist_url)
+    return dataset
+
+  return maybe_cached_dataset(request.config, 'obs_dataset', _obs_dataset_no_cache)
diff --git a/python/pycarbon/tests/core/__init__.py b/python/pycarbon/tests/core/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/tests/core/__init__.py
diff --git a/python/pycarbon/tests/core/test_carbon.py b/python/pycarbon/tests/core/test_carbon.py
new file mode 100644
index 0000000..8efe9c3
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon.py
@@ -0,0 +1,167 @@
+# 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.
+
+
+import pytest
+
+from pycarbon.core.Constants import LOCAL_FILE_PREFIX
+from pycarbon.core.carbon import CarbonDataset
+from pycarbon.core.carbon import CarbonDatasetPiece
+
+from pycarbon.sdk.ArrowCarbonReader import ArrowCarbonReader
+from pycarbon.sdk.Configuration import Configuration
+from pycarbon.sdk.CarbonSchemaReader import CarbonSchemaReader
+
+import os
+import jnius_config
+
+from pycarbon.tests import proxy
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+
+def test_invalid_carbondataset_obs_parameters(carbon_obs_dataset):
+  with pytest.raises(ValueError):
+    CarbonDataset(carbon_obs_dataset.url)
+
+  with pytest.raises(ValueError):
+    CarbonDataset(carbon_obs_dataset.url,
+                  key=pytest.config.getoption("--access_key"),
+                  secret=pytest.config.getoption("--secret_key"),
+                  endpoint=pytest.config.getoption("--end_point"),
+                  proxy=proxy)
+
+  with pytest.raises(ValueError):
+    CarbonDataset(carbon_obs_dataset.url,
+                  key=pytest.config.getoption("--access_key"),
+                  secret=pytest.config.getoption("--secret_key"),
+                  endpoint=pytest.config.getoption("--end_point"),
+                  proxy_port="8080")
+
+
+def test_create_carbondataset_obs(carbon_obs_dataset):
+  carbondataset_1 = CarbonDataset(carbon_obs_dataset.url,
+                                  key=pytest.config.getoption("--access_key"),
+                                  secret=pytest.config.getoption("--secret_key"),
+                                  endpoint=pytest.config.getoption("--end_point"))
+
+  carbondataset_2 = CarbonDataset(carbon_obs_dataset.url,
+                                  key=pytest.config.getoption("--access_key"),
+                                  secret=pytest.config.getoption("--secret_key"),
+                                  endpoint=pytest.config.getoption("--end_point"),
+                                  proxy=proxy,
+                                  proxy_port="8080")
+
+  assert len(carbondataset_1.pieces) == len(carbondataset_2.pieces)
+  assert carbondataset_1.pieces
+
+
+def test_create_carbondataset_local(carbon_synthetic_dataset):
+  carbondataset = CarbonDataset(carbon_synthetic_dataset.url)
+  assert len(carbondataset.pieces) == 2
+
+
+def test_invalid_carbondatasetpiece_obs_parameters(carbon_obs_dataset):
+  key = pytest.config.getoption("--access_key")
+  secret = pytest.config.getoption("--secret_key")
+  endpoint = pytest.config.getoption("--end_point")
+
+  carbon_splits = ArrowCarbonReader().builder(carbon_obs_dataset.url)\
+    .withHadoopConf("fs.s3a.access.key", key)\
+    .withHadoopConf("fs.s3a.secret.key", secret)\
+    .withHadoopConf("fs.s3a.endpoint", endpoint)\
+    .getSplits(True)
+
+  configuration = Configuration()
+  configuration.set("fs.s3a.access.key", key)
+  configuration.set("fs.s3a.secret.key", secret)
+  configuration.set("fs.s3a.endpoint", endpoint)
+
+  assert carbon_splits
+
+  carbon_schema = CarbonSchemaReader().readSchema(carbon_obs_dataset.url, configuration.conf)
+
+  with pytest.raises(ValueError):
+    CarbonDatasetPiece(carbon_obs_dataset.url, carbon_schema, carbon_splits[0])
+
+  with pytest.raises(ValueError):
+    CarbonDatasetPiece(carbon_obs_dataset.url, carbon_schema, carbon_splits[0],
+                       key=key, secret=secret, endpoint=endpoint,
+                       proxy=proxy)
+
+  with pytest.raises(ValueError):
+    CarbonDatasetPiece(carbon_obs_dataset.url, carbon_schema, carbon_splits[0],
+                       key=key, secret=secret, endpoint=endpoint,
+                       proxy_port="8080")
+
+
+def test_create_carbondatasetpiece_obs(carbon_obs_dataset):
+  key = pytest.config.getoption("--access_key")
+  secret = pytest.config.getoption("--secret_key")
+  endpoint = pytest.config.getoption("--end_point")
+
+  carbon_splits = ArrowCarbonReader().builder(carbon_obs_dataset.url)\
+    .withHadoopConf("fs.s3a.access.key", key)\
+    .withHadoopConf("fs.s3a.secret.key", secret)\
+    .withHadoopConf("fs.s3a.endpoint", endpoint)\
+    .getSplits(True)
+
+  configuration = Configuration()
+  configuration.set("fs.s3a.access.key", key)
+  configuration.set("fs.s3a.secret.key", secret)
+  configuration.set("fs.s3a.endpoint", endpoint)
+
+  assert carbon_splits
+
+  carbon_schema = CarbonSchemaReader().readSchema(carbon_obs_dataset.url, configuration.conf)
+
+  carbondatasetpiece_1 = CarbonDatasetPiece(carbon_obs_dataset.url, carbon_schema, carbon_splits[0],
+                                            key=key, secret=secret, endpoint=endpoint)
+
+  carbondatasetpiece_2 = CarbonDatasetPiece(carbon_obs_dataset.url, carbon_schema, carbon_splits[0],
+                                            key=key, secret=secret, endpoint=endpoint,
+                                            proxy=proxy,
+                                            proxy_port="8080")
+
+  num_rows_1 = len(carbondatasetpiece_1.read_all(columns=None))
+  num_rows_2 = len(carbondatasetpiece_2.read_all(columns=None))
+
+  assert num_rows_1 != 0
+  assert num_rows_1 == num_rows_2
+
+
+def test_carbondataset_dataset_url_not_exist(carbon_obs_dataset):
+  # local
+  with pytest.raises(Exception):
+    CarbonDataset(LOCAL_FILE_PREFIX + "/not_exist_dir")
+
+  # obs
+  with pytest.raises(Exception):
+    CarbonDataset(carbon_obs_dataset.not_exist_url,
+                  key=pytest.config.getoption("--access_key"),
+                  secret=pytest.config.getoption("--secret_key"),
+                  endpoint=pytest.config.getoption("--end_point"))
diff --git a/python/pycarbon/tests/core/test_carbon_common.py b/python/pycarbon/tests/core/test_carbon_common.py
new file mode 100644
index 0000000..ea64bff
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon_common.py
@@ -0,0 +1,228 @@
+# 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.
+
+
+from __future__ import division
+
+import random
+from decimal import Decimal
+from functools import partial
+
+import numpy as np
+from pyspark import Row
+from pyspark.sql import SparkSession
+from pyspark.sql.types import StringType, ShortType, LongType, DecimalType, DoubleType, BooleanType, StructField, \
+  IntegerType, StructType
+
+from petastorm.codecs import CompressedImageCodec, NdarrayCodec, \
+  ScalarCodec
+from petastorm.unischema import Unischema, UnischemaField, dict_to_spark_row
+
+from pycarbon.core.carbon_dataset_metadata import materialize_dataset_carbon
+
+_DEFAULT_IMAGE_SIZE = (32, 16, 3)
+
+TestSchema = Unischema('TestSchema', [
+  UnischemaField('partition_key', np.unicode_, (), ScalarCodec(StringType()), False),
+  UnischemaField('id', np.int64, (), ScalarCodec(LongType()), False),
+  UnischemaField('id2', np.int32, (), ScalarCodec(ShortType()), False),
+  UnischemaField('id_float', np.float64, (), ScalarCodec(DoubleType()), False),
+  UnischemaField('id_odd', np.bool_, (), ScalarCodec(BooleanType()), False),
+  UnischemaField('python_primitive_uint8', np.uint8, (), ScalarCodec(ShortType()), False),
+  UnischemaField('image_png', np.uint8, _DEFAULT_IMAGE_SIZE, CompressedImageCodec('png'), False),
+  UnischemaField('matrix', np.float32, _DEFAULT_IMAGE_SIZE, NdarrayCodec(), False),
+  UnischemaField('decimal', Decimal, (), ScalarCodec(DecimalType(10, 9)), False),
+  UnischemaField('matrix_uint16', np.uint16, _DEFAULT_IMAGE_SIZE, NdarrayCodec(), False),
+  UnischemaField('matrix_string', np.string_, (None, None,), NdarrayCodec(), False),
+  UnischemaField('empty_matrix_string', np.string_, (None,), NdarrayCodec(), False),
+  UnischemaField('sensor_name', np.unicode_, (1,), NdarrayCodec(), False),
+])
+
+
+def _random_binary_string_gen(max_length):
+  """Returns a single random string up to max_length specified length that may include \x00 character anywhere in the
+  string"""
+  size = random.randint(0, max_length)
+  return ''.join(random.choice(('\x00', 'A', 'B')) for _ in range(size))
+
+
+def _random_binary_string_matrix(rows, cols, max_length):
+  """Returns a list of lists of random strings"""
+  return [[_random_binary_string_gen(max_length) for _ in range(cols)] for _ in range(rows)]
+
+
+def _randomize_row(id_num):
+  """Returns a row with random values"""
+  row_dict = {
+    TestSchema.id.name: np.int64(id_num),
+    TestSchema.id2.name: np.int32(id_num % 2),
+    TestSchema.id_float.name: np.float64(id_num),
+    TestSchema.id_odd.name: np.bool_(id_num % 2),
+    TestSchema.partition_key.name: np.unicode_('p_{}'.format(int(id_num / 10))),
+    TestSchema.python_primitive_uint8.name: np.random.randint(0, 255, dtype=np.uint8),
+    TestSchema.image_png.name: np.random.randint(0, 255, _DEFAULT_IMAGE_SIZE).astype(np.uint8),
+    TestSchema.matrix.name: np.random.random(size=_DEFAULT_IMAGE_SIZE).astype(np.float32),
+    TestSchema.decimal.name: Decimal(np.random.randint(0, 255) / Decimal(100)),
+    TestSchema.matrix_uint16.name: np.random.randint(0, 255, _DEFAULT_IMAGE_SIZE).astype(np.uint16),
+    TestSchema.matrix_string.name: np.asarray(_random_binary_string_matrix(2, 3, 10)).astype(np.bytes_),
+    TestSchema.empty_matrix_string.name: np.asarray([], dtype=np.string_),
+    TestSchema.sensor_name.name: np.asarray(['test_sensor'], dtype=np.unicode_),
+  }
+  return row_dict
+
+
+def create_test_dataset(tmp_url, rows, num_files=2, spark=None):
+  """
+  Creates a test dataset under tmp_dir, with rows and num_files that has TestSchema.
+  :param tmp_url: The URL of the temp directory to store the test dataset in.
+  :param rows: The number of rows for the dataset.
+  :param num_files: The number of files to partition the data between.
+  :param spark: An optional spark session to use
+  :return: A list of the dataset dictionary.
+  """
+
+  shutdown = False
+  if not spark:
+    spark_session = SparkSession \
+      .builder \
+      .appName('pycarbon_end_to_end_test') \
+      .master('local[*]')
+
+    spark = spark_session.getOrCreate()
+    shutdown = True
+  spark_context = spark.sparkContext
+
+  with materialize_dataset_carbon(spark, tmp_url, TestSchema):
+    id_rdd = spark_context.parallelize(rows, numSlices=40)
+
+    # Make up some random data and store it for referencing in the tests
+    random_dicts_rdd = id_rdd.map(_randomize_row).cache()
+    dataset_dicts = random_dicts_rdd.collect()
+
+    def _partition_key_to_str(row):
+      row['partition_key'] = str(row['partition_key'])
+      return row
+
+    random_dicts_rdd = random_dicts_rdd.map(_partition_key_to_str)
+
+    random_rows_rdd = random_dicts_rdd.map(partial(dict_to_spark_row, TestSchema))
+
+    # Create a spark dataframe with the random rows
+    dataframe = spark. \
+      createDataFrame(random_rows_rdd, TestSchema.as_spark_schema()).sort('id')
+
+    # Save a parquet
+    dataframe. \
+      coalesce(num_files). \
+      write.option('compression', 'none'). \
+      mode('overwrite'). \
+      save(path=tmp_url, format='carbon')
+
+  if shutdown:
+    spark.stop()
+
+  return dataset_dicts
+
+
+def create_test_scalar_dataset(tmp_url, num_rows, num_files=4, spark=None):
+  shutdown = False
+  if not spark:
+    spark_session = SparkSession \
+      .builder \
+      .appName('pycarbon_end_to_end_test') \
+      .master('local[*]')
+
+    spark = spark_session.getOrCreate()
+    shutdown = True
+
+  expected_data = [{'id': np.int32(i),
+                    'string': np.unicode_('hello_{}'.format(i)),
+                    'string2': np.unicode_('world_{}'.format(i)),
+                    'float64': np.float64(i) * .66} for i in range(num_rows)]
+
+  expected_data_as_scalars = [{k: np.asscalar(v) if isinstance(v, np.generic) else v for k, v in row.items()} for row
+                              in expected_data]
+
+  rows = [Row(**row) for row in expected_data_as_scalars]
+
+  # WARNING: surprisingly, schema fields and row fields are matched only by order and not name.
+  # We must maintain alphabetical order of the struct fields for the code to work!!!
+  schema = StructType([
+    StructField('float64', DoubleType(), False),
+    StructField('id', IntegerType(), False),
+    StructField('string', StringType(), False),
+    StructField('string2', StringType(), False),
+  ])
+
+  dataframe = spark.createDataFrame(rows, schema)
+  dataframe. \
+    coalesce(num_files). \
+    write.option('compression', 'none'). \
+    mode('overwrite'). \
+    save(path=tmp_url, format='carbon')
+
+  if shutdown:
+    spark.stop()
+
+  return expected_data
+
+
+def create_many_columns_non_unischema_dataset(output_url, num_rows, num_columns=1000, num_files=4, spark=None):
+  """Creates a dataset with the following properties (used in tests)
+
+  1. Has 1000 columns
+  2. Each column is an int32 integer
+  3. Parquet store consists of 4 files (controlled by ``num_files`` argument)
+
+  :param output_url: The dataset is written to this url (e.g. ``file:///tmp/some_directory``)
+  :param num_rows: Number of rows in the generated dataset
+  :param num_columns: Number of columns (1000 is the default)
+  :param num_files: Number of parquet files that will be created in the store
+  :param spark: An instance of SparkSession object. A new instance will be created if non specified
+  :return:
+  """
+  shutdown = False
+  if not spark:
+    spark_session = SparkSession \
+      .builder \
+      .appName('pycarbon_end_to_end_test') \
+      .master('local[*]')
+
+    spark = spark_session.getOrCreate()
+    shutdown = True
+
+  column_names = ['col_{}'.format(col_id) for col_id in range(num_columns)]
+
+  def generate_row(i):
+    return {'col_{}'.format(col_id): i * 10000 for col_id, col_name in enumerate(column_names)}
+
+  expected_data = [generate_row(row_number) for row_number in range(num_rows)]
+
+  rows = [Row(**row) for row in expected_data]
+
+  # WARNING: surprisingly, schema fields and row fields are matched only by order and not name.
+  schema = StructType([StructField(column_name, IntegerType(), False) for column_name in column_names])
+
+  dataframe = spark.createDataFrame(rows, schema)
+  dataframe. \
+    coalesce(num_files). \
+    write.option('compression', 'none'). \
+    mode('overwrite'). \
+    save(path=output_url, format='carbon')
+
+  if shutdown:
+    spark.stop()
+
+  return expected_data
diff --git a/python/pycarbon/tests/core/test_carbon_end_to_end.py b/python/pycarbon/tests/core/test_carbon_end_to_end.py
new file mode 100644
index 0000000..617f406
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon_end_to_end.py
@@ -0,0 +1,399 @@
+# 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.
+
+
+from concurrent.futures import ThreadPoolExecutor
+from shutil import rmtree, copytree
+
+import numpy as np
+import pytest
+
+from petastorm.codecs import ScalarCodec
+
+from pycarbon import make_carbon_reader, make_batch_carbon_reader
+from pycarbon.reader import make_reader
+from pycarbon.tests.core.test_carbon_common import TestSchema
+
+from pycarbon.tests.conftest import _ROWS_COUNT
+
+import os
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+# pylint: disable=unnecessary-lambda
+ALL_READER_FLAVOR_FACTORIES = [
+  lambda url, **kwargs: make_carbon_reader(url, reader_pool_type='thread', **kwargs),
+]
+
+SCALAR_FIELDS = [f for f in TestSchema.fields.values() if isinstance(f.codec, ScalarCodec)]
+
+SCALAR_ONLY_READER_FACTORIES = [
+  lambda url, **kwargs: make_batch_carbon_reader(url, reader_pool_type='thread', **kwargs),
+]
+
+
+def _check_simple_reader(reader, expected_data, expected_rows_count=None, check_types=True, limit_checked_rows=None):
+  # Read a bunch of entries from the dataset and compare the data to reference
+  def _type(v):
+    if isinstance(v, np.ndarray):
+      if v.dtype.str.startswith('|S'):
+        return '|S'
+      else:
+        return v.dtype
+    else:
+      return type(v)
+
+  expected_rows_count = expected_rows_count or len(expected_data)
+  count = 0
+
+  for i, row in enumerate(reader):
+    if limit_checked_rows and i >= limit_checked_rows:
+      break
+
+    actual = row._asdict()
+    expected = next(d for d in expected_data if d['id'] == actual['id'])
+    np.testing.assert_equal(actual, expected)
+    actual_types = {k: _type(v) for k, v in actual.items()}
+    expected_types = {k: _type(v) for k, v in expected.items()}
+    assert not check_types or actual_types == expected_types
+    count += 1
+
+  if limit_checked_rows:
+    assert count == min(expected_rows_count, limit_checked_rows)
+  else:
+    assert count == expected_rows_count
+
+
+def _readout_all_ids(reader, limit=None):
+  ids = []
+  for i, row in enumerate(reader):
+    if limit is not None and i >= limit:
+      break
+    ids.append(row.id)
+
+  # Flatten ids if reader returns batches (make_batch_reader)
+  if isinstance(ids[0], np.ndarray):
+    ids = [i for arr in ids for i in arr]
+
+  return ids
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+def test_simple_read(carbon_synthetic_dataset, reader_factory):
+  """Just a bunch of read and compares of all values to the expected values using the different reader pools"""
+  with reader_factory(carbon_synthetic_dataset.url) as reader:
+    _check_simple_reader(reader, carbon_synthetic_dataset.data)
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES + SCALAR_ONLY_READER_FACTORIES)
+@pytest.mark.forked
+def test_simple_read_with_disk_cache(carbon_synthetic_dataset, reader_factory, tmpdir):
+  """Try using the Reader with LocalDiskCache using different flavors of pools"""
+  CACHE_SIZE = 10 * 2 ** 30  # 20GB
+  ROW_SIZE_BYTES = 100  # not really important for this test
+  with reader_factory(carbon_synthetic_dataset.url, num_epochs=2,
+                      cache_type='local-disk', cache_location=tmpdir.strpath,
+                      cache_size_limit=CACHE_SIZE, cache_row_size_estimate=ROW_SIZE_BYTES) as reader:
+    ids = _readout_all_ids(reader)
+    assert 200 == len(ids)  # We read 2 epochs
+    assert set(ids) == set(range(100))
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES + SCALAR_ONLY_READER_FACTORIES)
+def test_simple_read_with_added_slashes(carbon_synthetic_dataset, reader_factory):
+  """Tests that using relative paths for the dataset metadata works as expected"""
+  with reader_factory(carbon_synthetic_dataset.url + '///') as reader:
+    next(reader)
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES + SCALAR_ONLY_READER_FACTORIES)
+def test_simple_read_moved_dataset(carbon_synthetic_dataset, tmpdir, reader_factory):
+  """Tests that a dataset may be opened after being moved to a new location"""
+  a_moved_path = tmpdir.join('moved').strpath
+  copytree(carbon_synthetic_dataset.path, a_moved_path)
+
+  with reader_factory('file://{}'.format(a_moved_path)) as reader:
+    next(reader)
+
+  rmtree(a_moved_path)
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+def test_reading_subset_of_columns(carbon_synthetic_dataset, reader_factory):
+  """Just a bunch of read and compares of all values to the expected values"""
+  with reader_factory(carbon_synthetic_dataset.url, schema_fields=[TestSchema.id2, TestSchema.id]) as reader:
+    # Read a bunch of entries from the dataset and compare the data to reference
+    for row in reader:
+      actual = dict(row._asdict())
+      expected = next(d for d in carbon_synthetic_dataset.data if d['id'] == actual['id'])
+      np.testing.assert_equal(expected['id2'], actual['id2'])
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+def test_reading_subset_of_columns_using_regex(carbon_synthetic_dataset, reader_factory):
+  """Just a bunch of read and compares of all values to the expected values"""
+  with reader_factory(carbon_synthetic_dataset.url, schema_fields=['id$', 'id_.*$', 'partition_key$']) as reader:
+    # Read a bunch of entries from the dataset and compare the data to reference
+    for row in reader:
+      actual = dict(row._asdict())
+      assert set(actual.keys()) == {'id_float', 'id_odd', 'id', 'partition_key'}
+      expected = next(d for d in carbon_synthetic_dataset.data if d['id'] == actual['id'])
+      np.testing.assert_equal(expected['id_float'], actual['id_float'])
+
+
+# TODO: fix make_batch_carbon_reader error
+@pytest.mark.parametrize('reader_factory', [
+  lambda url, **kwargs: make_carbon_reader(url, reader_pool_type='thread', **kwargs),
+  lambda url, **kwargs: make_batch_carbon_reader(url, reader_pool_type='thread', **kwargs)])
+def test_shuffle(carbon_synthetic_dataset, reader_factory):
+  rows_count = len(carbon_synthetic_dataset.data)
+
+  # Read ids twice without shuffle: assert we have the same array and all expected ids are in the array
+  with reader_factory(carbon_synthetic_dataset.url, shuffle_blocklets=False, shuffle_row_drop_partitions=5,
+                      workers_count=1) as reader_1:
+    first_readout = _readout_all_ids(reader_1)
+  with reader_factory(carbon_synthetic_dataset.url, shuffle_blocklets=False, shuffle_row_drop_partitions=5,
+                      workers_count=1) as reader_2:
+    second_readout = _readout_all_ids(reader_2)
+
+  np.testing.assert_array_equal(range(rows_count), sorted(first_readout))
+  np.testing.assert_array_equal(first_readout, second_readout)
+
+  # Now read with shuffling
+  with reader_factory(carbon_synthetic_dataset.url, shuffle_blocklets=True, shuffle_row_drop_partitions=5,
+                      workers_count=1) as shuffled_reader:
+    shuffled_readout = _readout_all_ids(shuffled_reader)
+  assert np.any(np.not_equal(first_readout, shuffled_readout))
+
+
+# TODO: fix make_batch_carbon_reader error
+@pytest.mark.parametrize('reader_factory', [
+  lambda url, **kwargs: make_carbon_reader(url, reader_pool_type='thread', **kwargs),
+  lambda url, **kwargs: make_batch_carbon_reader(url, reader_pool_type='thread', **kwargs)])
+def test_shuffle_drop_ratio(carbon_synthetic_dataset, reader_factory):
+  # Read ids twice without shuffle: assert we have the same array and all expected ids are in the array
+  with reader_factory(carbon_synthetic_dataset.url, shuffle_blocklets=False, shuffle_row_drop_partitions=1) as reader:
+    first_readout = _readout_all_ids(reader)
+  np.testing.assert_array_equal([r['id'] for r in carbon_synthetic_dataset.data], sorted(first_readout))
+
+  # Test that the ids are increasingly not consecutive numbers as we increase the shuffle dropout
+  prev_jumps_not_1 = 0
+  for shuffle_dropout in [2, 5, 8]:
+    with reader_factory(carbon_synthetic_dataset.url, shuffle_blocklets=True,
+                        shuffle_row_drop_partitions=shuffle_dropout) as reader:
+      readout = _readout_all_ids(reader)
+
+    assert len(first_readout) == len(readout)
+    jumps_not_1 = np.sum(np.diff(readout) != 1)
+    assert jumps_not_1 > prev_jumps_not_1
+    prev_jumps_not_1 = jumps_not_1
+
+
+# TODO: fix test_partition_multi_node
+# @pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+# def test_partition_multi_node(carbon_synthetic_dataset, reader_factory):
+#   """Tests that the reader only returns half of the expected data consistently"""
+#   with reader_factory(carbon_synthetic_dataset.url, cur_shard=0, shard_count=5) as reader:
+#     with reader_factory(carbon_synthetic_dataset.url, cur_shard=0, shard_count=5) as reader_2:
+#       results_1 = set(_readout_all_ids(reader))
+#       results_2 = set(_readout_all_ids(reader_2))
+#
+#       assert results_1, 'Non empty shard expected'
+#
+#       np.testing.assert_equal(results_1, results_2)
+#
+#       assert len(results_1) < len(carbon_synthetic_dataset.data)
+#
+#       # Test that separate partitions also have no overlap by checking ids)
+#       for partition in range(1, 5):
+#         with reader_factory(carbon_synthetic_dataset.url, cur_shard=partition,
+#                             shard_count=5) as reader_other:
+#           ids_in_other_partition = set(_readout_all_ids(reader_other))
+#
+#           assert not ids_in_other_partition.intersection(results_1)
+
+
+# TODO: fix test_partition_value_error
+# @pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+# def test_partition_value_error(carbon_synthetic_dataset, reader_factory):
+#   """Tests that the reader raises value errors when appropriate"""
+#
+#   # shard_count has to be greater than 0
+#   with pytest.raises(ValueError):
+#     reader_factory(carbon_synthetic_dataset.url, shard_count=0)
+#
+#   # missing cur_shard value
+#   with pytest.raises(ValueError):
+#     reader_factory(carbon_synthetic_dataset.url, shard_count=5)
+#
+#   # cur_shard is a string
+#   with pytest.raises(ValueError):
+#     reader_factory(carbon_synthetic_dataset.url, cur_shard='0',
+#                    shard_count=5)
+#
+#   # shard_count is a string
+#   with pytest.raises(ValueError):
+#     reader_factory(carbon_synthetic_dataset.url, cur_shard=0,
+#                    shard_count='5')
+
+
+@pytest.mark.parametrize('reader_factory', [
+  lambda url, **kwargs: make_carbon_reader(url, reader_pool_type='thread', **kwargs),
+  lambda url, **kwargs: make_batch_carbon_reader(url, reader_pool_type='thread', **kwargs),
+])
+def test_stable_pieces_order(carbon_synthetic_dataset, reader_factory):
+  """Tests that the reader raises value errors when appropriate"""
+
+  RERUN_THE_TEST_COUNT = 4
+  baseline_run = None
+  for _ in range(RERUN_THE_TEST_COUNT):
+    # TODO(yevgeni): factor out. Reading all ids appears multiple times in this test.
+    with reader_factory(carbon_synthetic_dataset.url, shuffle_blocklets=False, workers_count=1) as reader:
+      this_run = _readout_all_ids(reader)
+
+    if baseline_run:
+      assert this_run == baseline_run
+
+    baseline_run = this_run
+
+
+@pytest.mark.parametrize('reader_factory',
+                         [lambda url, **kwargs: make_reader(url, reader_pool_type='thread', is_batch=False, **kwargs)])
+def test_multiple_epochs(carbon_synthetic_dataset, reader_factory):
+  """Tests that multiple epochs works as expected"""
+  num_epochs = 5
+  with reader_factory(carbon_synthetic_dataset.url, num_epochs=num_epochs) as reader:
+    # Read all expected entries from the dataset and compare the data to reference
+    single_epoch_id_set = [d['id'] for d in carbon_synthetic_dataset.data]
+    actual_ids_in_all_epochs = _readout_all_ids(reader)
+    np.testing.assert_equal(sorted(actual_ids_in_all_epochs), sorted(num_epochs * single_epoch_id_set))
+
+    # Reset reader should reset ventilator. Should produce another `num_epochs` results
+    reader.reset()
+    actual_ids_in_all_epochs = _readout_all_ids(reader)
+    np.testing.assert_equal(sorted(actual_ids_in_all_epochs), sorted(num_epochs * single_epoch_id_set))
+
+
+@pytest.mark.parametrize('reader_factory',
+                         [lambda url, **kwargs: make_reader(url, reader_pool_type='thread', is_batch=False, **kwargs)])
+def test_fail_if_resetting_in_the_middle_of_epoch(carbon_synthetic_dataset, reader_factory):
+  """Tests that multiple epochs works as expected"""
+  num_epochs = 5
+  with reader_factory(carbon_synthetic_dataset.url, num_epochs=num_epochs) as reader:
+    # Read all expected entries from the dataset and compare the data to reference
+    actual_ids = _readout_all_ids(reader, limit=20)
+    assert len(actual_ids) == 20
+
+    with pytest.raises(NotImplementedError):
+      reader.reset()
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES + SCALAR_ONLY_READER_FACTORIES)
+def test_unlimited_epochs(carbon_synthetic_dataset, reader_factory):
+  """Tests that unlimited epochs works as expected"""
+  with reader_factory(carbon_synthetic_dataset.url, num_epochs=None) as reader:
+    read_limit = len(carbon_synthetic_dataset.data) * 3 + 2
+    actual_ids = _readout_all_ids(reader, read_limit)
+    expected_ids = [d['id'] for d in carbon_synthetic_dataset.data]
+    assert len(actual_ids) > len(expected_ids)
+    assert set(actual_ids) == set(expected_ids)
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES + SCALAR_ONLY_READER_FACTORIES)
+def test_num_epochs_value_error(carbon_synthetic_dataset, reader_factory):
+  """Tests that the reader raises value errors when appropriate"""
+
+  # Testing only Reader v1, as the v2 uses an epoch generator. The error would raise only when the generator is
+  # evaluated. Parameter validation for Reader v2 is covered by test_epoch_generator.py
+
+  with pytest.raises(ValueError):
+    reader_factory(carbon_synthetic_dataset.url, num_epochs=-10)
+
+  with pytest.raises(ValueError):
+    reader_factory(carbon_synthetic_dataset.url, num_epochs='abc')
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES + SCALAR_ONLY_READER_FACTORIES)
+def test_dataset_path_is_a_unicode(carbon_synthetic_dataset, reader_factory):
+  """Just a bunch of read and compares of all values to the expected values using the different reader pools"""
+  # Making sure unicode_in_p23 is a unicode both in python 2 and 3
+  unicode_in_p23 = carbon_synthetic_dataset.url.encode().decode('utf-8')
+  with reader_factory(unicode_in_p23) as reader:
+    next(reader)
+
+
+# TODO: fix test_make_carbon_reader_fails_loading_non_unischema_dataset
+def test_make_carbon_reader_fails_loading_non_unischema_dataset(carbon_many_columns_non_unischema_dataset):
+  with pytest.raises(RuntimeError, match='use make_batch_carbon_reader'):
+    make_carbon_reader(carbon_many_columns_non_unischema_dataset.url)
+
+
+def test_multithreaded_reads(carbon_synthetic_dataset):
+  with make_reader(carbon_synthetic_dataset.url, is_batch=False, workers_count=5, num_epochs=1) as reader:
+    with ThreadPoolExecutor(max_workers=10) as executor:
+      def read_one_row():
+        return next(reader)
+
+      futures = [executor.submit(read_one_row) for _ in range(100)]
+      results = [f.result() for f in futures]
+      assert len(results) == len(carbon_synthetic_dataset.data)
+      assert set(r.id for r in results) == set(d['id'] for d in carbon_synthetic_dataset.data)
+
+
+@pytest.mark.forked
+def test_make_reader(carbon_synthetic_dataset):
+  with make_reader(carbon_synthetic_dataset.url, is_batch=False, num_epochs=1) as reader:
+    i = 0
+    for sample in reader:
+      print(sample.id)
+      i += 1
+    assert i == _ROWS_COUNT
+
+
+@pytest.mark.forked
+def test_batch_carbon_reader(carbon_synthetic_dataset):
+  with make_reader(carbon_synthetic_dataset.url, num_epochs=1) as reader:
+    i = 0
+    for sample in reader:
+      for ele in sample.id:
+        print(ele)
+        i += 1
+    assert i == _ROWS_COUNT
+
+
+@pytest.mark.forked
+def test_make_reader_of_obs(carbon_obs_dataset):
+  with make_reader(carbon_obs_dataset.url,
+                   key=pytest.config.getoption("--access_key"),
+                   secret=pytest.config.getoption("--secret_key"),
+                   endpoint=pytest.config.getoption("--end_point")) as reader:
+    i = 0
+    for sample in reader:
+      i += len(sample.id)
+
+    assert i == _ROWS_COUNT
diff --git a/python/pycarbon/tests/core/test_carbon_fs_utils.py b/python/pycarbon/tests/core/test_carbon_fs_utils.py
new file mode 100644
index 0000000..8c1edda
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon_fs_utils.py
@@ -0,0 +1,164 @@
+# 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.
+
+
+import mock
+import unittest
+
+from pyarrow.filesystem import LocalFileSystem, S3FSWrapper
+from pyarrow.lib import ArrowIOError
+from six.moves.urllib.parse import urlparse
+
+from petastorm.hdfs.tests.test_hdfs_namenode import HC, MockHadoopConfiguration, \
+  MockHdfs, MockHdfsConnector
+
+from pycarbon.core.carbon_fs_utils import CarbonFilesystemResolver
+from pycarbon.tests import access_key, secret_key, endpoint
+
+ABS_PATH = '/abs/path'
+
+
+class FilesystemResolverTest(unittest.TestCase):
+  """
+  Checks the full filesystem resolution functionality, exercising each URL interpretation case.
+  """
+
+  @classmethod
+  def setUpClass(cls):
+    cls.mock = MockHdfsConnector()
+
+  def setUp(self):
+    """Initializes a mock hadoop config and populate with basic properties."""
+    # Reset counters in mock connector
+    self.mock.reset()
+    self._hadoop_configuration = MockHadoopConfiguration()
+    self._hadoop_configuration.set('fs.defaultFS', HC.FS_WARP_TURTLE)
+    self._hadoop_configuration.set('dfs.ha.namenodes.{}'.format(HC.WARP_TURTLE), 'nn2,nn1')
+    self._hadoop_configuration.set('dfs.namenode.rpc-address.{}.nn1'.format(HC.WARP_TURTLE), HC.WARP_TURTLE_NN1)
+    self._hadoop_configuration.set('dfs.namenode.rpc-address.{}.nn2'.format(HC.WARP_TURTLE), HC.WARP_TURTLE_NN2)
+
+  def test_error_url_cases(self):
+    """Various error cases that result in exception raised."""
+    # Case 1: Schemeless path asserts
+    with self.assertRaises(ValueError):
+      CarbonFilesystemResolver(ABS_PATH, {})
+
+    # Case 4b: HDFS default path case with NO defaultFS
+    with self.assertRaises(RuntimeError):
+      CarbonFilesystemResolver('hdfs:///some/path', {})
+
+    # Case 4b: Using `default` as host, while apparently a pyarrow convention, is NOT valid
+    with self.assertRaises(ArrowIOError):
+      CarbonFilesystemResolver('hdfs://default', {})
+
+    # Case 5: other schemes result in ValueError; urlparse to cover an else branch!
+    with self.assertRaises(ValueError):
+      CarbonFilesystemResolver(urlparse('http://foo/bar'), {})
+    with self.assertRaises(ValueError):
+      CarbonFilesystemResolver(urlparse('ftp://foo/bar'), {})
+    with self.assertRaises(ValueError):
+      CarbonFilesystemResolver(urlparse('ssh://foo/bar'), {})
+
+    # s3 paths must have the bucket as the netloc
+    with self.assertRaises(ValueError):
+      CarbonFilesystemResolver(urlparse('s3:///foo/bar'), {})
+
+  def test_file_url(self):
+    """ Case 2: File path, agnostic to content of hadoop configuration."""
+    suj = CarbonFilesystemResolver('file://{}'.format(ABS_PATH), hadoop_configuration=self._hadoop_configuration,
+                                   connector=self.mock)
+    self.assertTrue(isinstance(suj.filesystem(), LocalFileSystem))
+    self.assertEqual('', suj.parsed_dataset_url().netloc)
+    self.assertEqual(ABS_PATH, suj.get_dataset_path())
+
+  def test_hdfs_url_with_nameservice(self):
+    """ Case 3a: HDFS nameservice."""
+    suj = CarbonFilesystemResolver(dataset_url=HC.WARP_TURTLE_PATH, hadoop_configuration=self._hadoop_configuration,
+                                   connector=self.mock)
+    self.assertEqual(MockHdfs, type(suj.filesystem()._hdfs))
+    self.assertEqual(HC.WARP_TURTLE, suj.parsed_dataset_url().netloc)
+    self.assertEqual(1, self.mock.connect_attempted(HC.WARP_TURTLE_NN2))
+    self.assertEqual(0, self.mock.connect_attempted(HC.WARP_TURTLE_NN1))
+    self.assertEqual(0, self.mock.connect_attempted(HC.DEFAULT_NN))
+
+  def test_hdfs_url_no_nameservice(self):
+    """ Case 3b: HDFS with no nameservice should connect to default namenode."""
+    suj = CarbonFilesystemResolver(dataset_url='hdfs:///some/path', hadoop_configuration=self._hadoop_configuration,
+                                   connector=self.mock)
+    self.assertEqual(MockHdfs, type(suj.filesystem()._hdfs))
+    self.assertEqual(HC.WARP_TURTLE, suj.parsed_dataset_url().netloc)
+    # ensure path is preserved in parsed URL
+    self.assertEqual('/some/path', suj.get_dataset_path())
+    self.assertEqual(1, self.mock.connect_attempted(HC.WARP_TURTLE_NN2))
+    self.assertEqual(0, self.mock.connect_attempted(HC.WARP_TURTLE_NN1))
+    self.assertEqual(0, self.mock.connect_attempted(HC.DEFAULT_NN))
+
+  def test_hdfs_url_direct_namenode(self):
+    """ Case 4: direct namenode."""
+    suj = CarbonFilesystemResolver('hdfs://{}/path'.format(HC.WARP_TURTLE_NN1),
+                                   hadoop_configuration=self._hadoop_configuration,
+                                   connector=self.mock)
+    self.assertEqual(MockHdfs, type(suj.filesystem()))
+    self.assertEqual(HC.WARP_TURTLE_NN1, suj.parsed_dataset_url().netloc)
+    self.assertEqual(0, self.mock.connect_attempted(HC.WARP_TURTLE_NN2))
+    self.assertEqual(1, self.mock.connect_attempted(HC.WARP_TURTLE_NN1))
+    self.assertEqual(0, self.mock.connect_attempted(HC.DEFAULT_NN))
+
+  def test_hdfs_url_direct_namenode_retries(self):
+    """ Case 4: direct namenode fails first two times thru, but 2nd retry succeeds."""
+    self.mock.set_fail_n_next_connect(2)
+    with self.assertRaises(ArrowIOError):
+      suj = CarbonFilesystemResolver('hdfs://{}/path'.format(HC.WARP_TURTLE_NN2),
+                                     hadoop_configuration=self._hadoop_configuration,
+                                     connector=self.mock)
+    self.assertEqual(1, self.mock.connect_attempted(HC.WARP_TURTLE_NN2))
+    self.assertEqual(0, self.mock.connect_attempted(HC.WARP_TURTLE_NN1))
+    self.assertEqual(0, self.mock.connect_attempted(HC.DEFAULT_NN))
+    with self.assertRaises(ArrowIOError):
+      suj = CarbonFilesystemResolver('hdfs://{}/path'.format(HC.WARP_TURTLE_NN2),
+                                     hadoop_configuration=self._hadoop_configuration,
+                                     connector=self.mock)
+    self.assertEqual(2, self.mock.connect_attempted(HC.WARP_TURTLE_NN2))
+    self.assertEqual(0, self.mock.connect_attempted(HC.WARP_TURTLE_NN1))
+    self.assertEqual(0, self.mock.connect_attempted(HC.DEFAULT_NN))
+    # this one should connect "successfully"
+    suj = CarbonFilesystemResolver('hdfs://{}/path'.format(HC.WARP_TURTLE_NN2),
+                                   hadoop_configuration=self._hadoop_configuration,
+                                   connector=self.mock)
+    self.assertEqual(MockHdfs, type(suj.filesystem()))
+    self.assertEqual(HC.WARP_TURTLE_NN2, suj.parsed_dataset_url().netloc)
+    self.assertEqual(3, self.mock.connect_attempted(HC.WARP_TURTLE_NN2))
+    self.assertEqual(0, self.mock.connect_attempted(HC.WARP_TURTLE_NN1))
+    self.assertEqual(0, self.mock.connect_attempted(HC.DEFAULT_NN))
+
+  def test_s3_without_s3fs(self):
+    with mock.patch.dict('sys.modules', s3fs=None):
+      # `import s3fs` will fail in this context
+      with self.assertRaises(ValueError):
+        CarbonFilesystemResolver(urlparse('s3a://foo/bar'), {})
+
+  def test_s3_url(self):
+    suj = CarbonFilesystemResolver('s3a://bucket{}'.format(ABS_PATH),
+                                   key=access_key,
+                                   secret=secret_key,
+                                   endpoint=endpoint,
+                                   hadoop_configuration=self._hadoop_configuration, connector=self.mock)
+    self.assertTrue(isinstance(suj.filesystem(), S3FSWrapper))
+    self.assertEqual('bucket', suj.parsed_dataset_url().netloc)
+    self.assertEqual('bucket' + ABS_PATH, suj.get_dataset_path())
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/python/pycarbon/tests/core/test_carbon_memory_cache.py b/python/pycarbon/tests/core/test_carbon_memory_cache.py
new file mode 100644
index 0000000..70f3caf
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon_memory_cache.py
@@ -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.
+
+
+import pytest
+
+from pycarbon import make_carbon_reader, make_batch_carbon_reader
+from pycarbon.core.carbon_local_memory_cache import LocalMemoryCache
+
+import os
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+
+def test_invalid_cache_parameter(carbon_synthetic_dataset, carbon_scalar_dataset):
+  with make_carbon_reader(carbon_synthetic_dataset.url,
+                          cache_type='memory-cache',
+                          shuffle_row_drop_partitions=5) as reader:
+    with pytest.raises(RuntimeError):
+      next(reader)
+
+  with make_batch_carbon_reader(carbon_scalar_dataset.url,
+                                cache_type='memory-cache',
+                                shuffle_row_drop_partitions=5) as reader:
+    with pytest.raises(RuntimeError):
+      next(reader)
+
+
+def test_simple_scalar_cache():
+  """Testing trivial NullCache: should trigger value generating function on each run"""
+  cache = LocalMemoryCache(100)
+  assert 42 == cache.get('some_key', lambda: 42)
+  assert 42 == cache.get('some_key', lambda: 43)
+
+
+def test_cache_cleanup():
+  cache = LocalMemoryCache(100)
+  assert 42 == cache.get('some_key', lambda: 42)
+  cache.cleanup()
+
+  assert 0 == cache.size()
diff --git a/python/pycarbon/tests/core/test_carbon_predicates.py b/python/pycarbon/tests/core/test_carbon_predicates.py
new file mode 100644
index 0000000..a6c09ba
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon_predicates.py
@@ -0,0 +1,130 @@
+# 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.
+
+
+import pytest
+
+import numpy as np
+from pyspark.sql import SparkSession
+from pyspark.sql.types import IntegerType
+
+from petastorm.codecs import ScalarCodec
+from petastorm.predicates import in_lambda
+from petastorm.unischema import dict_to_spark_row, Unischema, UnischemaField
+
+from pycarbon.core.carbon_reader import make_carbon_reader, make_batch_carbon_reader
+from pycarbon.core.carbon_dataset_metadata import materialize_dataset_carbon
+from pycarbon.tests.core.test_carbon_common import TestSchema
+
+import os
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+
+def test_invalid_carbon_reader_predicate_parameters(carbon_synthetic_dataset):
+  with make_carbon_reader(carbon_synthetic_dataset.url,
+                          cache_type="memory-cache",
+                          predicate=in_lambda(['id2'], lambda id2: True)) as reader:
+    with pytest.raises(RuntimeError):
+      next(reader)
+
+  with make_carbon_reader(carbon_synthetic_dataset.url,
+                          predicate=in_lambda([], lambda x: False)) as reader:
+    with pytest.raises(ValueError):
+      next(reader)
+
+  with make_carbon_reader(carbon_synthetic_dataset.url,
+                          predicate=in_lambda(['not_exist_col'], lambda x: False)) as reader:
+    with pytest.raises(ValueError):
+      next(reader)
+
+
+def test_invalid_batch_carbon_reader_predicate_parameters(carbon_scalar_dataset):
+  with make_batch_carbon_reader(carbon_scalar_dataset.url,
+                                cache_type="memory-cache",
+                                predicate=in_lambda(['id2'], lambda id2: True)) as reader:
+    with pytest.raises(RuntimeError):
+      next(reader)
+
+  with make_batch_carbon_reader(carbon_scalar_dataset.url,
+                                predicate=in_lambda([], lambda x: False)) as reader:
+    with pytest.raises(ValueError):
+      next(reader)
+
+  with make_batch_carbon_reader(carbon_scalar_dataset.url,
+                                predicate=in_lambda(['not_exist_col'], lambda x: False)) as reader:
+    with pytest.raises(ValueError):
+      next(reader)
+
+
+def test_predicate_on_single_column(carbon_synthetic_dataset):
+  reader = make_carbon_reader(carbon_synthetic_dataset.url,
+                              schema_fields=[TestSchema.id2],
+                              predicate=in_lambda(['id2'], lambda id2: True),
+                              reader_pool_type='thread')
+  counter = 0
+  for row in reader:
+    counter += 1
+    actual = dict(row._asdict())
+    assert actual['id2'] < 2
+  assert counter == len(carbon_synthetic_dataset.data)
+
+
+def test_predicate_on_dataset(tmpdir):
+  TestSchema = Unischema('TestSchema', [
+    UnischemaField('id', np.int32, (), ScalarCodec(IntegerType()), False),
+    UnischemaField('test_field', np.int32, (), ScalarCodec(IntegerType()), False),
+  ])
+
+  def test_row_generator(x):
+    """Returns a single entry in the generated dataset."""
+    return {'id': x,
+            'test_field': x * x}
+
+  blocklet_size_mb = 256
+  dataset_url = "file://{0}/partitioned_test_dataset".format(tmpdir)
+
+  spark = SparkSession.builder.config('spark.driver.memory', '2g').master('local[2]').getOrCreate()
+  sc = spark.sparkContext
+
+  rows_count = 10
+  with materialize_dataset_carbon(spark, dataset_url, TestSchema, blocklet_size_mb):
+    rows_rdd = sc.parallelize(range(rows_count)) \
+      .map(test_row_generator) \
+      .map(lambda x: dict_to_spark_row(TestSchema, x))
+
+    spark.createDataFrame(rows_rdd, TestSchema.as_spark_schema()) \
+      .write \
+      .save(path=dataset_url, format='carbon')
+
+  with make_carbon_reader(dataset_url, predicate=in_lambda(['id'], lambda x: x == 3)) as reader:
+    assert next(reader).id == 3
+  with make_carbon_reader(dataset_url, predicate=in_lambda(['id'], lambda x: x == '3')) as reader:
+    with pytest.raises(StopIteration):
+      # Predicate should have selected none, so a StopIteration should be raised.
+      next(reader)
diff --git a/python/pycarbon/tests/core/test_carbon_reader.py b/python/pycarbon/tests/core/test_carbon_reader.py
new file mode 100644
index 0000000..bba84e2
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon_reader.py
@@ -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.
+
+
+import numpy as np
+import pytest
+
+from pycarbon.core.carbon_reader import make_batch_carbon_reader
+
+import os
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+# pylint: disable=unnecessary-lambda
+_TP = [
+  lambda url, **kwargs: make_batch_carbon_reader(url, reader_pool_type='thread', **kwargs),
+]
+
+
+def _check_simple_reader(reader, expected_data):
+  # Read a bunch of entries from the dataset and compare the data to reference
+  expected_field_names = expected_data[0].keys()
+  count = 0
+  for row in reader:
+    actual = row._asdict()
+
+    # Compare value of each entry in the batch
+    for i, id_value in enumerate(actual['id']):
+      expected = next(d for d in expected_data if d['id'] == id_value)
+      for field in expected_field_names:
+        expected_value = expected[field]
+        actual_value = actual[field][i, ...]
+        np.testing.assert_equal(actual_value, expected_value)
+
+    count += len(actual['id'])
+
+  assert count == len(expected_data)
+
+
+@pytest.mark.parametrize('reader_factory', _TP)
+def test_simple_read(carbon_scalar_dataset, reader_factory):
+  """Just a bunch of read and compares of all values to the expected values using the different reader pools"""
+  with reader_factory(carbon_scalar_dataset.url) as reader:
+    _check_simple_reader(reader, carbon_scalar_dataset.data)
+
+
+@pytest.mark.parametrize('reader_factory', _TP)
+def test_specify_columns_to_read(carbon_scalar_dataset, reader_factory):
+  """Just a bunch of read and compares of all values to the expected values using the different reader pools"""
+  with reader_factory(carbon_scalar_dataset.url, schema_fields=['id', 'float.*$']) as reader:
+    sample = next(reader)
+    assert set(sample._asdict().keys()) == {'id', 'float64'}
+    assert sample.float64.size > 0
+
+
+@pytest.mark.parametrize('reader_factory', _TP)
+def test_many_columns_non_unischema_dataset(carbon_many_columns_non_unischema_dataset, reader_factory):
+  """Check if we can read a dataset with huge number of columns (1000 in this case)"""
+  with reader_factory(carbon_many_columns_non_unischema_dataset.url) as reader:
+    sample = next(reader)
+    assert set(sample._fields) == set(carbon_many_columns_non_unischema_dataset.data[0].keys())
diff --git a/python/pycarbon/tests/core/test_carbon_tf_dataset.py b/python/pycarbon/tests/core/test_carbon_tf_dataset.py
new file mode 100644
index 0000000..1131c8f
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon_tf_dataset.py
@@ -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.
+
+
+import functools
+import operator
+import os
+from copy import copy
+
+import numpy as np
+import pytest
+import tensorflow as tf
+
+from petastorm.ngram import NGram
+from petastorm.predicates import in_lambda
+
+from pycarbon.core.carbon_reader import make_carbon_reader, make_batch_carbon_reader
+from pycarbon.core.carbon_tf_utils import make_pycarbon_dataset
+from pycarbon.tests.conftest import _ROWS_COUNT
+
+from pycarbon.tests.core.test_carbon_common import TestSchema
+
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+_EXCLUDE_FIELDS = set(TestSchema.fields.values()) - {TestSchema.decimal}
+
+ALL_READER_FLAVOR_FACTORIES = [
+  lambda url, **kwargs: make_carbon_reader(url, **_merge_params({'reader_pool_type': 'thread', 'workers_count': 1,
+                                                                 'schema_fields': _EXCLUDE_FIELDS}, kwargs)),
+]
+
+
+def _merge_params(base, overwrite):
+  """Merges two dictionaries when values from ``overwrite`` takes precedence over values of ``base`` dictionary.
+
+  Both input parameters are not modified.
+
+  :param base: A dictionary
+  :param overwrite: A dictionary. If a value with the same key exists in ``base``, it is overwritten by the value from
+    this dictionary.
+  :return: A combined dictionary
+  """
+  # Create a shallow copy of base
+  combined = copy(base)
+  combined.update(overwrite)
+  return combined
+
+
+@pytest.mark.forked
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+def test_with_one_shot_iterator(carbon_synthetic_dataset, reader_factory):
+  """Just a bunch of read and compares of all values to the expected values"""
+  with reader_factory(carbon_synthetic_dataset.url) as reader:
+    dataset = make_pycarbon_dataset(reader)
+    iterator = dataset.make_one_shot_iterator()
+
+    # Make sure we have static shape info for all fields
+    for shape in dataset.output_shapes:
+      # TODO(yevgeni): check that the shapes are actually correct, not just not None
+      assert shape.dims is not None
+
+    # Read a bunch of entries from the dataset and compare the data to reference
+    with tf.Session() as sess:
+      iterator = iterator.get_next()
+      for _, _ in enumerate(carbon_synthetic_dataset.data):
+        actual = sess.run(iterator)._asdict()
+        expected = next(d for d in carbon_synthetic_dataset.data if d['id'] == actual['id'])
+        for key in actual.keys():
+          if isinstance(expected[key], str):
+            # Tensorflow returns all strings as bytes in python3. So we will need to decode it
+            actual_value = actual[key].decode()
+          elif isinstance(expected[key], np.ndarray) and expected[key].dtype.type == np.unicode_:
+            actual_value = np.array([item.decode() for item in actual[key]])
+          else:
+            actual_value = actual[key]
+
+          np.testing.assert_equal(actual_value, expected[key])
+
+      # Exhausted one full epoch. Fetching next value should trigger OutOfRangeError
+      with pytest.raises(tf.errors.OutOfRangeError):
+        sess.run(iterator)
+
+
+@pytest.mark.forked
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+def test_with_dataset_repeat(carbon_synthetic_dataset, reader_factory):
+  """``tf.data.Dataset``'s ``repeat`` should not be used on ``make_pycarbon_dataset`` due to high costs of
+  ``Reader initialization``. A user should use ``Reader`` built-in epochs support. Check that we raise an
+  error to alert of misuse."""
+  with reader_factory(carbon_synthetic_dataset.url) as reader:
+    dataset = make_pycarbon_dataset(reader)
+
+    dataset = dataset.repeat(2)
+
+    iterator = dataset.make_one_shot_iterator()
+
+    # Read a bunch of entries from the dataset and compare the data to reference
+    with tf.Session() as sess:
+      iterator = iterator.get_next()
+
+      for _, _ in enumerate(carbon_synthetic_dataset.data):
+        sess.run(iterator)
+
+      with pytest.raises(tf.errors.UnknownError, match=r'.*Multiple iterations.*'):
+        sess.run(iterator)
+
+
+@pytest.mark.forked
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+def test_some_processing_functions(carbon_synthetic_dataset, reader_factory):
+  """Try several ``tf.data.Dataset`` dataset operations on make_pycarbon_dataset"""
+
+  # reader1 will have a single row with id=1, reader2: a single row with id=2
+
+  # Using functools.partial(_eq, 1)) which is equivalent to lambda x: x==1 because standard python pickle
+  # can not pickle this lambda
+  with reader_factory(carbon_synthetic_dataset.url,
+                      predicate=in_lambda(['id'], functools.partial(operator.eq, 1))) as reader1:
+    with reader_factory(carbon_synthetic_dataset.url,
+                        predicate=in_lambda(['id'], functools.partial(operator.eq, 2))) as reader2:
+      dataset = make_pycarbon_dataset(reader1) \
+        .prefetch(10) \
+        .concatenate(make_pycarbon_dataset(reader2)) \
+        .map(lambda x: x.id) \
+        .batch(2)
+
+      next_sample = dataset.make_one_shot_iterator().get_next()
+
+      with tf.Session() as sess:
+        # 'actual' is expected to be content of id column of a concatenated dataset
+        actual = sess.run(next_sample)
+        np.testing.assert_array_equal(actual, [1, 2])
+
+
+@pytest.mark.parametrize('reader_factory', ALL_READER_FLAVOR_FACTORIES)
+def test_dataset_on_ngram_not_supported(carbon_synthetic_dataset, reader_factory):
+  ngram = NGram({0: list(_EXCLUDE_FIELDS), 1: [TestSchema.id]}, 100, TestSchema.id)
+  with reader_factory(carbon_synthetic_dataset.url, schema_fields=ngram) as reader:
+    with pytest.raises(NotImplementedError):
+      make_pycarbon_dataset(reader)
+
+
+@pytest.mark.forked
+def test_non_unischema_with_many_colums_with_one_shot_iterator(carbon_many_columns_non_unischema_dataset):
+  """Just a bunch of read and compares of all values to the expected values"""
+  with make_batch_carbon_reader(carbon_many_columns_non_unischema_dataset.url, workers_count=1) as reader:
+    dataset = make_pycarbon_dataset(reader)
+    iterator = dataset.make_one_shot_iterator()
+
+    # Make sure we have static shape info for all fields
+    for shape in dataset.output_shapes:
+      # TODO(yevgeni): check that the shapes are actually correct, not just not None
+      assert shape.dims is not None
+
+    # Read a bunch of entries from the dataset and compare the data to reference
+    with tf.Session() as sess:
+      iterator = iterator.get_next()
+      sample = sess.run(iterator)._asdict()
+      assert set(sample.keys()) == set(carbon_many_columns_non_unischema_dataset.data[0].keys())
+
+
+@pytest.mark.forked
+def test_dataset_carbon_reader(carbon_synthetic_dataset):
+  with make_carbon_reader(carbon_synthetic_dataset.url, num_epochs=1) as reader:
+    dataset = make_pycarbon_dataset(reader) \
+      .batch(batch_size=1)
+
+    iterator = dataset.make_one_shot_iterator()
+
+    tensor = iterator.get_next()
+
+    with tf.Session() as sess:
+      sess.run([
+        tf.local_variables_initializer(),
+        tf.global_variables_initializer(),
+      ])
+      i = 0
+      try:
+        while True:
+          sess.run(tensor)
+          i += 1
+      except tf.errors.OutOfRangeError:
+        print("Finish! the number is " + str(i))
+
+      assert i == _ROWS_COUNT
+
+
+@pytest.mark.forked
+def test_dataset_batch_carbon_reader(carbon_scalar_dataset):
+  with make_batch_carbon_reader(carbon_scalar_dataset.url, num_epochs=1) as reader:
+    dataset = make_pycarbon_dataset(reader) \
+      .apply(tf.data.experimental.unbatch()) \
+      .batch(batch_size=1)
+
+    iterator = dataset.make_one_shot_iterator()
+
+    tensor = iterator.get_next()
+
+    with tf.Session() as sess:
+      sess.run([
+        tf.local_variables_initializer(),
+        tf.global_variables_initializer(),
+      ])
+      i = 0
+      try:
+        while True:
+          sess.run(tensor)
+          i += 1
+      except tf.errors.OutOfRangeError:
+        print("Finish! the number is " + str(i))
+
+      assert i == _ROWS_COUNT
+
+
+@pytest.mark.forked
+def test_dynamic_batch_size_of_carbon_reader(carbon_synthetic_dataset):
+  with make_carbon_reader(carbon_synthetic_dataset.url, num_epochs=None) as reader:
+    batch_size = tf.data.Dataset.range(1, 10).make_one_shot_iterator().get_next()
+
+    dataset = make_pycarbon_dataset(reader) \
+      .batch(batch_size=batch_size)
+
+    iterator = dataset.make_initializable_iterator()
+    init_op = iterator.initializer
+
+    tensor = iterator.get_next()
+
+    with tf.train.MonitoredTrainingSession() as sess:
+      sess.run(init_op)
+      sample = sess.run(tensor)
+      assert 1 == len(sample.id)
+
+      sess.run(init_op)
+      sample = sess.run(tensor)
+      assert 2 == len(sample.id)
+
+
+@pytest.mark.forked
+def test_dynamic_batch_size_of_batch_carbon_reader(carbon_scalar_dataset):
+  with make_batch_carbon_reader(carbon_scalar_dataset.url, num_epochs=None) as reader:
+    batch_size = tf.data.Dataset.range(1, 10).make_one_shot_iterator().get_next()
+
+    dataset = make_pycarbon_dataset(reader) \
+      .apply(tf.data.experimental.unbatch()) \
+      .batch(batch_size=batch_size)
+
+    iterator = dataset.make_initializable_iterator()
+    init_op = iterator.initializer
+
+    tensor = iterator.get_next()
+
+    with tf.train.MonitoredTrainingSession() as sess:
+      sess.run(init_op)
+      sample = sess.run(tensor)
+      assert 1 == len(sample.id)
+
+      sess.run(init_op)
+      sample = sess.run(tensor)
+      assert 2 == len(sample.id)
diff --git a/python/pycarbon/tests/core/test_carbon_tf_utils.py b/python/pycarbon/tests/core/test_carbon_tf_utils.py
new file mode 100644
index 0000000..4527c5b
--- /dev/null
+++ b/python/pycarbon/tests/core/test_carbon_tf_utils.py
@@ -0,0 +1,347 @@
+# 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.
+
+
+from __future__ import division
+
+import datetime
+from calendar import timegm
+from collections import namedtuple
+from contextlib import contextmanager
+from decimal import Decimal
+
+import os
+import numpy as np
+import pytest
+import tensorflow as tf
+
+from petastorm.ngram import NGram
+from petastorm.unischema import Unischema, UnischemaField
+
+from pycarbon.core.carbon_reader import make_carbon_reader, make_batch_carbon_reader
+from pycarbon.tests.core.test_carbon_common import TestSchema
+from pycarbon.core.carbon_tf_utils import _sanitize_field_tf_types, _numpy_to_tf_dtypes, \
+  _schema_to_tf_dtypes, tf_tensors
+
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+NON_NULLABLE_FIELDS = set(TestSchema.fields.values())
+
+
+@contextmanager
+def _tf_session():
+  with tf.Session() as sess:
+    sess.run([tf.global_variables_initializer(), tf.local_variables_initializer()])
+
+    coord = tf.train.Coordinator()
+    threads = tf.train.start_queue_runners(coord=coord, start=True)
+
+    yield sess
+
+    coord.request_stop()
+    coord.join(threads)
+
+
+def test_sanitize_field_tf_types():
+  expected_datetime_array = [datetime.date(1970, 1, 1), datetime.date(2015, 9, 29)]
+  expected_datetime_ns_from_epoch = [timegm(dt.timetuple()) * 1000000000 for dt in expected_datetime_array]
+  assert expected_datetime_ns_from_epoch[0] == 0
+
+  sample_input_dict = {
+    'int32': np.asarray([-2 ** 31, 0, 100, 2 ** 31 - 1], dtype=np.int32),
+    'uint16': np.asarray([0, 2, 2 ** 16 - 1], dtype=np.uint16),
+    'Decimal': Decimal(1234) / Decimal(10),
+    'array_of_datetime_date': np.asarray(expected_datetime_array),
+    'array_of_np_datetime_64': np.asarray(expected_datetime_array).astype(np.datetime64),
+  }
+
+  TestNamedTuple = namedtuple('TestNamedTuple', sample_input_dict.keys())
+  sample_input_tuple = TestNamedTuple(**sample_input_dict)
+  sanitized_tuple = _sanitize_field_tf_types(sample_input_tuple)
+
+  np.testing.assert_equal(sanitized_tuple.int32.dtype, np.int32)
+  np.testing.assert_equal(sanitized_tuple.uint16.dtype, np.int32)
+  assert isinstance(sanitized_tuple.Decimal, str)
+
+  np.testing.assert_equal(sanitized_tuple.int32, sample_input_dict['int32'])
+  np.testing.assert_equal(sanitized_tuple.uint16, sample_input_dict['uint16'])
+  np.testing.assert_equal(str(sanitized_tuple.Decimal), str(sample_input_dict['Decimal'].normalize()))
+
+  np.testing.assert_equal(sanitized_tuple.array_of_datetime_date, expected_datetime_ns_from_epoch)
+  np.testing.assert_equal(sanitized_tuple.array_of_np_datetime_64, expected_datetime_ns_from_epoch)
+
+
+def test_invalid_sanitize_field_tf_types():
+  sample_input_dict = {
+    'int32': None,
+  }
+
+  TestNamedTuple = namedtuple('TestNamedTuple', sample_input_dict.keys())
+  sample_input_tuple = TestNamedTuple(**sample_input_dict)
+  with pytest.raises(RuntimeError):
+    _sanitize_field_tf_types(sample_input_tuple)
+
+
+def test_decimal_conversion():
+  assert _numpy_to_tf_dtypes(Decimal) == tf.string
+
+
+def test_uint16_promotion_to_int32():
+  assert _numpy_to_tf_dtypes(np.uint16) == tf.int32
+
+
+def test_unknown_type():
+  with pytest.raises(ValueError):
+    _numpy_to_tf_dtypes(np.uint64)
+
+
+def test_schema_to_dtype_list():
+  TestSchema = Unischema('TestSchema', [
+    UnischemaField('int32', np.int32, (), None, False),
+    UnischemaField('uint8', np.uint8, (), None, False),
+    UnischemaField('uint16', np.uint16, (), None, False),
+    UnischemaField('Decimal', Decimal, (), None, False),
+  ])
+
+  actual_tf_dtype_list = _schema_to_tf_dtypes(TestSchema)
+  # Note that the order of the fields is defined by alphabetical order of keys and always sorted by Unischema
+  # to avoid ambiguity
+  #  [Decimal,   int32,    uint16,   uint8] <- alphabetical order
+  #  [tf.string, tf.int32, tf.int32, tf.uint8]
+  np.testing.assert_equal(actual_tf_dtype_list, [tf.string, tf.int32, tf.int32, tf.uint8])
+
+
+def _read_from_tf_tensors(synthetic_dataset, count, shuffling_queue_capacity, min_after_dequeue, ngram,
+                          workers_count=10):
+  """Used by several test cases. Reads a 'count' rows using reader.
+
+  The reader is configured without row-group shuffling and guarantees deterministic order of rows up to the
+  results queue TF shuffling which is controlled by 'shuffling_queue_capacity', 'min_after_dequeue' arguments.
+
+  The function returns a tuple with: (actual data read from the dataset, a TF tensor returned by the reader)
+  """
+
+  schema_fields = (NON_NULLABLE_FIELDS if ngram is None else ngram)
+
+  with make_carbon_reader(schema_fields=schema_fields, dataset_url=synthetic_dataset.url, reader_pool_type='thread',
+                          shuffle_blocklets=False, workers_count=workers_count) as reader:
+    row_tensors = tf_tensors(reader, shuffling_queue_capacity=shuffling_queue_capacity,
+                             min_after_dequeue=min_after_dequeue)
+
+    with _tf_session() as sess:
+      rows_data = [sess.run(row_tensors) for _ in range(count)]
+
+  return rows_data, row_tensors
+
+
+def _assert_all_tensors_have_shape(row_tensors):
+  """Asserts that all elements in row_tensors list/tuple have static shape."""
+  for column in row_tensors:
+    assert column.get_shape().dims is not None
+
+
+def _assert_fields_eq(actual, desired):
+  if isinstance(desired, Decimal) or isinstance(actual, bytes):
+    # Tensorflow returns all strings as bytes in python3. So we will need to decode it
+    actual = actual.decode()
+  elif isinstance(desired, np.ndarray) and desired.dtype.type == np.unicode_:
+    actual = np.array([item.decode() for item in actual])
+
+  if isinstance(desired, Decimal):
+    np.testing.assert_equal(Decimal(actual), desired)
+  elif issubclass(desired.dtype.type, np.datetime64):
+    # tf_utils will convert timestamps to ns from epoch int64 value.
+    assert desired.astype('<M8[ns]').astype(np.int64) == actual
+  else:
+    np.testing.assert_equal(actual, desired)
+
+
+def _assert_expected_rows_data(expected_data, rows_data):
+  """Asserts all elements of rows_data list of rows match reference data used to create the dataset"""
+  for row_tuple in rows_data:
+
+    # It is easier to work with dict as we will be indexing column names using strings
+    row = row_tuple._asdict()
+
+    # Find corresponding row in the reference data
+    expected = next(d for d in expected_data if d['id'] == row['id'])
+
+    # Check equivalence of all values between a checked row and a row from reference data
+    for column_name, actual in row.items():
+      expected_val = expected[column_name]
+      _assert_fields_eq(actual, expected_val)
+
+
+@pytest.mark.forked
+def test_simple_read_tensorflow(carbon_synthetic_dataset):
+  """Read couple of rows. Make sure all tensors have static shape sizes assigned and the data matches reference
+  data"""
+  with make_carbon_reader(schema_fields=NON_NULLABLE_FIELDS, dataset_url=carbon_synthetic_dataset.url) as reader:
+    row_tensors = tf_tensors(reader)
+    with _tf_session() as sess:
+      rows_data = [sess.run(row_tensors) for _ in range(30)]
+
+  # Make sure we have static shape info for all fields
+  _assert_all_tensors_have_shape(row_tensors)
+  _assert_expected_rows_data(carbon_synthetic_dataset.data, rows_data)
+
+
+@pytest.mark.forked
+def test_shuffling_queue(carbon_synthetic_dataset):
+  """Read data without tensorflow shuffling queue and with it. Check the the order is deterministic within
+  unshuffled read and is random with shuffled read"""
+  unshuffled_1, _ = _read_from_tf_tensors(carbon_synthetic_dataset, 30, shuffling_queue_capacity=0, min_after_dequeue=0,
+                                          ngram=None, workers_count=1)
+  unshuffled_2, _ = _read_from_tf_tensors(carbon_synthetic_dataset, 30, shuffling_queue_capacity=0, min_after_dequeue=0,
+                                          ngram=None, workers_count=1)
+
+  shuffled_1, shuffled_1_row_tensors = \
+    _read_from_tf_tensors(carbon_synthetic_dataset, 30, shuffling_queue_capacity=10, min_after_dequeue=9, ngram=None)
+  shuffled_2, _ = \
+    _read_from_tf_tensors(carbon_synthetic_dataset, 30, shuffling_queue_capacity=10, min_after_dequeue=9, ngram=None)
+
+  # Make sure we have static shapes and the data matches reference data (important since a different code path
+  # is executed within tf_tensors when shuffling is specified
+  _assert_all_tensors_have_shape(shuffled_1_row_tensors)
+  _assert_expected_rows_data(carbon_synthetic_dataset.data, shuffled_1)
+
+  assert [f.id for f in unshuffled_1] == [f.id for f in unshuffled_2]
+  assert [f.id for f in unshuffled_1] != [f.id for f in shuffled_2]
+  assert [f.id for f in shuffled_1] != [f.id for f in shuffled_2]
+
+
+@pytest.mark.forked
+def test_simple_ngram_read_tensorflow(carbon_synthetic_dataset):
+  """Read a single ngram. Make sure all shapes are set and the data read matches reference data"""
+  fields = {
+    0: [TestSchema.id],
+    1: [TestSchema.id],
+    2: [TestSchema.id]
+  }
+
+  # Expecting delta between ids to be 1. Setting 1.5 as upper bound
+  ngram = NGram(fields=fields, delta_threshold=1.5, timestamp_field=TestSchema.id)
+
+  ngrams, row_tensors_seq = \
+    _read_from_tf_tensors(carbon_synthetic_dataset, 30, shuffling_queue_capacity=0, min_after_dequeue=0, ngram=ngram)
+
+  for row_tensors in row_tensors_seq.values():
+    _assert_all_tensors_have_shape(row_tensors)
+
+  for one_ngram_dict in ngrams:
+    _assert_expected_rows_data(carbon_synthetic_dataset.data, one_ngram_dict.values())
+
+
+@pytest.mark.forked
+def test_shuffling_queue_with_ngrams(carbon_synthetic_dataset):
+  """Read data without tensorflow shuffling queue and with it (no rowgroup shuffling). Read ngrams
+  Check the the order is deterministic within unshuffled read and is random with shuffled read"""
+  fields = {
+    0: [TestSchema.id],
+    1: [TestSchema.id],
+    2: [TestSchema.id]
+  }
+
+  # Expecting delta between ids to be 1. Setting 1.5 as upper bound
+  ngram = NGram(fields=fields, delta_threshold=1.5, timestamp_field=TestSchema.id)
+  unshuffled_1, _ = _read_from_tf_tensors(carbon_synthetic_dataset, 30, shuffling_queue_capacity=0, min_after_dequeue=0,
+                                          ngram=ngram, workers_count=1)
+  unshuffled_2, _ = _read_from_tf_tensors(carbon_synthetic_dataset, 30, shuffling_queue_capacity=0, min_after_dequeue=0,
+                                          ngram=ngram, workers_count=1)
+
+  shuffled_1, shuffled_1_ngram = \
+    _read_from_tf_tensors(carbon_synthetic_dataset, 20, shuffling_queue_capacity=30, min_after_dequeue=29, ngram=ngram)
+  shuffled_2, _ = \
+    _read_from_tf_tensors(carbon_synthetic_dataset, 20, shuffling_queue_capacity=30, min_after_dequeue=29, ngram=ngram)
+
+  # shuffled_1_ngram is a dictionary of named tuple indexed by time:
+  # {0: (tensor, tensor, tensor, ...),
+  #  1: (tensor, tensor, tensor, ...),
+  #  ...}
+  for row_tensor in shuffled_1_ngram.values():
+    _assert_all_tensors_have_shape(row_tensor)
+
+  # shuffled_1 is a list of dictionaries of named tuples indexed by time:
+  # [{0: (tensor, tensor, tensor, ...),
+  #  1: (tensor, tensor, tensor, ...),
+  #  ...}
+  # {0: (tensor, tensor, tensor, ...),
+  #  1: (tensor, tensor, tensor, ...),
+  #  ...},...
+  # ]
+  for one_ngram_dict in shuffled_1:
+    _assert_expected_rows_data(carbon_synthetic_dataset.data, one_ngram_dict.values())
+
+  def flatten(list_of_ngrams):
+    return [row for seq in list_of_ngrams for row in seq.values()]
+
+  assert [f.id for f in flatten(unshuffled_1)] == [f.id for f in flatten(unshuffled_2)]
+
+  assert [f.id for f in flatten(unshuffled_1)] != [f.id for f in flatten(shuffled_2)]
+  assert [f.id for f in flatten(shuffled_1)] != [f.id for f in flatten(shuffled_2)]
+
+
+@pytest.mark.forked
+def test_simple_read_tensorflow_with_carbon_dataset(carbon_scalar_dataset):
+  """Read couple of rows. Make sure all tensors have static shape sizes assigned and the data matches reference
+  data"""
+  with make_batch_carbon_reader(dataset_url=carbon_scalar_dataset.url) as reader:
+    row_tensors = tf_tensors(reader)
+    # Make sure we have static shape info for all fields
+    for column in row_tensors:
+      assert column.get_shape().as_list() == [None]
+
+    with _tf_session() as sess:
+      for _ in range(2):
+        batch = sess.run(row_tensors)._asdict()
+        for i, id_value in enumerate(batch['id']):
+          expected_row = next(d for d in carbon_scalar_dataset.data if d['id'] == id_value)
+          for field_name in expected_row.keys():
+            _assert_fields_eq(batch[field_name][i], expected_row[field_name])
+
+
+@pytest.mark.forked
+def test_simple_read_tensorflow_with_non_unischema_many_columns_dataset(carbon_many_columns_non_unischema_dataset):
+  """Read couple of rows. Make sure all tensors have static shape sizes assigned and the data matches reference
+  data"""
+  with make_batch_carbon_reader(dataset_url=carbon_many_columns_non_unischema_dataset.url) as reader:
+    row_tensors = tf_tensors(reader)
+    # Make sure we have static shape info for all fields
+    for column in row_tensors:
+      assert column.get_shape().as_list() == [None]
+
+    with _tf_session() as sess:
+      batch = sess.run(row_tensors)._asdict()
+      assert set(batch.keys()) == set(carbon_many_columns_non_unischema_dataset.data[0].keys())
+
+
+def test_shuffling_queue_with_make_batch_carbon_reader(carbon_scalar_dataset):
+  with make_batch_carbon_reader(dataset_url=carbon_scalar_dataset.url) as reader:
+    with pytest.raises(ValueError):
+      tf_tensors(reader, 100, 90)
diff --git a/python/pycarbon/tests/core/test_reader.py b/python/pycarbon/tests/core/test_reader.py
new file mode 100644
index 0000000..eb45903
--- /dev/null
+++ b/python/pycarbon/tests/core/test_reader.py
@@ -0,0 +1,127 @@
+# 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.
+
+
+from time import sleep
+
+import pytest
+
+from pycarbon.core.carbon_reader import make_carbon_reader, make_batch_carbon_reader
+from pycarbon.core.carbon_reader import CarbonDataReader
+from pycarbon.core.carbon import CarbonDataset
+
+import os
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+# pylint: disable=unnecessary-lambda
+READER_FACTORIES = [
+  make_carbon_reader,
+  lambda url, **kwargs: make_carbon_reader(url, **kwargs),
+  make_batch_carbon_reader,
+  lambda url, **kwargs: make_batch_carbon_reader(url, **kwargs),
+]
+
+
+@pytest.mark.parametrize('reader_factory', READER_FACTORIES)
+def test_dataset_url_must_be_string(reader_factory):
+  with pytest.raises(ValueError):
+    reader_factory(None)
+
+  with pytest.raises(ValueError):
+    reader_factory(123)
+
+  with pytest.raises(ValueError):
+    reader_factory([])
+
+
+def test_diagnostics_reader_v1(carbon_synthetic_dataset):
+  with make_carbon_reader(carbon_synthetic_dataset.url) as reader:
+    next(reader)
+    diags = reader.diagnostics
+    # # Hard to make a meaningful assert on the content of the diags without potentially introducing a race
+    assert 'output_queue_size' in diags
+
+
+def test_normalize_shuffle_partitions(carbon_synthetic_dataset):
+  dataset = CarbonDataset(carbon_synthetic_dataset.path)
+  row_drop_partitions = CarbonDataReader._normalize_shuffle_options(1, dataset)
+  assert row_drop_partitions == 1
+
+  row_drop_partitions = CarbonDataReader._normalize_shuffle_options(100, dataset)
+  assert row_drop_partitions == 100
+
+
+def test_bound_size_of_output_queue_size_reader(carbon_synthetic_dataset):
+  """This test is timing sensitive so it might become flaky"""
+  TIME_TO_GET_TO_STATIONARY_STATE = 0.5
+
+  with make_carbon_reader(carbon_synthetic_dataset.url) as reader:
+    next(reader)
+    sleep(TIME_TO_GET_TO_STATIONARY_STATE)
+    assert reader.diagnostics['output_queue_size'] > 0
+
+
+@pytest.mark.parametrize('reader_factory', READER_FACTORIES)
+def test_invalid_cache_type(carbon_synthetic_dataset, reader_factory):
+  with pytest.raises(ValueError, match='Unknown cache_type'):
+    reader_factory(carbon_synthetic_dataset.url, cache_type='bogus_cache_type')
+
+
+@pytest.mark.parametrize('reader_factory', READER_FACTORIES)
+def test_invalid_reader_pool_type(carbon_synthetic_dataset, reader_factory):
+  with pytest.raises(ValueError, match='Unknown reader_pool_type'):
+    reader_factory(carbon_synthetic_dataset.url, reader_pool_type='bogus_pool_type')
+
+
+@pytest.mark.parametrize('reader_factory', READER_FACTORIES)
+def test_unsupported_reader_pool_type(carbon_synthetic_dataset, reader_factory):
+  with pytest.raises(NotImplementedError):
+    reader_factory(carbon_synthetic_dataset.url, reader_pool_type='process')
+
+  with pytest.raises(NotImplementedError):
+    reader_factory(carbon_synthetic_dataset.url, reader_pool_type='dummy')
+
+
+def test_invalid_reader_engine(carbon_synthetic_dataset):
+  with pytest.raises(ValueError, match='Supported reader_engine values'):
+    make_carbon_reader(carbon_synthetic_dataset.url, reader_engine='bogus reader engine')
+
+
+def test_reader_engine_v2_not_supported(carbon_synthetic_dataset):
+  with pytest.raises(NotImplementedError):
+    make_carbon_reader(carbon_synthetic_dataset.url, reader_engine='experimental_reader_v2')
+
+
+@pytest.mark.parametrize('reader_factory', READER_FACTORIES)
+def test_invalid_obs_parameters(carbon_obs_dataset, reader_factory):
+  with pytest.raises(ValueError):
+    reader_factory(carbon_obs_dataset.url)
+
+  with pytest.raises(ValueError):
+    reader_factory(carbon_obs_dataset.wrong_url)
diff --git a/python/pycarbon/tests/hello_world/README.md b/python/pycarbon/tests/hello_world/README.md
new file mode 100644
index 0000000..d4a589f
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/README.md
@@ -0,0 +1,3 @@
+external_dataset: generating carbon dataset which has non-unischema (standard carbon schema)
+
+pycarbon_dataset: generating carbon dataset which has unischema
diff --git a/python/pycarbon/tests/hello_world/__init__.py b/python/pycarbon/tests/hello_world/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/__init__.py
diff --git a/python/pycarbon/tests/hello_world/dataset_with_normal_schema/__init__.py b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/__init__.py
diff --git a/python/pycarbon/tests/hello_world/dataset_with_normal_schema/generate_dataset_carbon.py b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/generate_dataset_carbon.py
new file mode 100644
index 0000000..8a8510b
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/generate_dataset_carbon.py
@@ -0,0 +1,77 @@
+# 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.
+
+
+"""
+This is part of a minimal example of how to use pycarbon to read a dataset not created
+with pycarbon. Generates a sample dataset from random data.
+"""
+
+import os
+import argparse
+import random
+from pyspark.sql import SparkSession, Row
+from pyspark.sql.types import StructType, StructField, IntegerType
+
+NON_PYCARBON_SCHEMA = StructType([
+  StructField("id", IntegerType(), True),
+  StructField("value1", IntegerType(), True),
+  StructField("value2", IntegerType(), True)
+])
+
+
+def row_generator(x):
+  """Returns a single entry in the generated dataset. Return a bunch of random values as an example."""
+  return Row(id=x, value1=random.randint(-255, 255), value2=random.randint(-255, 255))
+
+
+def generate_dataset_with_normal_schema(output_url='file:///tmp/carbon_external_dataset'):
+  # """Creates an example dataset at output_url in Carbon format"""
+  spark = SparkSession.builder \
+    .master('local') \
+    .getOrCreate()
+  sc = spark.sparkContext
+  sc.setLogLevel('INFO')
+
+  rows_count = 10
+  rows_rdd = sc.parallelize(range(rows_count)) \
+    .map(row_generator)
+
+  spark.createDataFrame(rows_rdd) \
+    .write \
+    .mode('overwrite') \
+    .save(path=output_url, format='carbon')
+
+
+if __name__ == '__main__':
+  parser = argparse.ArgumentParser(description='Pycarbon Hello World Example I')
+  parser.add_argument('-pp', '--pyspark-python', type=str, default=None,
+                      help='pyspark python env variable')
+  parser.add_argument('-pdp', '--pyspark-driver-python', type=str, default=None,
+                      help='pyspark driver python env variable')
+
+  args = parser.parse_args()
+
+  if args.pyspark_python is not None and args.pyspark_driver_python is not None:
+    os.environ['PYSPARK_PYTHON'] = args.pyspark_python
+    os.environ['PYSPARK_DRIVER_PYTHON'] = args.pyspark_driver_python
+  elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+    pass
+  else:
+    raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                     "using cmd line -pp PYSPARK_PYTHON_PATH -pdp PYSPARK_DRIVER_PYTHON_PATH, "
+                     "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+  generate_dataset_with_normal_schema()
diff --git a/python/pycarbon/tests/hello_world/dataset_with_normal_schema/python_hello_world_carbon.py b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/python_hello_world_carbon.py
new file mode 100644
index 0000000..0825934
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/python_hello_world_carbon.py
@@ -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.
+
+
+"""Minimal example of how to read samples from a dataset generated by `generate_external_dataset_carbon.py`
+using plain Python"""
+
+from __future__ import print_function
+
+import argparse
+import jnius_config
+
+from pycarbon.reader import make_reader
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+
+def python_hello_world(dataset_url='file:///tmp/carbon_external_dataset'):
+  # Reading data from the non-Pycarbon Carbon via pure Python
+  with make_reader(dataset_url, schema_fields=["id", "value1", "value2"]) as reader:
+    for schema_view in reader:
+      # make_reader() returns batches of rows instead of individual rows
+      print("Batched read:\nid: {0} value1: {1} value2: {2}".format(
+        schema_view.id, schema_view.value1, schema_view.value2))
+
+
+if __name__ == '__main__':
+  parser = argparse.ArgumentParser(description='Python hello world')
+  parser.add_argument('-c', '--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  python_hello_world()
diff --git a/python/pycarbon/tests/hello_world/dataset_with_normal_schema/tensorflow_hello_world_carbon.py b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/tensorflow_hello_world_carbon.py
new file mode 100644
index 0000000..5441e4b
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/tensorflow_hello_world_carbon.py
@@ -0,0 +1,62 @@
+# 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.
+
+
+"""Minimal example of how to read samples from a dataset generated by `generate_external_dataset_carbon.py`
+using tensorflow, using make_batch_carbon_reader() instead of make_carbon_reader()"""
+
+from __future__ import print_function
+
+import argparse
+import jnius_config
+
+import tensorflow as tf
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+from pycarbon.core.carbon_tf_utils import tf_tensors, make_pycarbon_dataset
+
+from pycarbon.core.carbon_reader import make_batch_carbon_reader
+
+from pycarbon.reader import make_reader
+from pycarbon.reader import make_tensor, make_dataset
+
+def tensorflow_hello_world(dataset_url='file:///tmp/carbon_external_dataset'):
+  # Example: tf_tensors will return tensors with dataset data
+  with make_reader(dataset_url) as reader:
+    tensor = tf_tensors(reader)
+    with tf.Session() as sess:
+      # Because we are using make_reader(), each read returns a batch of rows instead of a single row
+      batched_sample = sess.run(tensor)
+      print("id batch: {0}".format(batched_sample.id))
+
+  # Example: use tf.data.Dataset API
+  with make_reader(dataset_url) as reader:
+    dataset = make_dataset(reader)
+    iterator = dataset.make_one_shot_iterator()
+    tensor = iterator.get_next()
+    with tf.Session() as sess:
+      batched_sample = sess.run(tensor)
+      print("id batch: {0}".format(batched_sample.id))
+
+if __name__ == '__main__':
+  parser = argparse.ArgumentParser(description='Tensorflow hello world')
+  parser.add_argument('-c', '--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  tensorflow_hello_world()
diff --git a/python/pycarbon/tests/hello_world/dataset_with_normal_schema/tests/test_generate_dataset_carbon_with_normal_schema.py b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/tests/test_generate_dataset_carbon_with_normal_schema.py
new file mode 100644
index 0000000..bcc5342
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_normal_schema/tests/test_generate_dataset_carbon_with_normal_schema.py
@@ -0,0 +1,73 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+import os
+
+import pytest
+
+from pycarbon.tests.hello_world.dataset_with_normal_schema.generate_dataset_carbon import generate_dataset_with_normal_schema
+from pycarbon.tests.hello_world.dataset_with_normal_schema.python_hello_world_carbon import python_hello_world
+from pycarbon.tests.hello_world.dataset_with_normal_schema.tensorflow_hello_world_carbon import tensorflow_hello_world
+from petastorm.tests.conftest import SyntheticDataset
+
+from pycarbon.core.Constants import LOCAL_FILE_PREFIX
+from pycarbon.core.carbon_reader import make_batch_carbon_reader
+
+from pycarbon.reader import make_reader
+
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line --pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH, "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+
+@pytest.fixture(scope="session")
+def dataset(tmpdir_factory):
+  path = tmpdir_factory.mktemp("data").strpath
+  url = LOCAL_FILE_PREFIX + path
+
+  generate_dataset_with_normal_schema(url)
+
+  dataset = SyntheticDataset(url=url, path=path, data=None)
+
+  # Generate a dataset
+  assert os.path.exists(os.path.join(path, '_SUCCESS'))
+
+  return dataset
+
+
+def test_generate(dataset):
+  # Read from it using a plain reader
+  with make_reader(dataset.url) as reader:
+    all_samples = list(reader)
+  assert all_samples
+
+def test_python_hello_world_external_dataset_example(dataset):
+  python_hello_world(dataset.url)
+
+
+def test_tensorflow_hello_world_external_dataset_example(dataset):
+  tensorflow_hello_world(dataset.url)
diff --git a/python/pycarbon/tests/hello_world/dataset_with_unischema/__init__.py b/python/pycarbon/tests/hello_world/dataset_with_unischema/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_unischema/__init__.py
diff --git a/python/pycarbon/tests/hello_world/dataset_with_unischema/generate_pycarbon_dataset.py b/python/pycarbon/tests/hello_world/dataset_with_unischema/generate_pycarbon_dataset.py
new file mode 100644
index 0000000..b80e67b
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_unischema/generate_pycarbon_dataset.py
@@ -0,0 +1,96 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+"""
+This is a minimal example of how to generate a pycarbon dataset. Generates a
+sample dataset with some random data.
+"""
+
+import os
+import argparse
+
+import jnius_config
+
+import numpy as np
+from pyspark.sql import SparkSession
+from pyspark.sql.types import IntegerType
+
+from petastorm.codecs import ScalarCodec, CompressedImageCodec, NdarrayCodec
+from petastorm.unischema import dict_to_spark_row, Unischema, UnischemaField
+
+from pycarbon.core.carbon_dataset_metadata import materialize_dataset_carbon
+
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+# The schema defines how the dataset schema looks like
+HelloWorldSchema = Unischema('HelloWorldSchema', [
+  UnischemaField('id', np.int_, (), ScalarCodec(IntegerType()), False),
+  UnischemaField('image1', np.uint8, (128, 256, 3), CompressedImageCodec('png'), False),
+  UnischemaField('array_4d', np.uint8, (None, 128, 30, None), NdarrayCodec(), False),
+])
+
+
+def row_generator(x):
+  """Returns a single entry in the generated dataset. Return a bunch of random values as an example."""
+  return {'id': x,
+          'image1': np.random.randint(0, 255, dtype=np.uint8, size=(128, 256, 3)),
+          'array_4d': np.random.randint(0, 255, dtype=np.uint8, size=(4, 128, 30, 3))}
+
+
+def generate_pycarbon_dataset(output_url='file:///tmp/carbon_pycarbon_dataset'):
+  blocklet_size_mb = 256
+
+  spark = SparkSession.builder.config('spark.driver.memory', '2g').master('local[2]').getOrCreate()
+  sc = spark.sparkContext
+
+  # Wrap dataset materialization portion. Will take care of setting up spark environment variables as
+  # well as save pycarbon specific metadata
+  rows_count = 10
+  with materialize_dataset_carbon(spark, output_url, HelloWorldSchema, blocklet_size_mb):
+    rows_rdd = sc.parallelize(range(rows_count)) \
+      .map(row_generator) \
+      .map(lambda x: dict_to_spark_row(HelloWorldSchema, x))
+
+    spark.createDataFrame(rows_rdd, HelloWorldSchema.as_spark_schema()) \
+      .coalesce(10) \
+      .write \
+      .mode('overwrite') \
+      .save(path=output_url, format='carbon')
+
+
+if __name__ == '__main__':
+  parser = argparse.ArgumentParser(description='Pycarbon Example II')
+  parser.add_argument('-pp', '--pyspark-python', type=str, default=None,
+                      help='pyspark python env variable')
+  parser.add_argument('-pdp', '--pyspark-driver-python', type=str, default=None,
+                      help='pyspark driver python env variable')
+  parser.add_argument('-c', '--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  if args.pyspark_python is not None and args.pyspark_driver_python is not None:
+    os.environ['PYSPARK_PYTHON'] = args.pyspark_python
+    os.environ['PYSPARK_DRIVER_PYTHON'] = args.pyspark_driver_python
+  elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+    pass
+  else:
+    raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                     "using cmd line -pp PYSPARK_PYTHON_PATH -pdp PYSPARK_DRIVER_PYTHON_PATH, "
+                     "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+  generate_pycarbon_dataset()
diff --git a/python/pycarbon/tests/hello_world/dataset_with_unischema/pyspark_hello_world_carbon.py b/python/pycarbon/tests/hello_world/dataset_with_unischema/pyspark_hello_world_carbon.py
new file mode 100644
index 0000000..db270d8
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_unischema/pyspark_hello_world_carbon.py
@@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+"""Minimal example of how to read samples from a dataset generated by `generate_pycarbon_dataset.py`
+using pyspark"""
+
+from __future__ import print_function
+
+from pyspark.sql import SparkSession
+
+
+def pyspark_hello_world(dataset_url='file:///tmp/carbon_pycarbon_dataset'):
+  spark = SparkSession \
+    .builder \
+    .master('local[1]') \
+    .getOrCreate()
+
+  dataset_path = dataset_url[7:]
+  # Create a dataframe object from carbon files
+  spark.sql("create table readcarbon using carbon location '" + str(dataset_path) + "'")
+  dataframe = spark.sql("select * from readcarbon")
+
+  # Show a schema
+  dataframe.printSchema()
+
+  # Count all
+  dataframe.count()
+
+  # Show just some columns
+  dataframe.select('id').show()
+
+  # This is how you can use a standard SQL to query a dataset. Note that the data is not decoded in this case.
+  number_of_rows = spark.sql(
+    'SELECT count(id) '
+    'from carbon.`{}` '.format(dataset_url)).collect()
+  print('Number of rows in the dataset: {}'.format(number_of_rows[0][0]))
+
+
+if __name__ == '__main__':
+  pyspark_hello_world()
diff --git a/python/pycarbon/tests/hello_world/dataset_with_unischema/python_hello_world_carbon.py b/python/pycarbon/tests/hello_world/dataset_with_unischema/python_hello_world_carbon.py
new file mode 100644
index 0000000..b0468e8
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_unischema/python_hello_world_carbon.py
@@ -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.
+
+
+"""Minimal example of how to read samples from a dataset generated by `generate_pycarbon_dataset.py`
+using plain Python"""
+
+from __future__ import print_function
+
+import argparse
+import jnius_config
+
+from pycarbon.reader import make_reader
+
+
+def python_hello_world(dataset_url='file:///tmp/carbon_pycarbon_dataset'):
+  with make_reader(dataset_url) as reader:
+    # Pure python
+    for sample in reader:
+      print(sample.id)
+
+  with make_reader(dataset_url, is_batch=False) as reader:
+    # Pure python
+    for sample in reader:
+      print(sample.id)
+
+
+if __name__ == '__main__':
+  parser = argparse.ArgumentParser(description='Python hello world')
+  parser.add_argument('-c', '--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  python_hello_world()
diff --git a/python/pycarbon/tests/hello_world/dataset_with_unischema/tensorflow_hello_world_carbon.py b/python/pycarbon/tests/hello_world/dataset_with_unischema/tensorflow_hello_world_carbon.py
new file mode 100644
index 0000000..fa44807
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_unischema/tensorflow_hello_world_carbon.py
@@ -0,0 +1,58 @@
+# 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.
+
+
+"""Minimal example of how to read samples from a dataset generated by `generate_pycarbon_dataset.py`
+using tensorflow."""
+
+from __future__ import print_function
+
+import argparse
+import jnius_config
+import tensorflow as tf
+
+from pycarbon.reader import make_reader
+from pycarbon.reader import make_tensor, make_dataset
+
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+
+def tensorflow_hello_world(dataset_url='file:///tmp/carbon_pycarbon_dataset/'):
+  # Example: tf_tensors will return tensors with dataset data
+  with make_reader(dataset_url, is_batch=False) as reader:
+    tensor = make_tensor(reader)
+    with tf.Session() as sess:
+      sample = sess.run(tensor)
+      print(sample.id)
+
+  with make_reader(dataset_url, is_batch=False) as reader:
+    dataset = make_dataset(reader)
+    iterator = dataset.make_one_shot_iterator()
+    tensor = iterator.get_next()
+    with tf.Session() as sess:
+      sample = sess.run(tensor)
+      print(sample.id)
+
+
+if __name__ == '__main__':
+  parser = argparse.ArgumentParser(description='TensorFlow hello world')
+  parser.add_argument('-c', '--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  tensorflow_hello_world()
diff --git a/python/pycarbon/tests/hello_world/dataset_with_unischema/tests/test_generate_dataset.py b/python/pycarbon/tests/hello_world/dataset_with_unischema/tests/test_generate_dataset.py
new file mode 100644
index 0000000..080454d
--- /dev/null
+++ b/python/pycarbon/tests/hello_world/dataset_with_unischema/tests/test_generate_dataset.py
@@ -0,0 +1,82 @@
+# 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.
+
+
+import os
+
+import pytest
+
+from pycarbon.tests.hello_world.dataset_with_unischema.generate_pycarbon_dataset import generate_pycarbon_dataset
+from pycarbon.tests.hello_world.dataset_with_unischema.pyspark_hello_world_carbon import pyspark_hello_world
+from pycarbon.tests.hello_world.dataset_with_unischema.python_hello_world_carbon import python_hello_world
+from pycarbon.tests.hello_world.dataset_with_unischema.tensorflow_hello_world_carbon import tensorflow_hello_world
+from petastorm.tests.conftest import SyntheticDataset
+
+from pycarbon.core.Constants import LOCAL_FILE_PREFIX
+
+from pycarbon.reader import make_reader
+
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line --pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH, "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+
+@pytest.fixture(scope="session")
+def pycarbon_dataset(tmpdir_factory):
+  path = tmpdir_factory.mktemp("data").strpath
+  url = LOCAL_FILE_PREFIX + path
+
+  generate_pycarbon_dataset(url)
+
+  dataset = SyntheticDataset(url=url, path=path, data=None)
+
+  # Generate a dataset
+  assert os.path.exists(os.path.join(path, '_SUCCESS'))
+
+  return dataset
+
+
+def test_generate(pycarbon_dataset):
+  # Read from it using a plain reader
+  with make_reader(pycarbon_dataset.url) as reader:
+    all_samples = list(reader)
+  assert all_samples
+
+  with make_reader(pycarbon_dataset.url, is_batch=False) as reader:
+    all_samples = list(reader)
+  assert all_samples
+
+
+def test_pyspark_hello_world_pycarbon_dataset_example(pycarbon_dataset):
+  pyspark_hello_world(pycarbon_dataset.url)
+
+
+def test_python_hello_world_pycarbon_dataset_example(pycarbon_dataset):
+  python_hello_world(pycarbon_dataset.url)
+
+
+def test_tensorflow_hello_world_pycarbon_dataset_example(pycarbon_dataset):
+  tensorflow_hello_world(pycarbon_dataset.url)
diff --git a/python/pycarbon/tests/im/__init__.py b/python/pycarbon/tests/im/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/tests/im/__init__.py
diff --git a/python/pycarbon/tests/im/test.py b/python/pycarbon/tests/im/test.py
new file mode 100644
index 0000000..aff3d81
--- /dev/null
+++ b/python/pycarbon/tests/im/test.py
@@ -0,0 +1,3 @@
+
+def print_string(str):
+  print(str)
\ No newline at end of file
diff --git a/python/pycarbon/tests/mnist/README.md b/python/pycarbon/tests/mnist/README.md
new file mode 100644
index 0000000..d4a589f
--- /dev/null
+++ b/python/pycarbon/tests/mnist/README.md
@@ -0,0 +1,3 @@
+external_dataset: generating carbon dataset which has non-unischema (standard carbon schema)
+
+pycarbon_dataset: generating carbon dataset which has unischema
diff --git a/python/pycarbon/tests/mnist/__init__.py b/python/pycarbon/tests/mnist/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/tests/mnist/__init__.py
diff --git a/python/pycarbon/tests/mnist/dataset_with_normal_schema/README.md b/python/pycarbon/tests/mnist/dataset_with_normal_schema/README.md
new file mode 100644
index 0000000..c3e81c0
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_normal_schema/README.md
@@ -0,0 +1,49 @@
+# Pycarbon Tensorflow, Pytorch, Mxnet Example (carbon dataset does not have unischema)
+
+using make_batch_carbon_reader to read data
+
+## Setup
+```bash
+PYTHONPATH=~/dev/pycarbon  # replace with your pycarbon install path
+```
+
+## Generating a Pycarbon Dataset from MNIST Data
+
+This creates both a `train` and `test` carbon datasets in `/tmp/mnist_external`:
+
+```bash
+python generate_external_mnist_carbon.py
+```
+
+## TODO: Pytorch training using the Carbon MNIST Dataset
+
+
+## Tensorflow training using the Carbon MNIST Dataset
+
+This will invoke a training run using MNIST carbondata,
+for 1 epochs, using a batch size of 100, and log every 10 intervals.
+
+```bash
+python tf_external_example_carbon.py
+```
+
+```
+python tf_external_example_carbon.py -h
+usage: tf_external_example_carbon.py [-h] [--dataset-url S] [--num-epochs N]
+                     [--batch-size N] [--evaluation-interval N]
+
+Pycarbon Tensorflow MNIST Example
+
+optional arguments:
+  -h, --help            show this help message and exit
+  --dataset-url S       hdfs:// or file:/// URL to the MNIST pycarbon
+                        dataset(default: file:///tmp/mnist_external)
+  --num-epochs N
+                        number of epochs to train (default: 1)
+  --batch-size N        input batch size for training (default: 100)
+  --evaluation-interval N
+                        how many batches to wait before evaluating the model
+                        accuracy (default: 10)
+```
+
+## TODO: Mxnet training using the Carbon MNIST Dataset
diff --git a/python/pycarbon/tests/mnist/dataset_with_normal_schema/__init__.py b/python/pycarbon/tests/mnist/dataset_with_normal_schema/__init__.py
new file mode 100644
index 0000000..17fc008
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_normal_schema/__init__.py
@@ -0,0 +1,15 @@
+#  Copyright (c) 2017-2018 Uber Technologies, Inc.
+#
+# Licensed 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.
+
+DEFAULT_MNIST_DATA_PATH = '/tmp/mnist_external'
diff --git a/python/pycarbon/tests/mnist/dataset_with_normal_schema/generate_mnist_carbon.py b/python/pycarbon/tests/mnist/dataset_with_normal_schema/generate_mnist_carbon.py
new file mode 100644
index 0000000..406f876
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_normal_schema/generate_mnist_carbon.py
@@ -0,0 +1,124 @@
+# 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.
+
+
+from __future__ import absolute_import
+from __future__ import division
+from __future__ import print_function
+
+import argparse
+import os
+import shutil
+import tempfile
+import jnius_config
+from six.moves.urllib.parse import urlparse
+
+import tensorflow as tf
+
+from tensorflow.contrib.learn.python.learn.datasets import mnist
+
+from pycarbon.tests.mnist.dataset_with_normal_schema import DEFAULT_MNIST_DATA_PATH
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+from pyspark.sql import SparkSession, Row
+
+
+def _arg_parser():
+  parser = argparse.ArgumentParser(description=__doc__, add_help=True,
+                                   formatter_class=argparse.ArgumentDefaultsHelpFormatter)
+  parser.add_argument('-d', '--download-dir', type=str, required=False, default=None,
+                      help='Directory to where the MNIST data will be downloaded; '
+                           'default to a tempdir that gets wiped after generation.')
+  parser.add_argument('-v', '--validation-size', type=int, required=False, default=0,
+                      help='validation_size, default is 0 ')
+  parser.add_argument('-o', '--output-url', type=str, required=False,
+                      default='file://{}'.format(DEFAULT_MNIST_DATA_PATH),
+                      help='hdfs://... or file:/// url where the carbon dataset will be written to.')
+  parser.add_argument('-m', '--master', type=str, required=False, default='local[*]',
+                      help='Spark master; default is local[*] to run locally.')
+  parser.add_argument('-pp', '--pyspark-python', type=str, default=None,
+                      help='pyspark python env variable')
+  parser.add_argument('-pdp', '--pyspark-driver-python', type=str, default=None,
+                      help='pyspark driver python env variable')
+  parser.add_argument('-c', '--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  return parser
+
+
+def main(download_dir, validation_size, output_url, carbon_files_count=1):
+  # Get the data
+  data_sets = mnist.read_data_sets(download_dir,
+                                   dtype=tf.uint8,
+                                   reshape=False,
+                                   validation_size=validation_size,
+                                   source_url="http://yann.lecun.com/exdb/mnist/")
+  url_path = urlparse(output_url)
+  if os.path.exists(url_path.path):
+    shutil.rmtree(url_path.path)
+
+  spark = SparkSession.builder \
+    .master('local') \
+    .getOrCreate()
+
+  mnist_data = {
+    'train': data_sets.train,
+    'test': data_sets.test
+  }
+
+  for dset, data in mnist_data.items():
+    output_dir = os.path.join(output_url, dset)
+
+    images = data.images
+    labels = data.labels
+    num_examples = data.num_examples
+
+    if images.shape[0] != num_examples:
+      raise ValueError('Image size %d does not match label size %d.' %
+                       (images.shape[0], num_examples))
+
+    rows = map(lambda x: Row(idx=x, digit=(int(labels[x])), image=bytearray(images[x].tobytes())), range(num_examples))
+
+    spark.createDataFrame(rows) \
+      .coalesce(carbon_files_count) \
+      .write \
+      .save(path=output_dir, format='carbon')
+
+
+if __name__ == '__main__':
+  args = _arg_parser().parse_args()
+  if args.download_dir is None:
+    # Make a temp dir that we'll clean up afterward
+    download_dir = tempfile.mkdtemp()
+  else:
+    download_dir = args.download_dir
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  if args.pyspark_python is not None and args.pyspark_driver_python is not None:
+    os.environ['PYSPARK_PYTHON'] = args.pyspark_python
+    os.environ['PYSPARK_DRIVER_PYTHON'] = args.pyspark_driver_python
+  elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+    pass
+  else:
+    raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                     "using cmd line -pp PYSPARK_PYTHON_PATH -pdp PYSPARK_DRIVER_PYTHON_PATH, "
+                     "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+  main(download_dir=download_dir, validation_size=args.validation_size, output_url=args.output_url)
+
+  if args.download_dir is None:
+    if os.path.exists(download_dir):
+      shutil.rmtree(download_dir)
diff --git a/python/pycarbon/tests/mnist/dataset_with_normal_schema/tf_carbon.py b/python/pycarbon/tests/mnist/dataset_with_normal_schema/tf_carbon.py
new file mode 100644
index 0000000..5bded64
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_normal_schema/tf_carbon.py
@@ -0,0 +1,156 @@
+# 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.
+
+
+from __future__ import division, print_function
+
+import argparse
+import os
+import time
+
+import jnius_config
+import tensorflow as tf
+
+from pycarbon.reader import make_reader, make_dataset
+
+from pycarbon.tests.mnist.dataset_with_normal_schema import DEFAULT_MNIST_DATA_PATH
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+
+def decode(carbon_record):
+  """Parses an image and label from the given `carbon_reader`."""
+
+  image_raw = getattr(carbon_record, 'image')
+  image = tf.decode_raw(image_raw, tf.uint8)
+
+  label_raw = getattr(carbon_record, 'digit')
+  label = tf.cast(label_raw, tf.int64)
+
+  return label, image
+
+
+def train_and_test(dataset_url, num_epochs, batch_size, evaluation_interval):
+  """
+  Train a model for training iterations with a batch size batch_size, printing accuracy every log_interval.
+  :param dataset_url: The MNIST dataset url.
+  :param num_epochs: The number of epochs to train for.
+  :param batch_size: The batch size for training.
+  :param evaluation_interval: The interval used to print the accuracy.
+  :return:
+  """
+
+  with make_reader(os.path.join(dataset_url, 'train'), num_epochs=num_epochs) as train_reader:
+    with make_reader(os.path.join(dataset_url, 'test'), num_epochs=num_epochs) as test_reader:
+      # Create the model
+      x = tf.placeholder(tf.float32, [None, 784])
+      w = tf.Variable(tf.zeros([784, 10]))
+      b = tf.Variable(tf.zeros([10]))
+      y = tf.matmul(x, w) + b
+
+      # Define loss and optimizer
+      y_ = tf.placeholder(tf.int64, [None])
+
+      # Define the loss function
+      cross_entropy = tf.losses.sparse_softmax_cross_entropy(labels=y_, logits=y)
+
+      train_step = tf.train.GradientDescentOptimizer(0.5).minimize(cross_entropy)
+
+      correct_prediction = tf.equal(tf.argmax(y, 1), y_)
+
+      accuracy = tf.reduce_mean(tf.cast(correct_prediction, tf.float32))
+
+      train_dataset = make_dataset(train_reader) \
+        .apply(tf.data.experimental.unbatch()) \
+        .batch(batch_size) \
+        .map(decode)
+
+      train_iterator = train_dataset.make_one_shot_iterator()
+      label, image = train_iterator.get_next()
+
+      test_dataset = make_dataset(test_reader) \
+        .apply(tf.data.experimental.unbatch()) \
+        .batch(batch_size) \
+        .map(decode)
+
+      test_iterator = test_dataset.make_one_shot_iterator()
+      test_label, test_image = test_iterator.get_next()
+
+      # Train
+      print('Training model for {0} epoch with batch size {1} and evaluation interval {2}'.format(
+        num_epochs, batch_size, evaluation_interval
+      ))
+
+      i = 0
+      with tf.Session() as sess:
+        sess.run([
+          tf.local_variables_initializer(),
+          tf.global_variables_initializer(),
+        ])
+
+        try:
+          while True:
+            cur_label, cur_image = sess.run([label, image])
+
+            sess.run([train_step], feed_dict={x: cur_image, y_:cur_label})
+
+            if i % evaluation_interval == 0:
+              test_cur_label, test_cur_image = sess.run([test_label, test_image])
+              print('After {0} training iterations, the accuracy of the model is: {1:.2f}'.format(
+                i,
+                sess.run(accuracy, feed_dict={
+                  x: test_cur_image, y_: test_cur_label})))
+            i += 1
+
+        except tf.errors.OutOfRangeError:
+          print("Finish! the number is " + str(i))
+
+
+def main():
+  print("Start")
+  start = time.time()
+
+  # Training settings
+  parser = argparse.ArgumentParser(description='Pycarbon Tensorflow External MNIST Example')
+  default_dataset_url = 'file://{}'.format(DEFAULT_MNIST_DATA_PATH)
+  parser.add_argument('--dataset-url', type=str,
+                      default=default_dataset_url, metavar='S',
+                      help='hdfs:// or file:/// URL to the MNIST pycarbon dataset'
+                           '(default: %s)' % default_dataset_url)
+  parser.add_argument('--num-epochs', type=int, default=1, metavar='N',
+                      help='number of epochs to train (default: 1)')
+  parser.add_argument('--batch-size', type=int, default=100, metavar='N',
+                      help='input batch size for training (default: 100)')
+  parser.add_argument('--evaluation-interval', type=int, default=10, metavar='N',
+                      help='how many batches to wait before evaluating the model accuracy (default: 10)')
+  parser.add_argument('--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  train_and_test(
+    dataset_url=args.dataset_url,
+    num_epochs=args.num_epochs,
+    batch_size=args.batch_size,
+    evaluation_interval=args.evaluation_interval
+  )
+  end = time.time()
+  print("all time: " + str(end - start))
+  print("Finish")
+
+
+if __name__ == '__main__':
+  main()
diff --git a/python/pycarbon/tests/mnist/dataset_with_normal_schema/tf_external_example_carbon_unified_api.py b/python/pycarbon/tests/mnist/dataset_with_normal_schema/tf_external_example_carbon_unified_api.py
new file mode 100644
index 0000000..8260c02
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_normal_schema/tf_external_example_carbon_unified_api.py
@@ -0,0 +1,157 @@
+# 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.
+
+
+from __future__ import division, print_function
+
+import argparse
+import os
+import time
+
+import jnius_config
+import tensorflow as tf
+
+from pycarbon.reader import make_reader
+from pycarbon.reader import make_dataset
+
+from pycarbon.tests.mnist.dataset_with_normal_schema import DEFAULT_MNIST_DATA_PATH
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+
+def decode(carbon_record):
+  """Parses an image and label from the given `carbon_reader`."""
+
+  image_raw = getattr(carbon_record, 'image')
+  image = tf.decode_raw(image_raw, tf.uint8)
+
+  label_raw = getattr(carbon_record, 'digit')
+  label = tf.cast(label_raw, tf.int64)
+
+  return label, image
+
+
+def train_and_test(dataset_url, num_epochs, batch_size, evaluation_interval):
+  """
+  Train a model for training iterations with a batch size batch_size, printing accuracy every log_interval.
+  :param dataset_url: The MNIST dataset url.
+  :param num_epochs: The number of epochs to train for.
+  :param batch_size: The batch size for training.
+  :param evaluation_interval: The interval used to print the accuracy.
+  :return:
+  """
+
+  with make_reader(os.path.join(dataset_url, 'train'), num_epochs=num_epochs) as train_reader:
+    with make_reader(os.path.join(dataset_url, 'test'), num_epochs=num_epochs) as test_reader:
+      # Create the model
+      x = tf.placeholder(tf.float32, [None, 784])
+      w = tf.Variable(tf.zeros([784, 10]))
+      b = tf.Variable(tf.zeros([10]))
+      y = tf.matmul(x, w) + b
+
+      # Define loss and optimizer
+      y_ = tf.placeholder(tf.int64, [None])
+
+      # Define the loss function
+      cross_entropy = tf.losses.sparse_softmax_cross_entropy(labels=y_, logits=y)
+
+      train_step = tf.train.GradientDescentOptimizer(0.5).minimize(cross_entropy)
+
+      correct_prediction = tf.equal(tf.argmax(y, 1), y_)
+
+      accuracy = tf.reduce_mean(tf.cast(correct_prediction, tf.float32))
+
+      train_dataset = make_dataset(train_reader) \
+        .apply(tf.data.experimental.unbatch()) \
+        .batch(batch_size) \
+        .map(decode)
+
+      train_iterator = train_dataset.make_one_shot_iterator()
+      label, image = train_iterator.get_next()
+
+      test_dataset = make_dataset(test_reader) \
+        .apply(tf.data.experimental.unbatch()) \
+        .batch(batch_size) \
+        .map(decode)
+
+      test_iterator = test_dataset.make_one_shot_iterator()
+      test_label, test_image = test_iterator.get_next()
+
+      # Train
+      print('Training model for {0} epoch with batch size {1} and evaluation interval {2}'.format(
+        num_epochs, batch_size, evaluation_interval
+      ))
+
+      i = 0
+      with tf.Session() as sess:
+        sess.run([
+          tf.local_variables_initializer(),
+          tf.global_variables_initializer(),
+        ])
+
+        try:
+          while True:
+            cur_label, cur_image = sess.run([label, image])
+
+            sess.run([train_step], feed_dict={x: cur_image, y_:cur_label})
+
+            if i % evaluation_interval == 0:
+              test_cur_label, test_cur_image = sess.run([test_label, test_image])
+              print('After {0} training iterations, the accuracy of the model is: {1:.2f}'.format(
+                i,
+                sess.run(accuracy, feed_dict={
+                  x: test_cur_image, y_: test_cur_label})))
+            i += 1
+
+        except tf.errors.OutOfRangeError:
+          print("Finish! the number is " + str(i))
+
+
+def main():
+  print("Start")
+  start = time.time()
+
+  # Training settings
+  parser = argparse.ArgumentParser(description='Pycarbon Tensorflow External MNIST Example')
+  default_dataset_url = 'file://{}'.format(DEFAULT_MNIST_DATA_PATH)
+  parser.add_argument('--dataset-url', type=str,
+                      default=default_dataset_url, metavar='S',
+                      help='hdfs:// or file:/// URL to the MNIST pycarbon dataset'
+                           '(default: %s)' % default_dataset_url)
+  parser.add_argument('--num-epochs', type=int, default=1, metavar='N',
+                      help='number of epochs to train (default: 1)')
+  parser.add_argument('--batch-size', type=int, default=100, metavar='N',
+                      help='input batch size for training (default: 100)')
+  parser.add_argument('--evaluation-interval', type=int, default=10, metavar='N',
+                      help='how many batches to wait before evaluating the model accuracy (default: 10)')
+  parser.add_argument('--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  train_and_test(
+    dataset_url=args.dataset_url,
+    num_epochs=args.num_epochs,
+    batch_size=args.batch_size,
+    evaluation_interval=args.evaluation_interval
+  )
+  end = time.time()
+  print("all time: " + str(end - start))
+  print("Finish")
+
+
+if __name__ == '__main__':
+  main()
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/README.md b/python/pycarbon/tests/mnist/dataset_with_unischema/README.md
new file mode 100644
index 0000000..451dace
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/README.md
@@ -0,0 +1,103 @@
+# Pycarbon Tensorflow, Pytorch, Mxnet Example (carbon dataset has unischema)
+
+using make_carbon_reader to read data
+
+## Setup
+```bash
+cd python/
+PYTHONPATH=/yourpath/carbondata/python/
+pip install . --user
+```
+
+## Generating a Pycarbon Dataset from MNIST Data
+
+This creates both a `train` and `test` pycarbon datasets in `/tmp/mnist`:
+
+```bash
+cd pycarbon/tests/mnist/dataset_with_unischema
+python generate_pycarbon_mnist.py 
+```
+
+## Pytorch training using the PyCarbon MNIST Dataset
+
+This will invoke a 10-epoch training run using MNIST data in pycarbon form,
+stored by default in `/tmp/mnist`, and show accuracy against the test set:
+
+```bash
+python pytorch_example_carbon.py
+```
+
+```
+usage: pytorch_example_carbon.py [-h] [--dataset-url S] [--batch-size N] [--test-batch-size N]
+               [--epochs N] [--all-epochs] [--lr LR] [--momentum M]
+               [--no-cuda] [--seed S] [--log-interval N]
+
+Pycarbon Pytorch MNIST Example
+
+optional arguments:
+  -h, --help           show this help message and exit
+  --dataset-url S      hdfs:// or file:/// URL to the MNIST pycarbon dataset
+                       (default: file:///tmp/mnist)
+  --batch-size N       input batch size for training (default: 64)
+  --test-batch-size N  input batch size for testing (default: 1000)
+  --epochs N           number of epochs to train (default: 10)
+  --all-epochs         train all epochs before testing accuracy/loss
+  --lr LR              learning rate (default: 0.01)
+  --momentum M         SGD momentum (default: 0.5)
+  --no-cuda            disables CUDA training
+  --seed S             random seed (default: 1)
+  --log-interval N     how many batches to wait before logging training status
+```
+
+## Tensorflow training using the Pycarboned MNIST Dataset
+
+This will invoke a training run using MNIST data in pycarbon form,
+for 100 epochs, using a batch size of 100, and log every 10 intervals.
+
+```bash
+python tf_example_carbon_unified_api.py
+```
+
+```
+python tf_example_carbon_unified_api.py -h
+usage: tf_example_carbon_unified_api.py [-h] [--dataset-url S] [--training-iterations N]
+                     [--batch-size N] [--evaluation-interval N]
+
+Pycarbon Tensorflow MNIST Example
+
+optional arguments:
+  -h, --help            show this help message and exit
+  --dataset-url S       hdfs:// or file:/// URL to the MNIST pycarbon
+                        dataset(default: file:///tmp/mnist)
+  --training-iterations N
+                        number of training iterations to train (default: 100)
+  --batch-size N        input batch size for training (default: 100)
+  --evaluation-interval N
+                        how many batches to wait before evaluating the model
+                        accuracy (default: 10)
+```
+
+## Mxnet training using the Pycarboned MNIST Dataset
+
+This will invoke a training run using MNIST data in pycarbon form,
+for 1 epoch, using a batch size of 100.
+
+```bash
+python mxnet_example_carbon.py
+```
+
+```
+python mxnet_example_carbon.py -h
+usage: mxnet_example_carbon.py [-h] [--dataset-url S] [--num-epoch N]
+                     [--batch-size N]
+
+Pycarbon Mxnet MNIST Example
+
+optional arguments:
+  -h, --help            show this help message and exit
+  --dataset-url S       hdfs:// or file:/// URL to the MNIST pycarbon
+                        dataset(default: file:///tmp/mnist)
+  --num-epoch N
+                        the number of epoch for training (default: 1)
+  --batch-size N        input batch size for training (default: 100)
+```
\ No newline at end of file
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/__init__.py b/python/pycarbon/tests/mnist/dataset_with_unischema/__init__.py
new file mode 100644
index 0000000..9d7faeb
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/__init__.py
@@ -0,0 +1,15 @@
+#  Copyright (c) 2017-2018 Uber Technologies, Inc.
+#
+# Licensed 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.
+
+DEFAULT_MNIST_DATA_PATH = '/tmp/mnist'
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/generate_pycarbon_mnist.py b/python/pycarbon/tests/mnist/dataset_with_unischema/generate_pycarbon_mnist.py
new file mode 100644
index 0000000..a7f4e9e
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/generate_pycarbon_mnist.py
@@ -0,0 +1,172 @@
+# 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.
+
+
+"""
+This utility converts the MNIST standard dataset (http://yann.lecun.com/exdb/mnist/) into
+a pycarbon dataset (Carbon format).  The resulting dataset can then be used by main.py
+to demonstrate pycarbon usage with pytorch.
+
+The script can run locally (use '--master=local[*]' command line argument), or submitted to a spark cluster.
+
+Schema defined in examples.mnist.schema.MnistSchema will be used.
+
+NOTE: MNIST train and test data will be downloaded automatically.
+"""
+
+import argparse
+
+import jnius_config
+
+import numpy as np
+import os
+import shutil
+import tempfile
+from six.moves.urllib.parse import urlparse
+
+from pyspark.sql import SparkSession
+
+from pycarbon.tests.mnist.dataset_with_unischema import DEFAULT_MNIST_DATA_PATH
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+from pycarbon.tests.mnist.dataset_with_unischema.schema import MnistSchema
+from petastorm.unischema import dict_to_spark_row
+
+from pycarbon.core.carbon_dataset_metadata import materialize_dataset_carbon
+
+
+def _arg_parser():
+  parser = argparse.ArgumentParser(description=__doc__, add_help=True,
+                                   formatter_class=argparse.ArgumentDefaultsHelpFormatter)
+  parser.add_argument('-d', '--download-dir', type=str, required=False, default=None,
+                      help='Directory to where the MNIST data will be downloaded; '
+                           'default to a tempdir that gets wiped after generation.')
+  parser.add_argument('-o', '--output-url', type=str, required=False,
+                      default='file://{}'.format(DEFAULT_MNIST_DATA_PATH),
+                      help='hdfs://... or file:/// url where the carbon dataset will be written to.')
+  parser.add_argument('-m', '--master', type=str, required=False, default='local[*]',
+                      help='Spark master; default is local[*] to run locally.')
+  parser.add_argument('-pp', '--pyspark-python', type=str, default=None,
+                      help='pyspark python env variable')
+  parser.add_argument('-pdp', '--pyspark-driver-python', type=str, default=None,
+                      help='pyspark driver python env variable')
+  parser.add_argument('-c', '--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  return parser
+
+
+def download_mnist_data(download_dir, train=True):
+  """
+  Downloads the dataset files and returns the torch Dataset object, which
+  represents the data as an array of (img, label) pairs.
+
+  Each image is a PIL.Image of black-and-white 28x28 pixels.
+  Each label is a long integer representing the digit 0..9.
+  """
+  # This is the only function requiring torch in this module.
+
+  from torchvision import datasets
+
+  return datasets.MNIST('{}/{}'.format(download_dir, 'data'), train=train, download=True)
+
+
+def mnist_data_to_pycarbon_dataset(download_dir, output_url, spark_master=None, carbon_files_count=1,
+                                   mnist_data=None):
+  """Converts a directory with MNIST data into a pycarbon dataset.
+
+  Data files are as specified in http://yann.lecun.com/exdb/mnist/:
+      * train-images-idx3-ubyte.gz:  training set images (9912422 bytes)
+      * train-labels-idx1-ubyte.gz:  training set labels (28881 bytes)
+      * t10k-images-idx3-ubyte.gz:   test set images (1648877 bytes)
+      * t10k-labels-idx1-ubyte.gz:   test set labels (4542 bytes)
+
+  The images and labels and stored in the IDX file format for vectors and multidimensional matrices of
+  various numerical types, as defined in the same URL.
+
+  :param download_dir: the path to where the MNIST data will be downloaded.
+  :param output_url: the location where your dataset will be written to. Should be a url: either
+    file://... or hdfs://...
+  :param spark_master: A master parameter used by spark session builder. Use default value (None) to use system
+    environment configured spark cluster. Use 'local[*]' to run on a local box.
+  :param mnist_data: A dictionary of MNIST data, with name of dataset as key, and the dataset object as value;
+    if None is suplied, download it.
+  :return: None
+  """
+  session_builder = SparkSession \
+    .builder \
+    .appName('MNIST Dataset Creation')
+  if spark_master:
+    session_builder.master(spark_master)
+
+  spark = session_builder.getOrCreate()
+
+  # Get training and test data
+  if mnist_data is None:
+    mnist_data = {
+      'train': download_mnist_data(download_dir, train=True),
+      'test': download_mnist_data(download_dir, train=False)
+    }
+
+  url_path = urlparse(output_url)
+  if os.path.exists(url_path.path):
+    shutil.rmtree(url_path.path)
+
+  # The MNIST data is small enough to do everything here in Python
+  for dset, data in mnist_data.items():
+    dset_output_url = '{}/{}'.format(output_url, dset)
+    with materialize_dataset_carbon(spark, dset_output_url, MnistSchema):
+      # List of [(idx, image, digit), ...]
+      # where image is shaped as a 28x28 numpy matrix
+      idx_image_digit_list = map(lambda idx_image_digit: {
+        MnistSchema.idx.name: idx_image_digit[0],
+        MnistSchema.digit.name: idx_image_digit[1][1],
+        MnistSchema.image.name: np.array(list(idx_image_digit[1][0].getdata()), dtype=np.uint8).reshape(28, 28)
+      }, enumerate(data))
+
+      # Convert to pyspark.sql.Row
+      sql_rows = map(lambda r: dict_to_spark_row(MnistSchema, r), idx_image_digit_list)
+
+      # Write out the result
+      spark.createDataFrame(sql_rows, MnistSchema.as_spark_schema()) \
+        .coalesce(carbon_files_count) \
+        .write \
+        .save(path=dset_output_url, format='carbon')
+
+
+if __name__ == '__main__':
+  args = _arg_parser().parse_args()
+  if args.download_dir is None:
+    # Make a temp dir that we'll clean up afterward
+    download_dir = tempfile.mkdtemp()
+  else:
+    download_dir = args.download_dir
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  if args.pyspark_python is not None and args.pyspark_driver_python is not None:
+    os.environ['PYSPARK_PYTHON'] = args.pyspark_python
+    os.environ['PYSPARK_DRIVER_PYTHON'] = args.pyspark_driver_python
+  elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+    pass
+  else:
+    raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                     "using cmd line -pp PYSPARK_PYTHON_PATH -pdp PYSPARK_DRIVER_PYTHON_PATH, "
+                     "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+  mnist_data_to_pycarbon_dataset(download_dir, args.output_url)
+
+  if args.download_dir is None:
+    if os.path.exists(download_dir):
+      shutil.rmtree(download_dir)
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/schema.py b/python/pycarbon/tests/mnist/dataset_with_unischema/schema.py
new file mode 100644
index 0000000..493af97
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/schema.py
@@ -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.
+
+
+import numpy as np
+from pyspark.sql.types import IntegerType
+
+from petastorm.codecs import ScalarCodec, NdarrayCodec
+from petastorm.unischema import Unischema, UnischemaField
+
+MnistSchema = Unischema('MnistSchema', [
+  UnischemaField('idx', np.int_, (), ScalarCodec(IntegerType()), False),
+  UnischemaField('digit', np.int_, (), ScalarCodec(IntegerType()), False),
+  UnischemaField('image', np.uint8, (28, 28), NdarrayCodec(), False),
+])
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/tests/__init__.py b/python/pycarbon/tests/mnist/dataset_with_unischema/tests/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/tests/__init__.py
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/tests/conftest.py b/python/pycarbon/tests/mnist/dataset_with_unischema/tests/conftest.py
new file mode 100644
index 0000000..ca4eb32
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/tests/conftest.py
@@ -0,0 +1,70 @@
+# 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.
+
+
+import numpy as np
+import pytest
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+
+MOCK_IMAGE_SIZE = (28, 28)
+MOCK_IMAGE_3DIM_SIZE = (28, 28, 1)
+SMALL_MOCK_IMAGE_COUNT = {
+  'train': 30,
+  'test': 5
+}
+LARGE_MOCK_IMAGE_COUNT = {
+  'train': 600,
+  'test': 100
+}
+
+class MockDataObj(object):
+  """ Wraps a mock image array and provide a needed getdata() interface function. """
+
+  def __init__(self, a):
+    self.a = a
+
+  def getdata(self):
+    return self.a
+
+
+def _mock_mnist_data(mock_spec):
+  """
+  Creates a mock data dictionary with train and test sets, each containing 5 mock pairs:
+
+      ``(random images, random digit)``.
+  """
+  bogus_data = {
+    'train': [],
+    'test': []
+  }
+
+  for dset, data in bogus_data.items():
+    for _ in range(mock_spec[dset]):
+      pair = (MockDataObj(np.random.randint(0, 255, size=MOCK_IMAGE_SIZE, dtype=np.uint8)),
+              np.random.randint(0, 9))
+      data.append(pair)
+
+  return bogus_data
+
+
+@pytest.fixture(scope="session")
+def small_mock_mnist_data():
+  return _mock_mnist_data(SMALL_MOCK_IMAGE_COUNT)
+
+
+@pytest.fixture(scope="session")
+def large_mock_mnist_data():
+  return _mock_mnist_data(LARGE_MOCK_IMAGE_COUNT)
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/tests/test_tf_mnist_carbon.py b/python/pycarbon/tests/mnist/dataset_with_unischema/tests/test_tf_mnist_carbon.py
new file mode 100644
index 0000000..472b324
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/tests/test_tf_mnist_carbon.py
@@ -0,0 +1,72 @@
+# 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.
+
+
+import os
+import time
+from pycarbon.tests.mnist.dataset_with_unischema import tf_example_carbon as tf_example
+from pycarbon.tests.mnist.dataset_with_unischema import tf_example_carbon_unified_api as tf_example_unified
+from pycarbon.tests.mnist.dataset_with_unischema.generate_pycarbon_mnist import mnist_data_to_pycarbon_dataset
+
+import pytest
+import jnius_config
+
+jnius_config.set_classpath(pytest.config.getoption("--carbon-sdk-path"))
+
+if pytest.config.getoption("--pyspark-python") is not None and \
+    pytest.config.getoption("--pyspark-driver-python") is not None:
+  os.environ['PYSPARK_PYTHON'] = pytest.config.getoption("--pyspark-python")
+  os.environ['PYSPARK_DRIVER_PYTHON'] = pytest.config.getoption("--pyspark-driver-python")
+elif 'PYSPARK_PYTHON' in os.environ.keys() and 'PYSPARK_DRIVER_PYTHON' in os.environ.keys():
+  pass
+else:
+  raise ValueError("please set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON variables, "
+                   "using cmd line "
+                   "--pyspark-python=PYSPARK_PYTHON_PATH --pyspark-driver-python=PYSPARK_DRIVER_PYTHON_PATH "
+                   "or set PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON in system env")
+
+
+def test_full_tf_example(large_mock_mnist_data, tmpdir):
+  # First, generate mock dataset
+  dataset_url = 'file://{}'.format(tmpdir)
+  mnist_data_to_pycarbon_dataset(tmpdir, dataset_url, mnist_data=large_mock_mnist_data,
+                                 spark_master='local[1]', carbon_files_count=1)
+
+  start = time.time()
+  # Tensorflow train and test
+  tf_example.train_and_test(
+    dataset_url=dataset_url,
+    training_iterations=10,
+    batch_size=10,
+    evaluation_interval=10,
+    start=start
+  )
+
+
+def test_full_tf_example_unifeid(large_mock_mnist_data, tmpdir):
+  # First, generate mock dataset
+  dataset_url = 'file://{}'.format(tmpdir)
+  mnist_data_to_pycarbon_dataset(tmpdir, dataset_url, mnist_data=large_mock_mnist_data,
+                                 spark_master='local[1]', carbon_files_count=1)
+
+  start = time.time()
+  # Tensorflow train and test
+  tf_example_unified.train_and_test(
+    dataset_url=dataset_url,
+    training_iterations=10,
+    batch_size=10,
+    evaluation_interval=10,
+    start=start
+  )
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/tf_example_carbon.py b/python/pycarbon/tests/mnist/dataset_with_unischema/tf_example_carbon.py
new file mode 100644
index 0000000..853af76
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/tf_example_carbon.py
@@ -0,0 +1,150 @@
+# 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.
+
+
+###
+# Adapted to pycarbon dataset using original contents from
+# https://github.com/tensorflow/tensorflow/blob/master/tensorflow/examples/tutorials/mnist/mnist_softmax.py
+###
+
+from __future__ import division, print_function
+
+import argparse
+import os
+import time
+
+import jnius_config
+
+import tensorflow as tf
+
+from pycarbon.reader import make_reader
+from pycarbon.tests.mnist.dataset_with_unischema import DEFAULT_MNIST_DATA_PATH
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+from pycarbon.core.carbon_tf_utils import tf_tensors
+
+def train_and_test(dataset_url, training_iterations, batch_size, evaluation_interval, start):
+  """
+  Train a model for training iterations with a batch size batch_size, printing accuracy every log_interval.
+  :param dataset_url: The MNIST dataset url.
+  :param training_iterations: The training iterations to train for.
+  :param batch_size: The batch size for training.
+  :param evaluation_interval: The interval used to print the accuracy.
+  :return:
+  """
+  with make_reader(os.path.join(dataset_url, 'train'), num_epochs=None, is_batch=False) as train_reader:
+    with make_reader(os.path.join(dataset_url, 'test'), num_epochs=None, is_batch=False) as test_reader:
+      train_readout = tf_tensors(train_reader)
+      train_image = tf.cast(tf.reshape(train_readout.image, [784]), tf.float32)
+      train_label = train_readout.digit
+      batch_image, batch_label = tf.train.batch(
+        [train_image, train_label], batch_size=batch_size
+      )
+
+      W = tf.Variable(tf.zeros([784, 10]))
+      b = tf.Variable(tf.zeros([10]))
+      y = tf.matmul(batch_image, W) + b
+
+      # The raw formulation of cross-entropy,
+      #
+      #   tf.reduce_mean(-tf.reduce_sum(y_ * tf.log(tf.nn.softmax(y)),
+      #                                 reduction_indices=[1]))
+      #
+      # can be numerically unstable.
+      #
+      # So here we use tf.losses.sparse_softmax_cross_entropy on the raw
+      # outputs of 'y', and then average across the batch.
+      cross_entropy = tf.losses.sparse_softmax_cross_entropy(labels=batch_label, logits=y)
+      train_step = tf.train.GradientDescentOptimizer(0.5).minimize(cross_entropy)
+
+      correct_prediction = tf.equal(tf.argmax(y, 1), batch_label)
+      accuracy = tf.reduce_mean(tf.cast(correct_prediction, tf.float32))
+
+      test_readout = tf_tensors(test_reader)
+      test_image = tf.cast(tf.reshape(test_readout.image, [784]), tf.float32)
+      test_label = test_readout.digit
+      test_batch_image, test_batch_label = tf.train.batch(
+        [test_image, test_label], batch_size=batch_size
+      )
+
+      end = time.time()
+      print("before train time: " + str(end - start))
+
+      # Train
+      print('Training model for {0} training iterations with batch size {1} and evaluation interval {2}'.format(
+        training_iterations, batch_size, evaluation_interval
+      ))
+      with tf.Session() as sess:
+        sess.run([
+          tf.local_variables_initializer(),
+          tf.global_variables_initializer(),
+        ])
+        coord = tf.train.Coordinator()
+        threads = tf.train.start_queue_runners(sess=sess, coord=coord)
+        try:
+          for i in range(training_iterations):
+            if coord.should_stop():
+              break
+
+            sess.run(train_step)
+
+            if (i % evaluation_interval) == 0 or i == (training_iterations - 1):
+              feed_batch_image, feed_batch_label = sess.run([test_batch_image, test_batch_label])
+              print('After {0} training iterations, the accuracy of the model is: {1:.2f}'.format(
+                i,
+                sess.run(accuracy, feed_dict={
+                  batch_image: feed_batch_image, batch_label: feed_batch_label
+                })))
+        finally:
+          coord.request_stop()
+          coord.join(threads)
+
+
+def main():
+  print("Start")
+  start = time.time()
+  # Training settings
+  parser = argparse.ArgumentParser(description='Pycarbon Tensorflow MNIST Example')
+  default_dataset_url = 'file://{}'.format(DEFAULT_MNIST_DATA_PATH)
+  parser.add_argument('--dataset-url', type=str,
+                      default=default_dataset_url, metavar='S',
+                      help='hdfs:// or file:/// URL to the MNIST pycarbon dataset'
+                           '(default: %s)' % default_dataset_url)
+  parser.add_argument('--training-iterations', type=int, default=100, metavar='N',
+                      help='number of training iterations to train (default: 100)')
+  parser.add_argument('--batch-size', type=int, default=100, metavar='N',
+                      help='input batch size for training (default: 100)')
+  parser.add_argument('--evaluation-interval', type=int, default=10, metavar='N',
+                      help='how many batches to wait before evaluating the model accuracy (default: 10)')
+  parser.add_argument('--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  train_and_test(
+    dataset_url=args.dataset_url,
+    training_iterations=args.training_iterations,
+    batch_size=args.batch_size,
+    evaluation_interval=args.evaluation_interval,
+    start=start
+  )
+  end = time.time()
+  print("all time: " + str(end - start))
+  print("Finish")
+
+
+if __name__ == '__main__':
+  main()
diff --git a/python/pycarbon/tests/mnist/dataset_with_unischema/tf_example_carbon_unified_api.py b/python/pycarbon/tests/mnist/dataset_with_unischema/tf_example_carbon_unified_api.py
new file mode 100644
index 0000000..1476586
--- /dev/null
+++ b/python/pycarbon/tests/mnist/dataset_with_unischema/tf_example_carbon_unified_api.py
@@ -0,0 +1,152 @@
+# 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.
+
+
+###
+# Adapted to pycarbon dataset using original contents from
+# https://github.com/tensorflow/tensorflow/blob/master/tensorflow/examples/tutorials/mnist/mnist_softmax.py
+###
+
+from __future__ import division, print_function
+
+import argparse
+import os
+import time
+
+import jnius_config
+
+import tensorflow as tf
+
+from pycarbon.tests.mnist.dataset_with_unischema import DEFAULT_MNIST_DATA_PATH
+from pycarbon.tests import DEFAULT_CARBONSDK_PATH
+
+from pycarbon.reader import make_reader
+from pycarbon.reader import make_tensor
+
+
+def train_and_test(dataset_url, training_iterations, batch_size, evaluation_interval, start):
+  """
+  Train a model for training iterations with a batch size batch_size, printing accuracy every log_interval.
+  :param dataset_url: The MNIST dataset url.
+  :param training_iterations: The training iterations to train for.
+  :param batch_size: The batch size for training.
+  :param evaluation_interval: The interval used to print the accuracy.
+  :return:
+  """
+  with make_reader(os.path.join(dataset_url, 'train'), is_batch=False, num_epochs=None) as train_reader:
+    with make_reader(os.path.join(dataset_url, 'test'), is_batch=False, num_epochs=None) as test_reader:
+      train_readout = make_tensor(train_reader)
+      train_image = tf.cast(tf.reshape(train_readout.image, [784]), tf.float32)
+      train_label = train_readout.digit
+      batch_image, batch_label = tf.train.batch(
+        [train_image, train_label], batch_size=batch_size
+      )
+
+      W = tf.Variable(tf.zeros([784, 10]))
+      b = tf.Variable(tf.zeros([10]))
+      y = tf.matmul(batch_image, W) + b
+
+      # The raw formulation of cross-entropy,
+      #
+      #   tf.reduce_mean(-tf.reduce_sum(y_ * tf.log(tf.nn.softmax(y)),
+      #                                 reduction_indices=[1]))
+      #
+      # can be numerically unstable.
+      #
+      # So here we use tf.losses.sparse_softmax_cross_entropy on the raw
+      # outputs of 'y', and then average across the batch.
+      cross_entropy = tf.losses.sparse_softmax_cross_entropy(labels=batch_label, logits=y)
+      train_step = tf.train.GradientDescentOptimizer(0.5).minimize(cross_entropy)
+
+      correct_prediction = tf.equal(tf.argmax(y, 1), batch_label)
+      accuracy = tf.reduce_mean(tf.cast(correct_prediction, tf.float32))
+
+      test_readout = make_tensor(test_reader)
+      test_image = tf.cast(tf.reshape(test_readout.image, [784]), tf.float32)
+      test_label = test_readout.digit
+      test_batch_image, test_batch_label = tf.train.batch(
+        [test_image, test_label], batch_size=batch_size
+      )
+
+      end = time.time()
+      print("before train time: " + str(end - start))
+
+      # Train
+      print('Training model for {0} training iterations with batch size {1} and evaluation interval {2}'.format(
+        training_iterations, batch_size, evaluation_interval
+      ))
+      with tf.Session() as sess:
+        sess.run([
+          tf.local_variables_initializer(),
+          tf.global_variables_initializer(),
+        ])
+        coord = tf.train.Coordinator()
+        threads = tf.train.start_queue_runners(sess=sess, coord=coord)
+        try:
+          for i in range(training_iterations):
+            if coord.should_stop():
+              break
+
+            sess.run(train_step)
+
+            if (i % evaluation_interval) == 0 or i == (training_iterations - 1):
+              feed_batch_image, feed_batch_label = sess.run([test_batch_image, test_batch_label])
+              print('After {0} training iterations, the accuracy of the model is: {1:.2f}'.format(
+                i,
+                sess.run(accuracy, feed_dict={
+                  batch_image: feed_batch_image, batch_label: feed_batch_label
+                })))
+        finally:
+          coord.request_stop()
+          coord.join(threads)
+
+
+def main():
+  print("Start")
+  start = time.time()
+  # Training settings
+  parser = argparse.ArgumentParser(description='Pycarbon Tensorflow MNIST Example')
+  default_dataset_url = 'file://{}'.format(DEFAULT_MNIST_DATA_PATH)
+  parser.add_argument('--dataset-url', type=str,
+                      default=default_dataset_url, metavar='S',
+                      help='hdfs:// or file:/// URL to the MNIST pycarbon dataset'
+                           '(default: %s)' % default_dataset_url)
+  parser.add_argument('--training-iterations', type=int, default=100, metavar='N',
+                      help='number of training iterations to train (default: 100)')
+  parser.add_argument('--batch-size', type=int, default=100, metavar='N',
+                      help='input batch size for training (default: 100)')
+  parser.add_argument('--evaluation-interval', type=int, default=10, metavar='N',
+                      help='how many batches to wait before evaluating the model accuracy (default: 10)')
+  parser.add_argument('--carbon-sdk-path', type=str, default=DEFAULT_CARBONSDK_PATH,
+                      help='carbon sdk path')
+
+  args = parser.parse_args()
+
+  jnius_config.set_classpath(args.carbon_sdk_path)
+
+  train_and_test(
+    dataset_url=args.dataset_url,
+    training_iterations=args.training_iterations,
+    batch_size=args.batch_size,
+    evaluation_interval=args.evaluation_interval,
+    start=start
+  )
+  end = time.time()
+  print("all time: " + str(end - start))
+  print("Finish")
+
+
+if __name__ == '__main__':
+  main()
diff --git a/python/pycarbon/test/resources/carbondatalogo.jpg b/python/pycarbon/tests/resources/carbondatalogo.jpg
similarity index 100%
rename from python/pycarbon/test/resources/carbondatalogo.jpg
rename to python/pycarbon/tests/resources/carbondatalogo.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/carbondatalogo2.jpg b/python/pycarbon/tests/resources/carbondatalogo2.jpg
similarity index 100%
rename from python/pycarbon/test/resources/carbondatalogo2.jpg
rename to python/pycarbon/tests/resources/carbondatalogo2.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/flowers/10686568196_b1915544a8.jpg b/python/pycarbon/tests/resources/flowers/10686568196_b1915544a8.jpg
similarity index 100%
rename from python/pycarbon/test/resources/flowers/10686568196_b1915544a8.jpg
rename to python/pycarbon/tests/resources/flowers/10686568196_b1915544a8.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/flowers/10686568196_b1915544a8.txt b/python/pycarbon/tests/resources/flowers/10686568196_b1915544a8.txt
similarity index 100%
rename from python/pycarbon/test/resources/flowers/10686568196_b1915544a8.txt
rename to python/pycarbon/tests/resources/flowers/10686568196_b1915544a8.txt
diff --git a/python/pycarbon/test/resources/flowers/10712722853_5632165b04.jpg b/python/pycarbon/tests/resources/flowers/10712722853_5632165b04.jpg
similarity index 100%
rename from python/pycarbon/test/resources/flowers/10712722853_5632165b04.jpg
rename to python/pycarbon/tests/resources/flowers/10712722853_5632165b04.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/flowers/10712722853_5632165b04.txt b/python/pycarbon/tests/resources/flowers/10712722853_5632165b04.txt
similarity index 100%
rename from python/pycarbon/test/resources/flowers/10712722853_5632165b04.txt
rename to python/pycarbon/tests/resources/flowers/10712722853_5632165b04.txt
diff --git a/python/pycarbon/test/resources/flowers/subfolder/10841136265_af473efc60.jpg b/python/pycarbon/tests/resources/flowers/subfolder/10841136265_af473efc60.jpg
similarity index 100%
rename from python/pycarbon/test/resources/flowers/subfolder/10841136265_af473efc60.jpg
rename to python/pycarbon/tests/resources/flowers/subfolder/10841136265_af473efc60.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/flowers/subfolder/10841136265_af473efc60.txt b/python/pycarbon/tests/resources/flowers/subfolder/10841136265_af473efc60.txt
similarity index 100%
rename from python/pycarbon/test/resources/flowers/subfolder/10841136265_af473efc60.txt
rename to python/pycarbon/tests/resources/flowers/subfolder/10841136265_af473efc60.txt
diff --git a/python/pycarbon/test/resources/voc/2007_000027.jpg b/python/pycarbon/tests/resources/voc/2007_000027.jpg
similarity index 100%
rename from python/pycarbon/test/resources/voc/2007_000027.jpg
rename to python/pycarbon/tests/resources/voc/2007_000027.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/voc/2007_000027.xml b/python/pycarbon/tests/resources/voc/2007_000027.xml
similarity index 100%
rename from python/pycarbon/test/resources/voc/2007_000027.xml
rename to python/pycarbon/tests/resources/voc/2007_000027.xml
diff --git a/python/pycarbon/test/resources/voc/2007_000032.jpg b/python/pycarbon/tests/resources/voc/2007_000032.jpg
similarity index 100%
rename from python/pycarbon/test/resources/voc/2007_000032.jpg
rename to python/pycarbon/tests/resources/voc/2007_000032.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/voc/2007_000032.xml b/python/pycarbon/tests/resources/voc/2007_000032.xml
similarity index 100%
rename from python/pycarbon/test/resources/voc/2007_000032.xml
rename to python/pycarbon/tests/resources/voc/2007_000032.xml
diff --git a/python/pycarbon/test/resources/voc/2007_000033.jpg b/python/pycarbon/tests/resources/voc/2007_000033.jpg
similarity index 100%
rename from python/pycarbon/test/resources/voc/2007_000033.jpg
rename to python/pycarbon/tests/resources/voc/2007_000033.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/voc/2007_000033.xml b/python/pycarbon/tests/resources/voc/2007_000033.xml
similarity index 100%
rename from python/pycarbon/test/resources/voc/2007_000033.xml
rename to python/pycarbon/tests/resources/voc/2007_000033.xml
diff --git a/python/pycarbon/test/resources/voc/2007_000039.jpg b/python/pycarbon/tests/resources/voc/2007_000039.jpg
similarity index 100%
rename from python/pycarbon/test/resources/voc/2007_000039.jpg
rename to python/pycarbon/tests/resources/voc/2007_000039.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/voc/2007_000039.xml b/python/pycarbon/tests/resources/voc/2007_000039.xml
similarity index 100%
rename from python/pycarbon/test/resources/voc/2007_000039.xml
rename to python/pycarbon/tests/resources/voc/2007_000039.xml
diff --git a/python/pycarbon/test/resources/voc/2009_001444.jpg b/python/pycarbon/tests/resources/voc/2009_001444.jpg
similarity index 100%
rename from python/pycarbon/test/resources/voc/2009_001444.jpg
rename to python/pycarbon/tests/resources/voc/2009_001444.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/voc/2009_001444.xml b/python/pycarbon/tests/resources/voc/2009_001444.xml
similarity index 100%
rename from python/pycarbon/test/resources/voc/2009_001444.xml
rename to python/pycarbon/tests/resources/voc/2009_001444.xml
diff --git a/python/pycarbon/test/resources/vocForSegmentationClass/2007_000032.jpg b/python/pycarbon/tests/resources/vocForSegmentationClass/2007_000032.jpg
similarity index 100%
rename from python/pycarbon/test/resources/vocForSegmentationClass/2007_000032.jpg
rename to python/pycarbon/tests/resources/vocForSegmentationClass/2007_000032.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/vocForSegmentationClass/2007_000032.png b/python/pycarbon/tests/resources/vocForSegmentationClass/2007_000032.png
similarity index 100%
rename from python/pycarbon/test/resources/vocForSegmentationClass/2007_000032.png
rename to python/pycarbon/tests/resources/vocForSegmentationClass/2007_000032.png
Binary files differ
diff --git a/python/pycarbon/test/resources/vocForSegmentationClass/2007_000033.jpg b/python/pycarbon/tests/resources/vocForSegmentationClass/2007_000033.jpg
similarity index 100%
rename from python/pycarbon/test/resources/vocForSegmentationClass/2007_000033.jpg
rename to python/pycarbon/tests/resources/vocForSegmentationClass/2007_000033.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/vocForSegmentationClass/2007_000033.png b/python/pycarbon/tests/resources/vocForSegmentationClass/2007_000033.png
similarity index 100%
rename from python/pycarbon/test/resources/vocForSegmentationClass/2007_000033.png
rename to python/pycarbon/tests/resources/vocForSegmentationClass/2007_000033.png
Binary files differ
diff --git a/python/pycarbon/test/resources/vocForSegmentationClass/2007_000042.jpg b/python/pycarbon/tests/resources/vocForSegmentationClass/2007_000042.jpg
similarity index 100%
rename from python/pycarbon/test/resources/vocForSegmentationClass/2007_000042.jpg
rename to python/pycarbon/tests/resources/vocForSegmentationClass/2007_000042.jpg
Binary files differ
diff --git a/python/pycarbon/test/resources/vocForSegmentationClass/2007_000042.png b/python/pycarbon/tests/resources/vocForSegmentationClass/2007_000042.png
similarity index 100%
rename from python/pycarbon/test/resources/vocForSegmentationClass/2007_000042.png
rename to python/pycarbon/tests/resources/vocForSegmentationClass/2007_000042.png
Binary files differ
diff --git a/python/pycarbon/tests/sdk/__init__.py b/python/pycarbon/tests/sdk/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/python/pycarbon/tests/sdk/__init__.py
diff --git a/python/pycarbon/test/test_read_write_carbon.py b/python/pycarbon/tests/sdk/test_read_write_carbon.py
similarity index 100%
rename from python/pycarbon/test/test_read_write_carbon.py
rename to python/pycarbon/tests/sdk/test_read_write_carbon.py
diff --git a/python/pycarbon/tests/test.py b/python/pycarbon/tests/test.py
new file mode 100644
index 0000000..52b10a4
--- /dev/null
+++ b/python/pycarbon/tests/test.py
@@ -0,0 +1,8 @@
+import sys
+sys.path.append('../../')
+
+from pycarbon.tests.im.test import print_string
+
+if __name__ == '__main__':
+    str = "hello"
+    print_string(str)
\ No newline at end of file
diff --git a/python/setup.cfg b/python/setup.cfg
new file mode 100644
index 0000000..2a9acf1
--- /dev/null
+++ b/python/setup.cfg
@@ -0,0 +1,2 @@
+[bdist_wheel]
+universal = 1
diff --git a/python/setup.py b/python/setup.py
new file mode 100644
index 0000000..dae0185
--- /dev/null
+++ b/python/setup.py
@@ -0,0 +1,111 @@
+# 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.
+
+
+
+import io
+import re
+
+import setuptools
+from setuptools import setup
+
+PACKAGE_NAME = 'pycarbon'
+
+with open('README.md') as f:
+    long_description = f.read()
+
+with io.open('__init__.py', 'rt', encoding='utf8') as f:
+    version = re.search(r'__version__ = \'(.*?)\'', f.read()).group(1)
+    if version is None:
+        raise ImportError('Could not find __version__ in __init__.py')
+
+REQUIRED_PACKAGES = [
+    'petastorm==0.7.2',
+    'dill>=0.2.1',
+    'diskcache>=3.0.0',
+    'numpy>=1.13.3',
+    'pandas>=0.19.0',
+    'psutil>=4.0.0',
+    'pyspark==2.3.2',
+    'pyzmq>=14.0.0',
+    'pyarrow==0.11.1',
+    'six>=1.5.0',
+    'torchvision>=0.2.1',
+    'tensorflow>=1.4.0',
+    'jnius>=1.1.0',
+    'pyjnius>=1.2.0',
+    'huaweicloud-sdk-python-modelarts-dataset>=0.1.1',
+    'future==0.17.1',
+    'futures>=2.0; python_version == "2.7"',
+]
+
+EXTRA_REQUIRE = {
+    # `docs` versions are to facilitate local generation of documentation.
+    # Sphinx 1.3 would be desirable, but is incompatible with current ATG setup.
+    # Thus the pinning of both sphinx and alabaster versions.
+    'docs': [
+        'sphinx==1.2.2',
+        'alabaster==0.7.11'
+    ],
+    'opencv': ['opencv-python>=3.2.0.6'],
+    'tf': ['tensorflow>=1.4.0'],
+    'tf_gpu': ['tensorflow-gpu>=1.4.0'],
+    'test': [
+        'Pillow>=3.0',
+        'codecov>=2.0.15',
+        'mock>=2.0.0',
+        'opencv-python>=3.2.0.6',
+        'flake8',
+        'pylint>=1.9',
+        'pytest>=3.0.0',
+        'pytest-cov>=2.5.1',
+        'pytest-forked>=0.2',
+        'pytest-logger>=0.4.0',
+        'pytest-timeout>=1.3.3',
+        'pytest-xdist',
+        's3fs>=0.0.1',
+    ],
+    'torch': ['torchvision>=0.2.1'],
+}
+
+packages = setuptools.find_packages()
+
+setup(
+    name=PACKAGE_NAME,
+    version=version,
+    install_requires=REQUIRED_PACKAGES,
+    packages=packages,
+    description='Pycarbon is a library that optimizes data access for AI based on CarbonData files, '
+                ' and it is based on Petastorm.',
+    long_description=long_description,
+    long_description_content_type="text/markdown", 
+    license='Apache License, Version 2.0',
+    extras_require=EXTRA_REQUIRE,
+    entry_points={
+    },
+    url='https://github.com/apache/carbondata',
+    author='Apache CarbonData',
+    classifiers=[
+        'Environment :: Console',
+        'Environment :: Web Environment',
+        'Intended Audience :: Developers',
+        'Intended Audience :: Science/Research',
+        'License :: OSI Approved :: Apache Software License',
+        'Programming Language :: Python :: 2.7',
+        'Programming Language :: Python :: 3.4',
+        'Programming Language :: Python :: 3.5',
+        'Programming Language :: Python :: 3.6',
+    ],
+)