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 2017/09/15 13:56:03 UTC

carbondata git commit: [CARBONDATA-1482] fixed the failing test cases of presto integration

Repository: carbondata
Updated Branches:
  refs/heads/master 6f204376f -> d60d973df


[CARBONDATA-1482] fixed the failing test cases of presto integration

1.builds are passing
2.added dependency for spark guava
3.throw exception in case carbondatastore creator is unable to load the data
4.fixed the timestamp issue due to which test cases are failing

This closes #1360


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

Branch: refs/heads/master
Commit: d60d973dfbb69c1b740a6a00cab878bc168c14ae
Parents: 6f20437
Author: anubhav100 <an...@knoldus.in>
Authored: Fri Sep 15 15:12:10 2017 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Fri Sep 15 19:25:10 2017 +0530

----------------------------------------------------------------------
 integration/presto/pom.xml                      |  7 ++++
 .../integrationtest/PrestoAllDataTypeTest.scala | 26 +++++++-------
 .../presto/util/CarbonDataStoreCreator.scala    | 36 ++++++++++++--------
 3 files changed, 42 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d60d973d/integration/presto/pom.xml
----------------------------------------------------------------------
diff --git a/integration/presto/pom.xml b/integration/presto/pom.xml
index b23b1be..50e6349 100644
--- a/integration/presto/pom.xml
+++ b/integration/presto/pom.xml
@@ -438,6 +438,13 @@
         </exclusion>
       </exclusions>
     </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-network-common_2.11</artifactId>
+      <scope>test</scope>
+      <version>2.1.0</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d60d973d/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
index 1743be6..433ddd4 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.carbondata.presto.integrationtest
 
 import java.io.File
 
 import org.scalatest.{BeforeAndAfterAll, FunSuiteLike}
-import util.CarbonDataStoreCreator
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.presto.server.PrestoServer
@@ -35,8 +35,10 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
   private val storePath = s"$rootPath/integration/presto/target/store"
 
   override def beforeAll: Unit = {
+    import org.apache.carbondata.presto.util.CarbonDataStoreCreator
     CarbonDataStoreCreator
-      .createCarbonStore(storePath, s"$rootPath/integration/presto/src/test/resources/alldatatype.csv")
+      .createCarbonStore(storePath,
+        s"$rootPath/integration/presto/src/test/resources/alldatatype.csv")
     logger.info(s"\nCarbon store is created at location: $storePath")
     PrestoServer.startServer(storePath)
   }
@@ -91,21 +93,21 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
   test("test the result for count()clause with distinct operator on decimal column in presto") {
     val actualResult: List[Map[String, Any]] = PrestoServer
       .executeQuery("SELECT COUNT(DISTINCT BONUS) AS RESULT FROM TESTDB.TESTTABLE ")
-    val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> 7))
-    actualResult.equals(expectedResult)
+    val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> 10))
+    assert(actualResult.equals(expectedResult))
   }
   test("test the result for count()clause with out  distinct operator on decimal column in presto")
   {
     val actualResult: List[Map[String, Any]] = PrestoServer
       .executeQuery("SELECT COUNT(BONUS) AS RESULT FROM TESTDB.TESTTABLE ")
     val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> 10))
-    actualResult.equals(expectedResult)
+    assert(actualResult.equals(expectedResult))
   }
   test("test the result for sum()with out distinct operator for decimal column in presto") {
     val actualResult: List[Map[String, Any]] = PrestoServer
       .executeQuery("SELECT SUM(DISTINCT BONUS) AS RESULT FROM TESTDB.TESTTABLE ")
-    val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> 54))
-    actualResult.equals(expectedResult)
+    val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> 20774.6475))
+    assert(actualResult.toString().equals(expectedResult.toString()))
   }
   test("test the result for sum() with distinct operator for decimal column in presto") {
     val actualResult: List[Map[String, Any]] = PrestoServer
@@ -115,11 +117,11 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
       actualResult.head("RESULT").toString.toDouble ==
       expectedResult.head("RESULT").toString.toDouble)
   }
-  test("test the result for avg() with distinct operator on decimal coin presto") {
+  test("test the result for avg() with distinct operator on decimal on presto") {
     val actualResult: List[Map[String, Any]] = PrestoServer
       .executeQuery("SELECT AVG(DISTINCT BONUS) AS RESULT FROM TESTDB.TESTTABLE ")
-    val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> 8900))
-    actualResult.equals(expectedResult)
+    val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> 2077.4648))
+    assert(actualResult.toString.equals(expectedResult.toString))
   }
 
   test("test the result for min() with distinct operator in decimalType of presto") {
@@ -127,7 +129,7 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
       .executeQuery("SELECT MIN(BONUS) AS RESULT FROM TESTDB.TESTTABLE ")
     val expectedResult: List[Map[String, Any]] = List(Map(
       "RESULT" -> java.math.BigDecimal.valueOf(500.414).setScale(4)))
-    actualResult.equals(expectedResult)
+    assert(actualResult.equals(expectedResult))
   }
 
   test("test the result for max() with distinct operator in decimalType of presto") {
@@ -135,7 +137,7 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
       .executeQuery("SELECT MAX(BONUS) AS RESULT FROM TESTDB.TESTTABLE ")
     val expectedResult: List[Map[String, Any]] = List(Map(
       "RESULT" -> java.math.BigDecimal.valueOf(9999.999).setScale(4)))
-    actualResult.equals(expectedResult)
+    assert(actualResult.equals(expectedResult))
   }
   test("select decimal data type with ORDER BY  clause") {
     val actualResult: List[Map[String, Any]] = PrestoServer

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d60d973d/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index 6cb97f1..64f892d 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package util
+package org.apache.carbondata.presto.util
 
+import java.util
 import java.io._
 import java.nio.charset.Charset
 import java.text.SimpleDateFormat
-import java.util
 import java.util.{ArrayList, Date, List, UUID}
 
 import scala.collection.JavaConversions._
@@ -30,34 +30,34 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.mapred.TaskAttemptID
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.hadoop.mapreduce.{RecordReader, TaskType}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 
 import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier,
 ReverseDictionary}
-import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.fileoperations.{AtomicFileOperations, AtomicFileOperationsImpl,
 FileWriteOperation}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata,
+CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.converter.{SchemaConverter,
 ThriftWrapperSchemaConverterImpl}
 import org.apache.carbondata.core.metadata.datatype.DataType
 import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolutionEntry}
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension,
 CarbonMeasure, ColumnSchema}
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
-import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolutionEntry}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata,
-CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter,
-CarbonDictionarySortIndexWriterImpl, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
+import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.writer.{CarbonDictionaryWriter, CarbonDictionaryWriterImpl,
 ThriftWriter}
+import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter,
+CarbonDictionarySortIndexWriterImpl, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
 import org.apache.carbondata.processing.api.dataloader.SchemaInfo
 import org.apache.carbondata.processing.constants.TableOptionConstant
 import org.apache.carbondata.processing.csvload.{BlockDetails, CSVInputFormat,
@@ -65,6 +65,7 @@ CSVRecordReaderIterator, StringArrayWritable}
 import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.newflow.DataLoadExecutor
 import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
+import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
 
 object CarbonDataStoreCreator {
 
@@ -110,7 +111,7 @@ object CarbonDataStoreCreator {
       loadModel.setDefaultTimestampFormat(
         CarbonProperties.getInstance.getProperty(
           CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+          CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS))
       loadModel.setDefaultDateFormat(
         CarbonProperties.getInstance.getProperty(
           CarbonCommonConstants.CARBON_DATE_FORMAT,
@@ -397,7 +398,11 @@ object CarbonDataStoreCreator {
           dictionarySortInfo.getSortIndexInverted)
       }
       catch {
-        case exception: Exception => logger.error(s"exception occurs $exception")
+        case exception: Exception =>
+
+
+          logger.error(s"exception occurs $exception")
+          throw new CarbonDataLoadingException("Data Loading Failed")
       }
       finally carbonDictionaryWriter.close()
     }
@@ -545,13 +550,14 @@ object CarbonDataStoreCreator {
       writeOperation.close()
     }
     catch {
-      case exception: Exception => logger.error(s"Exception occurs $exception")
+      case exception: Exception => logger.error(s"exception occurs $exception")
+        throw new CarbonDataLoadingException("Data Loading Failed")
     }
   }
 
   private def readCurrentTime(): String = {
     val sdf: SimpleDateFormat = new SimpleDateFormat(
-      CarbonCommonConstants.CARBON_TIMESTAMP)
+      CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS)
     sdf.format(new Date())
   }