You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/11/21 17:59:47 UTC

[20/50] [abbrv] carbondata git commit: [CARBONDATA-2996] CarbonSchemaReader support read schema from folder path

[CARBONDATA-2996] CarbonSchemaReader support read schema from folder path

1.Deprecated readSchemaInIndexFile and readSchemaInDataFile, unify them to readSchema in SDK
2.delete readSchemaInIndexFile and readSchemaInDataFile, unify them to readSchema in CSDK
3.readSchema support read schema from folder path,carbonindex file, and carbondata file. and user can decide whether check all files schema

This closes #2804


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

Branch: refs/heads/branch-1.5
Commit: a0a01233eebb4ad7fc28c476c00de3bd7b0fca39
Parents: 6f19fb1
Author: xubo245 <xu...@huawei.com>
Authored: Tue Oct 9 15:16:09 2018 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Nov 21 22:40:11 2018 +0530

----------------------------------------------------------------------
 docs/csdk-guide.md                              |  23 ++-
 docs/sdk-guide.md                               |  25 +++
 .../examples/sdk/CarbonReaderExample.java       |   2 +-
 store/CSDK/src/CarbonSchemaReader.cpp           |  21 +--
 store/CSDK/src/CarbonSchemaReader.h             |  21 ++-
 store/CSDK/test/main.cpp                        |  46 +----
 .../carbondata/sdk/file/CarbonSchemaReader.java | 171 ++++++++++++++++---
 .../org/apache/carbondata/sdk/file/Field.java   |  21 +++
 .../org/apache/carbondata/sdk/file/Schema.java  |  20 +++
 .../carbondata/sdk/file/CarbonReaderTest.java   |  14 +-
 .../sdk/file/CarbonSchemaReaderTest.java        |  76 ++++++++-
 11 files changed, 335 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/docs/csdk-guide.md
----------------------------------------------------------------------
diff --git a/docs/csdk-guide.md b/docs/csdk-guide.md
index b83b06d..95b9cf8 100644
--- a/docs/csdk-guide.md
+++ b/docs/csdk-guide.md
@@ -219,23 +219,28 @@ release the memory and destroy JVM.
 ```
 ```
     /**
-     * read Schema from Data File
+     * read schema from path,
+     * path can be folder path, carbonindex file path, and carbondata file path
+     * and will not check all files schema
      *
-     * @param path Data File path
-     * @return carbon schema object
+     * @param path file/folder path
+     * @return schema
      */
-    jobject readSchemaInDataFile(char *path);
+    jobject readSchema(char *path);
 ```
 ```
     /**
-     * read Schema from index File
+     *  read schema from path,
+     *  path can be folder path, carbonindex file path, and carbondata file path
+     *  and user can decide whether check all files schema
      *
-     * @param path index File path
-     * @return carbon schema object
+     * @param path carbon data path
+     * @param validateSchema whether check all files schema
+     * @return schema
      */
-    jobject readSchemaInIndexFile(char *path);
-
+    jobject readSchema(char *path, bool validateSchema);
 ```
+
 ###Schema
 ``` 
  /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/docs/sdk-guide.md
----------------------------------------------------------------------
diff --git a/docs/sdk-guide.md b/docs/sdk-guide.md
index 8988dc3..6f5c58d 100644
--- a/docs/sdk-guide.md
+++ b/docs/sdk-guide.md
@@ -727,6 +727,31 @@ Find example code at [CarbonReaderExample](https://github.com/apache/carbondata/
    */
   public static Schema readSchemaInIndexFile(String indexFilePath);
 ```
+```
+  /**
+   * read schema from path,
+   * path can be folder path,carbonindex file path, and carbondata file path
+   * and will not check all files schema
+   *
+   * @param path file/folder path
+   * @return schema
+   * @throws IOException
+   */
+  public static Schema readSchema(String path);
+```
+```
+  /**
+   * read schema from path,
+   * path can be folder path,carbonindex file path, and carbondata file path
+   * and user can decide whether check all files schema
+   *
+   * @param path             file/folder path
+   * @param validateSchema whether check all files schema
+   * @return schema
+   * @throws IOException
+   */
+  public static Schema readSchema(String path, boolean validateSchema);
+```
 
 ```
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
index a2f2eee..c730070 100644
--- a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
+++ b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
@@ -96,7 +96,7 @@ public class CarbonReaderExample {
                 throw new RuntimeException("Carbon index file not exists.");
             }
             Schema schema = CarbonSchemaReader
-                .readSchemaInIndexFile(dataFiles[0].getAbsolutePath())
+                .readSchema(dataFiles[0].getAbsolutePath())
                 .asOriginOrder();
             // Transform the schema
             String[] strings = new String[schema.getFields().length];

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/store/CSDK/src/CarbonSchemaReader.cpp
----------------------------------------------------------------------
diff --git a/store/CSDK/src/CarbonSchemaReader.cpp b/store/CSDK/src/CarbonSchemaReader.cpp
index 95eae7d..c895f4b 100644
--- a/store/CSDK/src/CarbonSchemaReader.cpp
+++ b/store/CSDK/src/CarbonSchemaReader.cpp
@@ -29,14 +29,14 @@ CarbonSchemaReader::CarbonSchemaReader(JNIEnv *env) {
     this->jniEnv = env;
 }
 
-jobject CarbonSchemaReader::readSchemaInDataFile(char *path) {
+jobject CarbonSchemaReader::readSchema(char *path) {
     if (path == NULL) {
         throw std::runtime_error("path parameter can't be NULL.");
     }
-    jmethodID methodID = jniEnv->GetStaticMethodID(carbonSchemaReaderClass, "readSchemaInDataFile",
-        "(Ljava/lang/String;)Lorg/apache/carbondata/sdk/file/Schema;");
+    jmethodID methodID = jniEnv->GetStaticMethodID(carbonSchemaReaderClass, "readSchema",
+                                                   "(Ljava/lang/String;)Lorg/apache/carbondata/sdk/file/Schema;");
     if (methodID == NULL) {
-        throw std::runtime_error("Can't find the method in java: readSchemaInDataFile");
+        throw std::runtime_error("Can't find the method in java: readSchema");
     }
     jstring jPath = jniEnv->NewStringUTF(path);
     jvalue args[1];
@@ -48,21 +48,22 @@ jobject CarbonSchemaReader::readSchemaInDataFile(char *path) {
     return result;
 }
 
-jobject CarbonSchemaReader::readSchemaInIndexFile(char *path) {
+jobject CarbonSchemaReader::readSchema(char *path, bool validateSchema) {
     if (path == NULL) {
         throw std::runtime_error("path parameter can't be NULL.");
     }
-    jmethodID methodID = jniEnv->GetStaticMethodID(carbonSchemaReaderClass, "readSchemaInIndexFile",
-        "(Ljava/lang/String;)Lorg/apache/carbondata/sdk/file/Schema;");
+    jmethodID methodID = jniEnv->GetStaticMethodID(carbonSchemaReaderClass, "readSchema",
+                                                   "(Ljava/lang/String;)Lorg/apache/carbondata/sdk/file/Schema;");
     if (methodID == NULL) {
-        throw std::runtime_error("Can't find the method in java: readSchemaInDataFile");
+        throw std::runtime_error("Can't find the method in java: readSchema");
     }
     jstring jPath = jniEnv->NewStringUTF(path);
-    jvalue args[1];
+    jvalue args[2];
     args[0].l = jPath;
+    args[1].z = validateSchema;
     jobject result = jniEnv->CallStaticObjectMethodA(carbonSchemaReaderClass, methodID, args);
     if (jniEnv->ExceptionCheck()) {
         throw jniEnv->ExceptionOccurred();
     }
     return result;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/store/CSDK/src/CarbonSchemaReader.h
----------------------------------------------------------------------
diff --git a/store/CSDK/src/CarbonSchemaReader.h b/store/CSDK/src/CarbonSchemaReader.h
index 1b12995..5c1b324 100644
--- a/store/CSDK/src/CarbonSchemaReader.h
+++ b/store/CSDK/src/CarbonSchemaReader.h
@@ -40,19 +40,24 @@ public:
     CarbonSchemaReader(JNIEnv *env);
 
     /**
-     * read Schema from Data File
+     * read schema from path,
+     * path can be folder path, carbonindex file path, and carbondata file path
+     * and will not check all files schema
      *
-     * @param path Data File path
-     * @return carbon schema object
+     * @param path file/folder path
+     * @return schema
      */
-    jobject readSchemaInDataFile(char *path);
+    jobject readSchema(char *path);
 
     /**
-     * read Schema from index File
+     *  read schema from path,
+     *  path can be folder path, carbonindex file path, and carbondata file path
+     *  and user can decide whether check all files schema
      *
-     * @param path index File path
-     * @return carbon schema object
+     * @param path carbon data path
+     * @param validateSchema whether check all files schema
+     * @return schema
      */
-    jobject readSchemaInIndexFile(char *path);
+    jobject readSchema(char *path, bool validateSchema);
 
 };
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/store/CSDK/test/main.cpp
----------------------------------------------------------------------
diff --git a/store/CSDK/test/main.cpp b/store/CSDK/test/main.cpp
index 5d875e5..44a7c69 100644
--- a/store/CSDK/test/main.cpp
+++ b/store/CSDK/test/main.cpp
@@ -126,17 +126,21 @@ void printResult(JNIEnv *env, CarbonReader reader) {
 }
 
 /**
- * test read Schema from Index File
+ * test read Schema from path
  *
  * @param env jni env
  * @return whether it is success
  */
-bool readSchemaInIndexFile(JNIEnv *env, char *indexFilePath) {
+bool readSchema(JNIEnv *env, char *Path, bool validateSchema) {
     printf("\nread Schema from Index File:\n");
     CarbonSchemaReader carbonSchemaReader(env);
     jobject schema;
     try {
-        schema = carbonSchemaReader.readSchemaInIndexFile(indexFilePath);
+        if (validateSchema) {
+            schema = carbonSchemaReader.readSchema(Path, validateSchema);
+        } else {
+            schema = carbonSchemaReader.readSchema(Path);
+        }
         Schema carbonSchema(env, schema);
         int length = carbonSchema.getFieldsLength();
         printf("schema length is:%d\n", length);
@@ -148,7 +152,6 @@ bool readSchemaInIndexFile(JNIEnv *env, char *indexFilePath) {
                 printf("Array Element Type Name is:%s\n", carbonSchema.getArrayElementTypeName(i));
             }
         }
-
     } catch (jthrowable e) {
         env->ExceptionDescribe();
     }
@@ -156,35 +159,6 @@ bool readSchemaInIndexFile(JNIEnv *env, char *indexFilePath) {
 }
 
 /**
- * test read Schema from Data File
- *
- * @param env jni env
- * @return whether it is success
- */
-bool readSchemaInDataFile(JNIEnv *env, char *dataFilePath) {
-    printf("\nread Schema from Data File:\n");
-    CarbonSchemaReader carbonSchemaReader(env);
-    jobject schema;
-    try {
-        schema = carbonSchemaReader.readSchemaInDataFile(dataFilePath);
-    } catch (jthrowable e) {
-        env->ExceptionDescribe();
-    }
-    Schema carbonSchema(env, schema);
-    int length = carbonSchema.getFieldsLength();
-    printf("schema length is:%d\n", length);
-    for (int i = 0; i < length; i++) {
-        printf("%d\t", i);
-        printf("%s\t", carbonSchema.getFieldName(i));
-        printf("%s\n", carbonSchema.getFieldDataTypeName(i));
-        if (strcmp(carbonSchema.getFieldDataTypeName(i), "ARRAY") == 0) {
-            printf("Array Element Type Name is:%s\n", carbonSchema.getArrayElementTypeName(i));
-        }
-    }
-    return true;
-}
-
-/**
  * test read data from local disk, without projection
  *
  * @param env  jni env
@@ -647,12 +621,8 @@ int main(int argc, char *argv[]) {
         testReadNextBatchRow(env, S3Path, 100000, 100000, argv, 4, true);
     } else {
         tryCatchException(env);
-        char *indexFilePath = argv[1];
-        char *dataFilePath = argv[2];
         testCarbonProperties(env);
         testWriteData(env, "./data", 1, argv);
-        readSchemaInIndexFile(env, indexFilePath);
-        readSchemaInDataFile(env, dataFilePath);
         testWriteData(env, "./data", 1, argv);
         readFromLocalWithoutProjection(env, smallFilePath);
         readFromLocalWithProjection(env, smallFilePath);
@@ -662,6 +632,8 @@ int main(int argc, char *argv[]) {
         testReadNextRow(env, path, printNum, argv, 0, false);
         testReadNextBatchRow(env, path, batch, printNum, argv, 0, true);
         testReadNextBatchRow(env, path, batch, printNum, argv, 0, false);
+        readSchema(env, path, false);
+        readSchema(env, path, true);
     }
     (jvm)->DestroyJavaVM();
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java
index fe8e168..aadd13a 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java
@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
@@ -34,10 +35,12 @@ import org.apache.carbondata.core.reader.CarbonFooterReaderV3;
 import org.apache.carbondata.core.reader.CarbonHeaderReader;
 import org.apache.carbondata.core.reader.CarbonIndexFileReader;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.FileFooter3;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
 
 import static org.apache.carbondata.core.util.CarbonUtil.thriftColumnSchemaToWrapperColumnSchema;
+import static org.apache.carbondata.core.util.path.CarbonTablePath.CARBON_DATA_EXT;
+import static org.apache.carbondata.core.util.path.CarbonTablePath.INDEX_FILE_EXT;
 
 /**
  * Schema reader for carbon files, including carbondata file, carbonindex file, and schema file
@@ -51,6 +54,7 @@ public class CarbonSchemaReader {
    * @return schema object
    * @throws IOException
    */
+  @Deprecated
   public static Schema readSchemaInSchemaFile(String schemaFilePath) throws IOException {
     org.apache.carbondata.format.TableInfo tableInfo = CarbonUtil.readSchemaFile(schemaFilePath);
     SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
@@ -62,13 +66,114 @@ public class CarbonSchemaReader {
   }
 
   /**
+   * get carbondata/carbonindex file in path
+   *
+   * @param path carbon file path
+   * @return CarbonFile array
+   */
+  private static CarbonFile[] getCarbonFile(String path, final String extension)
+      throws IOException {
+    String dataFilePath = path;
+    if (!(dataFilePath.endsWith(extension))) {
+      CarbonFile[] carbonFiles = FileFactory
+          .getCarbonFile(path)
+          .listFiles(new CarbonFileFilter() {
+            @Override
+            public boolean accept(CarbonFile file) {
+              if (file == null) {
+                return false;
+              }
+              return file.getName().endsWith(extension);
+            }
+          });
+      if (carbonFiles == null || carbonFiles.length < 1) {
+        throw new IOException("Carbon file not exists.");
+      }
+      return carbonFiles;
+    } else {
+      throw new CarbonDataLoadingException("Please ensure path "
+          + path + " end with " + extension);
+    }
+  }
+
+  /**
+   * read schema from path,
+   * path can be folder path, carbonindex file path, and carbondata file path
+   * and will not check all files schema
+   *
+   * @param path file/folder path
+   * @return schema
+   * @throws IOException
+   */
+  public static Schema readSchema(String path) throws IOException {
+    return readSchema(path, false);
+  }
+
+  /**
+   * read schema from path,
+   * path can be folder path, carbonindex file path, and carbondata file path
+   * and user can decide whether check all files schema
+   *
+   * @param path             file/folder path
+   * @param validateSchema whether check all files schema
+   * @return schema
+   * @throws IOException
+   */
+  public static Schema readSchema(String path, boolean validateSchema) throws IOException {
+    if (path.endsWith(INDEX_FILE_EXT)) {
+      return readSchemaFromIndexFile(path);
+    } else if (path.endsWith(CARBON_DATA_EXT)) {
+      return readSchemaFromDataFile(path);
+    } else if (validateSchema) {
+      CarbonFile[] carbonIndexFiles = getCarbonFile(path, INDEX_FILE_EXT);
+      Schema schema;
+      if (carbonIndexFiles != null && carbonIndexFiles.length != 0) {
+        schema = readSchemaFromIndexFile(carbonIndexFiles[0].getAbsolutePath());
+        for (int i = 1; i < carbonIndexFiles.length; i++) {
+          Schema schema2 = readSchemaFromIndexFile(carbonIndexFiles[i].getAbsolutePath());
+          if (!schema.equals(schema2)) {
+            throw new CarbonDataLoadingException("Schema is different between different files.");
+          }
+        }
+        CarbonFile[] carbonDataFiles = getCarbonFile(path, CARBON_DATA_EXT);
+        for (int i = 0; i < carbonDataFiles.length; i++) {
+          Schema schema2 = readSchemaFromDataFile(carbonDataFiles[i].getAbsolutePath());
+          if (!schema.equals(schema2)) {
+            throw new CarbonDataLoadingException("Schema is different between different files.");
+          }
+        }
+        return schema;
+      } else {
+        throw new CarbonDataLoadingException("No carbonindex file in this path.");
+      }
+    } else {
+      String indexFilePath = getCarbonFile(path, INDEX_FILE_EXT)[0].getAbsolutePath();
+      return readSchemaFromIndexFile(indexFilePath);
+    }
+  }
+
+  /**
    * Read carbondata file and return the schema
+   * This interface will be removed,
+   * please use readSchema instead of this interface
    *
-   * @param dataFilePath complete path including carbondata file name
+   * @param dataFilePath carbondata file store path
    * @return Schema object
    * @throws IOException
    */
+  @Deprecated
   public static Schema readSchemaInDataFile(String dataFilePath) throws IOException {
+    return readSchema(dataFilePath, false);
+  }
+
+  /**
+   * Read schema from carbondata file
+   *
+   * @param dataFilePath carbondata file path
+   * @return carbon data schema
+   * @throws IOException
+   */
+  public static Schema readSchemaFromDataFile(String dataFilePath) throws IOException {
     CarbonHeaderReader reader = new CarbonHeaderReader(dataFilePath);
     List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
     List<ColumnSchema> schemaList = reader.readSchema();
@@ -82,40 +187,30 @@ public class CarbonSchemaReader {
   }
 
   /**
-   * This method return the version details in formatted string by reading from carbondata file
-   * @param dataFilePath
-   * @return
+   * Read carbonindex file and return the schema
+   * This interface will be removed,
+   * please use readSchema instead of this interface
+   *
+   * @param indexFilePath complete path including index file name
+   * @return schema object
    * @throws IOException
    */
-  public static String getVersionDetails(String dataFilePath) throws IOException {
-    long fileSize =
-        FileFactory.getCarbonFile(dataFilePath, FileFactory.getFileType(dataFilePath)).getSize();
-    FileReader fileReader = FileFactory.getFileHolder(FileFactory.getFileType(dataFilePath));
-    ByteBuffer buffer =
-        fileReader.readByteBuffer(FileFactory.getUpdatedFilePath(dataFilePath), fileSize - 8, 8);
-    fileReader.finish();
-    CarbonFooterReaderV3 footerReader = new CarbonFooterReaderV3(dataFilePath, buffer.getLong());
-    FileFooter3 footer = footerReader.readFooterVersion3();
-    if (null != footer.getExtra_info()) {
-      return footer.getExtra_info().get(CarbonCommonConstants.CARBON_WRITTEN_BY_FOOTER_INFO)
-          + " in version: " + footer.getExtra_info()
-          .get(CarbonCommonConstants.CARBON_WRITTEN_VERSION);
-    } else {
-      return "Version Details are not found in carbondata file";
-    }
+  @Deprecated
+  public static Schema readSchemaInIndexFile(String indexFilePath) throws IOException {
+    return readSchema(indexFilePath, false);
   }
 
   /**
-   * Read carbonindex file and return the schema
+   * Read schema from carbonindex file
    *
-   * @param indexFilePath complete path including index file name
-   * @return schema object
+   * @param indexFilePath carbonindex file path
+   * @return carbon data Schema
    * @throws IOException
    */
-  public static Schema readSchemaInIndexFile(String indexFilePath) throws IOException {
+  private static Schema readSchemaFromIndexFile(String indexFilePath) throws IOException {
     CarbonFile indexFile =
         FileFactory.getCarbonFile(indexFilePath, FileFactory.getFileType(indexFilePath));
-    if (!indexFile.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
+    if (!indexFile.getName().endsWith(INDEX_FILE_EXT)) {
       throw new IOException("Not an index file name");
     }
     // read schema from the first index file
@@ -144,4 +239,28 @@ public class CarbonSchemaReader {
     }
   }
 
+  /**
+   * This method return the version details in formatted string by reading from carbondata file
+   *
+   * @param dataFilePath
+   * @return
+   * @throws IOException
+   */
+  public static String getVersionDetails(String dataFilePath) throws IOException {
+    long fileSize =
+        FileFactory.getCarbonFile(dataFilePath, FileFactory.getFileType(dataFilePath)).getSize();
+    FileReader fileReader = FileFactory.getFileHolder(FileFactory.getFileType(dataFilePath));
+    ByteBuffer buffer =
+        fileReader.readByteBuffer(FileFactory.getUpdatedFilePath(dataFilePath), fileSize - 8, 8);
+    fileReader.finish();
+    CarbonFooterReaderV3 footerReader = new CarbonFooterReaderV3(dataFilePath, buffer.getLong());
+    FileFooter3 footer = footerReader.readFooterVersion3();
+    if (null != footer.getExtra_info()) {
+      return footer.getExtra_info().get(CarbonCommonConstants.CARBON_WRITTEN_BY_FOOTER_INFO)
+          + " in version: " + footer.getExtra_info()
+          .get(CarbonCommonConstants.CARBON_WRITTEN_VERSION);
+    } else {
+      return "Version Details are not found in carbondata file";
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java
index 065ff80..fb9a6d8 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java
@@ -291,4 +291,25 @@ public class Field {
       return new StructField(fieldName, dataType);
     }
   }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof Field) {
+      Field field = (Field) obj;
+      if ((!this.getDataType().equals(field.getDataType()))
+          || (!this.getFieldName().equals(field.getFieldName()))
+          || (!(this.getSchemaOrdinal() == (field.getSchemaOrdinal())))
+          ) {
+        return false;
+      }
+    } else {
+      return false;
+    }
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java
index 05d3d9e..fb23725 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java
@@ -151,4 +151,24 @@ public class Schema {
     });
     return this;
   }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof Schema) {
+      Schema schema = (Schema) obj;
+      for (int i = 0; i < this.fields.length; i++) {
+        if (!(schema.fields)[i].equals((this.fields)[i])) {
+          return false;
+        }
+      }
+    } else {
+      return false;
+    }
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
index d79a1ad..c176f06 100644
--- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
@@ -777,7 +777,7 @@ public class CarbonReaderTest extends TestCase {
     });
     Assert.assertTrue(dataFiles != null);
     Assert.assertTrue(dataFiles.length > 0);
-    Schema schema = CarbonSchemaReader.readSchemaInDataFile(dataFiles[0].getAbsolutePath());
+    Schema schema = CarbonSchemaReader.readSchema(dataFiles[0].getAbsolutePath());
     Assert.assertTrue(schema.getFields().length == 2);
     Assert.assertEquals("name", (schema.getFields())[0].getFieldName());
     Assert.assertEquals("age", (schema.getFields())[1].getFieldName());
@@ -1021,7 +1021,7 @@ public class CarbonReaderTest extends TestCase {
       }
     });
 
-    Schema schema = CarbonSchemaReader.readSchemaInDataFile(dataFiles2[0].getAbsolutePath());
+    Schema schema = CarbonSchemaReader.readSchema(dataFiles2[0].getAbsolutePath());
 
     // sort the schema
     Arrays.sort(schema.getFields(), new Comparator<Field>() {
@@ -1140,7 +1140,7 @@ public class CarbonReaderTest extends TestCase {
       }
     });
 
-    Schema schema = CarbonSchemaReader.readSchemaInIndexFile(dataFiles2[0].getAbsolutePath()).asOriginOrder();
+    Schema schema = CarbonSchemaReader.readSchema(dataFiles2[0].getAbsolutePath()).asOriginOrder();
     // Transform the schema
     String[] strings = new String[schema.getFields().length];
     for (int i = 0; i < schema.getFields().length; i++) {
@@ -1355,7 +1355,7 @@ public class CarbonReaderTest extends TestCase {
       }
     });
 
-    Schema schema = CarbonSchemaReader.readSchemaInIndexFile(dataFiles2[0].getAbsolutePath()).asOriginOrder();
+    Schema schema = CarbonSchemaReader.readSchema(dataFiles2[0].getAbsolutePath()).asOriginOrder();
 
     for (int i = 0; i < schema.getFields().length; i++) {
       System.out.println((schema.getFields())[i].getFieldName() + "\t" + schema.getFields()[i].getSchemaOrdinal());
@@ -1518,10 +1518,10 @@ public class CarbonReaderTest extends TestCase {
         }
       });
       if (dataFiles == null || dataFiles.length < 1) {
-        throw new RuntimeException("Carbon index file not exists.");
+        throw new RuntimeException("Carbon data file not exists.");
       }
       Schema schema = CarbonSchemaReader
-          .readSchemaInDataFile(dataFiles[0].getAbsolutePath())
+          .readSchema(dataFiles[0].getAbsolutePath())
           .asOriginOrder();
       // Transform the schema
       String[] strings = new String[schema.getFields().length];
@@ -1613,7 +1613,7 @@ public class CarbonReaderTest extends TestCase {
         throw new RuntimeException("Carbon index file not exists.");
       }
       Schema schema = CarbonSchemaReader
-          .readSchemaInIndexFile(dataFiles[0].getAbsolutePath())
+          .readSchema(dataFiles[0].getAbsolutePath())
           .asOriginOrder();
       // Transform the schema
       int count = 0;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a0a01233/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonSchemaReaderTest.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonSchemaReaderTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonSchemaReaderTest.java
index db41fa0..1e70ada 100644
--- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonSchemaReaderTest.java
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonSchemaReaderTest.java
@@ -22,17 +22,20 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import junit.framework.TestCase;
 import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.commons.io.FileUtils;
-import org.junit.*;
 
-public class CarbonSchemaReaderTest extends TestCase {
+import junit.framework.TestCase;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
 
+public class CarbonSchemaReaderTest extends TestCase {
   String path = "./testWriteFiles";
 
   @Before
@@ -101,18 +104,32 @@ public class CarbonSchemaReaderTest extends TestCase {
       String dataFilePath = carbonFiles[0].getAbsolutePath();
 
       Schema schema = CarbonSchemaReader
-          .readSchemaInDataFile(dataFilePath)
+          .readSchema(dataFilePath)
           .asOriginOrder();
 
       assertEquals(schema.getFieldsLength(), 12);
       checkSchema(schema);
+    } catch (Throwable e) {
+      e.printStackTrace();
+      Assert.fail();
+    }
+  }
 
+  @Test
+  public void testReadSchemaWithoutSchemaFilesSchema() {
+    try {
+      Schema schema = CarbonSchemaReader
+          .readSchema(path)
+          .asOriginOrder();
+      checkSchema(schema);
     } catch (Throwable e) {
       e.printStackTrace();
+      Assert.fail();
     }
   }
 
   public boolean checkSchema(Schema schema) {
+    assertEquals(schema.getFields().length, 12);
     assert (schema.getFieldName(0).equalsIgnoreCase("stringField"));
     assert (schema.getFieldName(1).equalsIgnoreCase("shortField"));
     assert (schema.getFieldName(2).equalsIgnoreCase("intField"));
@@ -159,17 +176,63 @@ public class CarbonSchemaReaderTest extends TestCase {
       if (carbonFiles == null || carbonFiles.length < 1) {
         throw new RuntimeException("Carbon index file not exists.");
       }
-      String dataFilePath = carbonFiles[0].getAbsolutePath();
+      String indexFilePath = carbonFiles[0].getAbsolutePath();
 
       Schema schema = CarbonSchemaReader
-          .readSchemaInDataFile(dataFilePath)
+          .readSchema(indexFilePath)
           .asOriginOrder();
 
       assertEquals(schema.getFieldsLength(), 12);
       checkSchema(schema);
+    } catch (Throwable e) {
+      e.printStackTrace();
+      Assert.fail();
+    }
+  }
+
+  @Test
+  public void testReadSchemaAndCheckFilesSchema() {
+    try {
+      Schema schema = CarbonSchemaReader
+          .readSchema(path, true)
+          .asOriginOrder();
+      checkSchema(schema);
+    } catch (Throwable e) {
+      e.printStackTrace();
+      Assert.fail();
+    }
+  }
 
+  @Test
+  public void testReadSchemaWithDifferentSchema() {
+    try {
+      int num = 10;
+      Field[] fields = new Field[2];
+      fields[0] = new Field("name", DataTypes.STRING);
+      fields[1] = new Field("age", DataTypes.INT);
+      CarbonWriter writer = CarbonWriter
+          .builder()
+          .outputPath(path)
+          .withCsvInput(new Schema(fields))
+          .writtenBy("testReadSchemaWithDifferentSchema")
+          .build();
+
+      for (int i = 0; i < num; i++) {
+        writer.write(new String[]{"robot" + (i % 10), String.valueOf(i)});
+      }
+      writer.close();
+      try {
+        CarbonSchemaReader
+            .readSchema(path, true)
+            .asOriginOrder();
+        Assert.fail();
+      } catch (Exception e) {
+        Assert.assertTrue(e.getMessage()
+            .equalsIgnoreCase("Schema is different between different files."));
+      }
     } catch (Throwable e) {
       e.printStackTrace();
+      Assert.fail();
     }
   }
 
@@ -177,5 +240,4 @@ public class CarbonSchemaReaderTest extends TestCase {
   public void tearDown() throws IOException {
     FileUtils.deleteDirectory(new File(path));
   }
-
 }