diff --git a/cluster/sbin/snappy-nodes.sh b/cluster/sbin/snappy-nodes.sh
index e717eeff35..a2ab7d5580 100755
--- a/cluster/sbin/snappy-nodes.sh
+++ b/cluster/sbin/snappy-nodes.sh
@@ -223,14 +223,6 @@ function execute() {
2>&1 | sed "s/^/$host: /") &
LAST_PID="$!"
fi
- if [ -z "$RUN_IN_BACKGROUND" ]; then
- wait $LAST_PID
- else
- sleep 1
- if [ -e "/proc/$LAST_PID/status" ]; then
- sleep 1
- fi
- fi
else
if [ "$dirfolder" != "" ]; then
# Create the directory for the snappy component if the folder is a default folder
@@ -239,7 +231,16 @@ function execute() {
fi
fi
launchcommand="${@// /\\ } ${args} < /dev/null 2>&1"
- eval $launchcommand
+ eval $launchcommand &
+ LAST_PID="$!"
+ fi
+ if [ -z "$RUN_IN_BACKGROUND" ]; then
+ wait $LAST_PID
+ else
+ sleep 1
+ if [ -e "/proc/$LAST_PID/status" ]; then
+ sleep 1
+ fi
fi
df=${dirfolder}
diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala
index 3e46f47ec2..5d95f27b55 100644
--- a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala
+++ b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala
@@ -75,7 +75,7 @@ abstract class ClusterManagerTestBase(s: String)
sysProps.setProperty("p2p.minJoinTries", "1")
// spark memory fill to detect any uninitialized memory accesses
- // sysProps.setProperty("spark.memory.debugFill", "true")
+ sysProps.setProperty("spark.memory.debugFill", "true")
var host: Host = _
var vm0: VM = _
diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SnappySecureJob.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SnappySecureJob.scala
index cbba116e03..bbfe14fae7 100644
--- a/cluster/src/dunit/scala/io/snappydata/cluster/SnappySecureJob.scala
+++ b/cluster/src/dunit/scala/io/snappydata/cluster/SnappySecureJob.scala
@@ -21,11 +21,14 @@ import java.io.{FileOutputStream, PrintWriter}
import com.pivotal.gemfirexd.Attribute
import com.typesafe.config.{Config, ConfigException}
-import io.snappydata.Constant
+import io.snappydata.{Constant, ServiceManager}
+import io.snappydata.impl.LeadImpl
+import org.apache.spark.SparkCallbacks
import org.apache.spark.sql.types.{DecimalType, IntegerType, StructField, StructType}
import org.apache.spark.sql._
import org.apache.spark.sql.streaming.SnappyStreamingJob
import org.apache.spark.streaming.SnappyStreamingContext
+import org.apache.spark.ui.SnappyBasicAuthenticator
// scalastyle:off println
class SnappySecureJob extends SnappySQLJob {
@@ -56,6 +59,12 @@ class SnappySecureJob extends SnappySQLJob {
} else {
accessAndModifyTablesOwnedByOthers(snSession, jobConfig)
}
+ // Confirm that our zeppelin interpreter is not initialized.
+ assert(ServiceManager.getLeadInstance.asInstanceOf[LeadImpl].getInterpreterServerClass() ==
+ null, "Zeppelin interpreter must not be initialized in secure cluster")
+ // Check SnappyData Pulse UI is secured by our custom authenticator.
+ assert(SparkCallbacks.getAuthenticatorForJettyServer().get
+ .isInstanceOf[SnappyBasicAuthenticator], "SnappyData Pulse UI not secured")
pw.println(msg)
} finally {
pw.close()
diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala
index de2cc365cf..900dac55c7 100644
--- a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala
+++ b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala
@@ -266,6 +266,7 @@ class SplitSnappyClusterDUnitTest(s: String)
ColumnUpdateDeleteTests.testSNAP1925(session)
ColumnUpdateDeleteTests.testSNAP1926(session)
ColumnUpdateDeleteTests.testConcurrentOps(session)
+ ColumnUpdateDeleteTests.testSNAP2124(session, checkPruning = true)
} finally {
StoreUtils.TEST_RANDOM_BUCKETID_ASSIGNMENT = false
}
diff --git a/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala b/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala
index 24354630bd..ff2a89de9c 100644
--- a/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala
+++ b/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala
@@ -81,6 +81,7 @@ class LeadImpl extends ServerImpl with Lead
isTestSetup = bootProperties.getProperty("isTest", "false").toBoolean
bootProperties.remove("isTest")
+ val authSpecified = Misc.checkAuthProvider(bootProperties)
// prefix all store properties with "snappydata.store" for SparkConf
@@ -162,7 +163,7 @@ class LeadImpl extends ServerImpl with Lead
val zeppelinEnabled = bootProperties.getProperty(
Constant.ENABLE_ZEPPELIN_INTERPRETER, "false").equalsIgnoreCase("true")
- if (zeppelinEnabled) {
+ if (zeppelinEnabled && !authSpecified) {
try {
val zeppelinIntpUtilClass = Utils.classForName(
@@ -190,7 +191,7 @@ class LeadImpl extends ServerImpl with Lead
// The auth service is not yet initialized at this point.
// So simply check the auth-provider property value.
- if (Misc.checkAuthProvider(bootProperties)) {
+ if (authSpecified) {
logInfo("Enabling user authentication for SnappyData Pulse")
SparkCallbacks.setAuthenticatorForJettyServer()
}
@@ -234,7 +235,7 @@ class LeadImpl extends ServerImpl with Lead
}
// wait for a while until servers get registered
- val endWait = System.currentTimeMillis() + 10000
+ val endWait = System.currentTimeMillis() + 120000
while (!SnappyContext.hasServerBlockIds && System.currentTimeMillis() <= endWait) {
Thread.sleep(100)
}
@@ -258,8 +259,10 @@ class LeadImpl extends ServerImpl with Lead
// start other add-on services (job server)
startAddOnServices(conf, confFile, jobServerConfig)
- // finally start embedded zeppelin interpreter if configured
- checkAndStartZeppelinInterpreter(zeppelinEnabled, bootProperties)
+ // finally start embedded zeppelin interpreter if configured and security is not enabled.
+ if (!authSpecified) {
+ checkAndStartZeppelinInterpreter(zeppelinEnabled, bootProperties)
+ }
if (jobServerWait) {
// mark RUNNING after job server and zeppelin initialization if so configured
@@ -620,13 +623,13 @@ class LeadImpl extends ServerImpl with Lead
/**
* This method is used to start the zeppelin interpreter thread.
- * As discussed by default zeppelin interpreter will be enabled.User can disable it by
- * setting "zeppelin.interpreter.enable" to false in leads conf file.User can also specify
- * the port on which intrepreter should listen using property zeppelin.interpreter.port
+ * By default, zeppelin interpreter will be disabled. User can enable it by
+ * setting "zeppelin.interpreter.enable" to true in leads conf file. User can also specify
+ * the port on which interpreter should listen using property "zeppelin.interpreter.port"
*/
private def checkAndStartZeppelinInterpreter(enabled: Boolean,
bootProperties: Properties): Unit = {
- // As discussed ZeppelinRemoteInterpreter Server will be enabled by default.
+ // As discussed ZeppelinRemoteInterpreter Server will be disabled by default.
// [sumedh] Our startup times are already very high and we are looking to
// cut that down and not increase further with these external utilities.
if (enabled) {
@@ -654,6 +657,10 @@ class LeadImpl extends ServerImpl with Lead
}
}
+
+ def getInterpreterServerClass(): Class[_] = {
+ remoteInterpreterServerClass
+ }
}
object LeadImpl {
diff --git a/cluster/src/main/scala/org/apache/spark/SparkCallbacks.scala b/cluster/src/main/scala/org/apache/spark/SparkCallbacks.scala
index 00c61d9246..338a61225d 100644
--- a/cluster/src/main/scala/org/apache/spark/SparkCallbacks.scala
+++ b/cluster/src/main/scala/org/apache/spark/SparkCallbacks.scala
@@ -17,12 +17,12 @@
package org.apache.spark
import org.apache.spark
-
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.memory.StoreUnifiedManager
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RetrieveSparkAppConfig, SparkAppConfig}
import org.apache.spark.ui.{JettyUtils, SnappyBasicAuthenticator}
+import org.eclipse.jetty.security.authentication.BasicAuthenticator
/**
* Calls that are needed to be sent to snappy-cluster classes because
@@ -39,8 +39,10 @@ object SparkCallbacks {
ioEncryptionKey: Option[Array[Byte]],
isLocal: Boolean): SparkEnv = {
- SparkEnv.createExecutorEnv(driverConf, executorId, hostname,
+ val env = SparkEnv.createExecutorEnv(driverConf, executorId, hostname,
port, numCores, ioEncryptionKey, isLocal)
+ env.memoryManager.asInstanceOf[StoreUnifiedManager].init()
+ env
}
def getRpcEnv(sparkEnv: SparkEnv): RpcEnv = {
@@ -92,6 +94,10 @@ object SparkCallbacks {
}
}
+ def getAuthenticatorForJettyServer(): Option[BasicAuthenticator] = {
+ JettyUtils.customAuthenticator
+ }
+
def setSparkConf(sc: SparkContext, key: String, value: String): Unit = {
if (value ne null) sc.conf.set(key, value) else sc.conf.remove(key)
}
diff --git a/cluster/src/main/scala/org/apache/spark/memory/SnappyUnifiedMemoryManager.scala b/cluster/src/main/scala/org/apache/spark/memory/SnappyUnifiedMemoryManager.scala
index 82895712dc..1025465400 100644
--- a/cluster/src/main/scala/org/apache/spark/memory/SnappyUnifiedMemoryManager.scala
+++ b/cluster/src/main/scala/org/apache/spark/memory/SnappyUnifiedMemoryManager.scala
@@ -785,6 +785,10 @@ class SnappyUnifiedMemoryManager private[memory](
wrapperStats.setMemoryManagerStats(stats)
}
+ /**
+ * Initializes the memoryManager
+ */
+ override def init(): Unit = memoryForObject
}
object SnappyUnifiedMemoryManager extends Logging {
diff --git a/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyEmbeddedModeClusterManager.scala b/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyEmbeddedModeClusterManager.scala
index 2babc3bf8b..463faff505 100644
--- a/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyEmbeddedModeClusterManager.scala
+++ b/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyEmbeddedModeClusterManager.scala
@@ -50,7 +50,6 @@ class SnappyEmbeddedModeClusterManager extends ExternalClusterManager {
(split(0).trim, split(1).trim)
}
else if (locator.isEmpty ||
- locator == "" ||
locator == "null" ||
!ServiceUtils.LOCATOR_URL_PATTERN.matcher(locator).matches()
) {
diff --git a/cluster/src/test/scala/io/snappydata/QueryTest.scala b/cluster/src/test/scala/io/snappydata/QueryTest.scala
index 7a81378220..91156bdab3 100644
--- a/cluster/src/test/scala/io/snappydata/QueryTest.scala
+++ b/cluster/src/test/scala/io/snappydata/QueryTest.scala
@@ -19,7 +19,6 @@ package io.snappydata
import scala.collection.JavaConverters._
-import org.apache.spark.sql.QueryTest.checkAnswer
import org.apache.spark.sql.execution.benchmark.ColumnCacheBenchmark
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.{AnalysisException, Row, SnappyContext, SnappySession, SparkSession}
@@ -271,10 +270,11 @@ class QueryTest extends SnappyFunSuite {
"select count(*), city from $t group by city",
"select count(*), city from $t where country like 'country_1%' group by city",
"select count(*), city, collect_list(airport_id), collect_list(name), " +
- "collect_list(country) from (select * from $t order by airport_id) as t group by city",
+ "collect_list(country) from (select * from $t order by airport_id, name, country) " +
+ "as t group by city order by city",
"select count(*), city, collect_list(airport_id), collect_list(name), " +
"collect_list(country) from (select * from $t where country like 'country_1%' " +
- " order by airport_id) as t group by city"
+ " order by airport_id, name, country) as t group by city order by city"
)
// To validate the results against queries directly on data disabling snappy aggregation.
@@ -287,8 +287,8 @@ class QueryTest extends SnappyFunSuite {
}
for (((r1, r2), e) <- results.zip(expectedResults)) {
- org.apache.spark.sql.QueryTest.checkAnswer(r1, e)
- org.apache.spark.sql.QueryTest.checkAnswer(r2, e)
+ checkAnswer(r1, e)
+ checkAnswer(r2, e)
}
// fire updates and check again
@@ -302,8 +302,8 @@ class QueryTest extends SnappyFunSuite {
}
for (((r1, r2), e) <- results.zip(expectedResults)) {
- org.apache.spark.sql.QueryTest.checkAnswer(r1, e)
- org.apache.spark.sql.QueryTest.checkAnswer(r2, e)
+ checkAnswer(r1, e)
+ checkAnswer(r2, e)
}
}
}
diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCDSSuite.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCDSSuite.scala
new file mode 100644
index 0000000000..43eb2f6872
--- /dev/null
+++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCDSSuite.scala
@@ -0,0 +1,127 @@
+/*
+ * Copyright (c) 2017 SnappyData, Inc. All rights reserved.
+ *
+ * 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. See accompanying
+ * LICENSE file.
+ */
+
+package io.snappydata.benchmark.snappy
+
+import java.io.{File, FileOutputStream, PrintStream}
+
+import io.snappydata.SnappyFunSuite
+import org.apache.spark.sql.execution.benchmark.TPCDSQuerySnappyBenchmark
+import org.apache.spark.sql.{SnappySession, SparkSession}
+import org.apache.spark.{SparkConf, SparkContext}
+import org.scalatest.BeforeAndAfterAll
+
+
+class TPCDSSuite extends SnappyFunSuite
+ with BeforeAndAfterAll {
+
+ var tpcdsQueries = Seq[String]()
+
+
+ val conf =
+ new SparkConf()
+ .setMaster("local[*]")
+ .setAppName("test-sql-context")
+ .set("spark.driver.allowMultipleContexts", "true")
+ .set("spark.sql.shuffle.partitions", "4")
+ .set("spark.driver.memory", "1g")
+ .set("spark.executor.memory", "1g")
+ .set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString)
+
+ override def beforeAll(): Unit = {
+ super.beforeAll()
+ tpcdsQueries = Seq(
+ "q1", "q2", "q3", "q4", "q5", "q6", "q7", "q8", "q9", "q10", "q11",
+ "q12", "q13", "q14a", "q14b", "q15", "q16", "q17", "q18", "q19", "q20",
+ "q21", "q22", "q23a", "q23b", "q24a", "q24b", "q25", "q26", "q27", "q28", "q29", "q30",
+ "q31", "q32", "q33", "q34", "q35", "q36", "q37", "q38", "q39a", "q39b", "q40",
+ "q41", "q42", "q43", "q44", "q45", "q46", "q47", "q48", "q49", "q50",
+ "q51", "q52", "q53", "q54", "q55", "q56", "q57", "q58", "q59", "q60",
+ "q61", "q62", "q63", "q64", "q65", "q66", "q67", "q68", "q69", "q70",
+ "q71", "q72", "q73", "q74", "q75", "q76", "q77", "q78", "q79", "q80",
+ "q81", "q82", "q83", "q84", "q85", "q86", "q87", "q88", "q89", "q90",
+ "q91", "q92", "q93", "q94", "q95", "q96", "q97", "q98", "q99")
+ }
+
+ // Disabling the test run from precheckin as it takes around an hour.
+ // TODO : Add TPCDS tests to be run as a part of smokePerf bt which will run on a dedicated
+ // machine.
+
+ ignore("Test with Snappy") {
+ val sc = new SparkContext(conf)
+ TPCDSQuerySnappyBenchmark.snappy = new SnappySession(sc)
+ val dataLocation = "/export/shared/QA_DATA/TPCDS/data"
+ val snappyHome = System.getenv("SNAPPY_HOME")
+ val snappyRepo = s"$snappyHome/../../.."
+
+ TPCDSQuerySnappyBenchmark.execute(dataLocation,
+ queries = tpcdsQueries, true, s"$snappyRepo/spark/sql/core/src/test/resources/tpcds")
+ }
+
+ // Disabling the test run from precheckin as it takes around an hour.
+ // TODO : Add TPCDS tests to be run as a part of smokePerf bt which will run on a dedicated
+ // machine.
+
+ ignore("Test with Spark") {
+ TPCDSQuerySnappyBenchmark.spark = SparkSession.builder.config(conf).getOrCreate()
+ val dataLocation = "/export/shared/QA_DATA/TPCDS/data"
+ val snappyHome = System.getenv("SNAPPY_HOME")
+ val snappyRepo = s"$snappyHome/../../..";
+
+ TPCDSQuerySnappyBenchmark.execute(dataLocation,
+ queries = tpcdsQueries, false, s"$snappyRepo/spark/sql/core/src/test/resources/tpcds")
+
+ }
+
+ // Disabling the validation for now as this requires the expected result files to be created
+ // using stock spark before hand.
+
+ ignore("Validate Results") {
+
+ for (query <- tpcdsQueries) {
+
+ val actualResultsAvailableAt = "path for actual result"
+ val expectedResultsAvailableAt = "path for expected result"
+
+ val resultFileStream: FileOutputStream = new FileOutputStream(new File("Comparison.out"))
+ val resultOutputStream: PrintStream = new PrintStream(resultFileStream)
+
+ val expectedFile = sc.textFile(s"file://$expectedResultsAvailableAt/Spark_$query.out")
+ val actualFile = sc.textFile(s"file://$actualResultsAvailableAt/Snappy_$query.out")
+
+ val expectedLineSet = expectedFile.collect().toList.sorted
+ val actualLineSet = actualFile.collect().toList.sorted
+
+ if (!actualLineSet.equals(expectedLineSet)) {
+ if (!(expectedLineSet.size == actualLineSet.size)) {
+ resultOutputStream.println(s"For $query " +
+ s"result count mismatched observed with " +
+ s"expected ${expectedLineSet.size} and actual ${actualLineSet.size}")
+ } else {
+ for ((expectedLine, actualLine) <- expectedLineSet zip actualLineSet) {
+ if (!expectedLine.equals(actualLine)) {
+ resultOutputStream.println(s"For $query result mismatched observed")
+ resultOutputStream.println(s"Excpected : $expectedLine")
+ resultOutputStream.println(s"Found : $actualLine")
+ resultOutputStream.println(s"-------------------------------------")
+ }
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCHPerfComparer.java b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCHPerfComparer.java
index 6ee4449966..ec28be4479 100644
--- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCHPerfComparer.java
+++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCHPerfComparer.java
@@ -52,10 +52,10 @@ public static void main(String[] args) {
//treat first value are base and divide subsequent values with this base value and plot values
Path p = Paths.get(args[0]);
- final int maxDepth = 4;
+ final int maxDepth = 5;
List errorList = new ArrayList();
try {
- SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH-mm-ss") ;
+ SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH-mm-ss") ;
FileOutputStream reportOutputStream = new FileOutputStream(new File(p.toString(), "ComparisonReport_"+dateFormat.format(new Date())+".txt"));
PrintStream reportPrintStream = new PrintStream(reportOutputStream);
diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/QueryExecutionJob.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/QueryExecutionJob.scala
new file mode 100644
index 0000000000..1fa85a0103
--- /dev/null
+++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/QueryExecutionJob.scala
@@ -0,0 +1,144 @@
+package io.snappydata.benchmark.snappy.tpcds
+
+import java.io.{File, FileOutputStream, PrintStream}
+
+import org.apache.spark.Logging
+import com.typesafe.config.Config
+import io.snappydata.benchmark.TPCH_Queries
+import io.snappydata.benchmark.snappy.tpch.QueryExecutor
+
+import org.apache.spark.sql.catalyst.util.fileToString
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{Row, SnappyJobInvalid, SnappyJobValid, SnappyJobValidation, SnappySQLJob, SnappySession}
+
+object QueryExecutionJob extends SnappySQLJob with Logging{
+ var sqlSparkProperties: Array[String] = _
+ var queries: Array[String] = _
+ var queryPath: String = _
+ var isResultCollection: Boolean = _
+ var warmUp: Integer = _
+ var runsForAverage: Integer = _
+
+ def runSnappyJob(snSession: SnappySession, jobConfig: Config): Any = {
+ val snc = snSession.sqlContext
+
+ for (prop <- sqlSparkProperties) {
+ snc.sql(s"set $prop")
+ }
+
+ val avgFileStream: FileOutputStream = new FileOutputStream(
+ new File(s"Snappy_Average.out"))
+ val avgPrintStream: PrintStream = new PrintStream(avgFileStream)
+
+ queries.foreach { name =>
+ try {
+
+ val path: String = s"$queryPath/$name.sql"
+ val queryString = fileToString(new File(path))
+
+ val queryFileName = s"$name.out"
+
+ val queryFileStream: FileOutputStream = new FileOutputStream(new File(queryFileName))
+ val queryPrintStream: PrintStream = new PrintStream(queryFileStream)
+
+ var totalTime: Long = 0
+
+ // scalastyle:off println
+ //println("Query : " + queryString)
+
+ if (isResultCollection) {
+ // queryPrintStream.println(queryToBeExecuted)
+ val (resultSet, _) = QueryExecutor.queryExecution(name, queryString, snSession.sqlContext, true)
+ println(s"$name : ${resultSet.length}")
+
+ for (row <- resultSet) {
+ queryPrintStream.println(row.toSeq.map {
+ case d: Double => "%18.4f".format(d).trim()
+ case v => v
+ }.mkString(","))
+ }
+ println(s"$name Result Collected in file $queryFileName")
+ }else {
+ for (i <- 1 to (warmUp + runsForAverage)) {
+ // queryPrintStream.println(queryToBeExecuted)
+ val startTime = System.currentTimeMillis()
+ var cnts: Array[Row] = null
+ if (i == 1) {
+ cnts = QueryExecutor.queryExecution(name, queryString, snSession.sqlContext, true)._1
+ } else {
+ cnts = QueryExecutor.queryExecution(name, queryString, snSession.sqlContext)._1
+ }
+ for (s <- cnts) {
+ // just iterating over result
+ }
+ val endTime = System.currentTimeMillis()
+ val iterationTime = endTime - startTime
+ // scalastyle:off println
+ queryPrintStream.println(s"$iterationTime")
+
+ if (i > warmUp) {
+ totalTime += iterationTime
+ }
+ cnts = null
+ }
+ }
+
+ // scalastyle:off println
+ //println(s"${totalTime / runsForAverage}")
+ println("-----------------------------------------------")
+ queryPrintStream.println(s"${totalTime / runsForAverage}")
+ avgPrintStream.println(s"$name, executionTime = ${totalTime / runsForAverage}")
+ println("-----------------------------------------------")
+
+ }
+ catch {
+ case e: Exception => println(s"Failed $name ");
+ logError("Exception in job", e);
+ }
+ }
+ }
+
+ override def isValidJob(snSession: SnappySession, config: Config): SnappyJobValidation = {
+
+ val sqlSparkProps = if (config.hasPath("sparkSqlProps")) {
+ config.getString("sparkSqlProps")
+ }
+ else " "
+ sqlSparkProperties = sqlSparkProps.split(",")
+
+ val tempqueries = if (config.hasPath("queries")) {
+ config.getString("queries")
+ } else {
+ return SnappyJobInvalid("Specify Query number to be executed")
+ }
+ // scalastyle:off println
+ println(s"tempqueries : $tempqueries")
+ queries = tempqueries.split(",")
+
+ queryPath = if (config.hasPath("queryPath")) {
+ config.getString("queryPath")
+ } else {
+ ""
+ }
+
+ isResultCollection = if (config.hasPath("resultCollection")) {
+ config.getBoolean("resultCollection")
+ } else {
+ return SnappyJobInvalid("Specify whether to to collect results")
+ }
+
+ warmUp = if (config.hasPath("warmUpIterations")) {
+ config.getInt("warmUpIterations")
+ } else {
+ return SnappyJobInvalid("Specify number of warmup iterations ")
+ }
+ runsForAverage = if (config.hasPath("actualRuns")) {
+ config.getInt("actualRuns")
+ } else {
+ return SnappyJobInvalid("Specify number of iterations of which average result is " +
+ "calculated")
+ }
+
+ SnappyJobValid()
+ }
+}
diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/SparkApp.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/SparkApp.scala
new file mode 100644
index 0000000000..17c20d3d2c
--- /dev/null
+++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/SparkApp.scala
@@ -0,0 +1,214 @@
+/*
+ * Copyright (c) 2017 SnappyData, Inc. All rights reserved.
+ *
+ * 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. See accompanying
+ * LICENSE file.
+ */
+package io.snappydata.benchmark.snappy.tpcds
+
+import java.io.{File, FileOutputStream, PrintStream}
+
+import io.snappydata.benchmark.snappy.tpch.QueryExecutor
+
+import org.apache.spark.sql.catalyst.util.fileToString
+import org.apache.spark.sql.{Row, SparkSession}
+
+object SparkApp {
+
+ def main(args: Array[String]) {
+
+ val sc: SparkSession = SparkSession
+ .builder
+ .appName("TPCDS_Spark")
+ .getOrCreate()
+
+ val sparkSqlProps = args(0).split(",")
+ val dataLocation = args(1)
+ val queries = args(2).split(",").toSeq
+ val queryPath = args(3)
+ var buckets_ColumnTable = args(4).toInt
+ var isResultCollection = args(5).toBoolean
+ var warmUp = args(6).toInt
+ var runsForAverage = args(7).toInt
+
+ for (prop <- sparkSqlProps) {
+ // scalastyle:off println
+ println(prop)
+ sc.sql(s"set $prop")
+ }
+
+ val snc = sc.sqlContext
+
+ for (prop <- sparkSqlProps) {
+ snc.sql(s"set $prop")
+ }
+
+ // scalastyle:off println
+ println(s"****************queries : $queries")
+ // scalastyle:on println
+
+ /*catalog_page", "catalog_returns", "customer", "customer_address",
+ "customer_demographics", "date_dim", "household_demographics", "inventory", "item",
+ "promotion", "store", "store_returns", "catalog_sales", "web_sales", "store_sales",
+ "web_returns", "web_site", "reason", "call_center", "warehouse", "ship_mode", "income_band",
+ "time_dim", "web_page"*/
+
+ val tables = Seq("call_center", "catalog_page", "date_dim", "household_demographics",
+ "income_band", "promotion", "reason", "ship_mode", "store", "time_dim",
+ "warehouse", "web_page" , "web_site", "item", "customer_demographics")
+
+ tables.map { tableName =>
+
+ sc.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName)
+ snc.cacheTable(tableName)
+ val count = sc.table(tableName).count()
+ tableName -> count
+
+ // scalastyle:off println
+ println("-----------------------------------------------")
+ println(s"Table Created...$tableName with rows $count")
+ println("-----------------------------------------------")
+ }
+
+ /*
+ catalog_returns cr1.cr_order_number **cr_order_number**
+ catalog_sales (cs1.cs_order_number, cs_item_sk, cs_bill_customer_sk) **cs_order_number**
+ customer (c_customer_sk , c_current_addr_sk) **c_customer_sk**
+ customer_addr (ca_address_sk) **ca_address_sk**
+ inventory (i_item_sk) **i_item_sk**
+ store_returns (sr_item_sk, sr_customer_sk) **sr_item_k**
+ store_sales (customer_sk, address_sk, ss_item_sk) **ss_item_k**
+ web_returns wr.wr_order_number **wr_order_number**
+ web_sales (customer_sk, ws_bill_customer_sk , ws.ws_order_number) **.ws_order_number***
+ */
+
+ var partitionBy : String = "cr_order_number"
+ var tableName : String = "catalog_returns"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+ partitionBy = "cs_order_number"
+ tableName = "catalog_sales"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+ partitionBy = "c_customer_sk"
+ tableName = "customer"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+ partitionBy = "ca_address_sk"
+ tableName = "customer_address"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+ partitionBy = "inv_item_sk"
+ tableName = "inventory"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+ partitionBy = "sr_item_sk"
+ tableName = "store_returns"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+ partitionBy = "ss_item_sk"
+ tableName = "store_sales"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+ partitionBy = "wr_order_number"
+ tableName = "web_returns"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+ partitionBy = "ws_order_number"
+ tableName = "web_sales"
+ createPartitionedTables(sc, dataLocation, partitionBy, tableName, buckets_ColumnTable)
+
+
+ var avgFileStream: FileOutputStream = new FileOutputStream(
+ new File(s"Spark_Average.out"))
+ var avgPrintStream: PrintStream = new PrintStream(avgFileStream)
+
+ queries.foreach { name =>
+
+ try {
+
+ val path: String = s"$queryPath/$name.sql"
+ val queryString = fileToString(new File(path))
+
+ var totalTime: Long = 0
+
+ // scalastyle:off println
+ //println("Query : " + queryString)
+
+ for (i <- 1 to (warmUp + runsForAverage)) {
+ // queryPrintStream.println(queryToBeExecuted)
+ val startTime = System.currentTimeMillis()
+ var cnts: Array[Row] = null
+ if (i == 1) {
+ QueryExecutor.planPrintStream = avgPrintStream
+ cnts = QueryExecutor.queryExecution(name, queryString, sc.sqlContext, false)._1
+ } else {
+ cnts = QueryExecutor.queryExecution(name, queryString, sc.sqlContext)._1
+ }
+ for (s <- cnts) {
+ // just iterating over result
+ }
+ val endTime = System.currentTimeMillis()
+ val iterationTime = endTime - startTime
+
+ // scalastyle:off println
+ println(s"iterationTime = $iterationTime")
+
+ if (i > warmUp) {
+ totalTime += iterationTime
+ }
+ cnts = null
+ }
+
+ // scalastyle:off println
+ //println(s"${totalTime / runsForAverage}")
+ println("-----------------------------------------------")
+ avgPrintStream.println(s"$name, executionTime = ${totalTime / runsForAverage}")
+ println("-----------------------------------------------")
+
+ }
+ catch {
+ case e: Exception => println(s"Failed $name " + e.printStackTrace())
+ }
+ }
+
+// try {
+// Thread.sleep(Long.MaxValue)
+// }
+// catch {
+// case _ =>
+// }
+ //TPCDSQuerySnappyBenchmark.snappy = snSession
+ //TPCDSQuerySnappyBenchmark.execute(dataLocation, queries, true, queryPath)
+ }
+
+
+ def createPartitionedTables(sc: SparkSession, dataLocation: String,
+ partitionBy: String , tableName: String, buckets: Int): Unit = {
+ val df = sc.sqlContext.read.parquet(s"$dataLocation/$tableName")
+ df.createOrReplaceTempView(tableName)
+ df.repartition(buckets, df(partitionBy)).createOrReplaceTempView(tableName)
+ df.createOrReplaceTempView(tableName)
+ sc.sqlContext.cacheTable(tableName)
+ // tableName -> sc.table(tableName).count()
+ val count = sc.table(tableName).count()
+
+ // scalastyle:off println
+ println("-----------------------------------------------")
+ println(s"Table Created...$tableName with row $count")
+ println("-----------------------------------------------")
+ }
+
+ //TPCDSQuerySnappyBenchmark.spark = sc
+ //TPCDSQuerySnappyBenchmark.execute(dataLocation, queries, false, queryPath)
+}
diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/TableCreationJob.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/TableCreationJob.scala
new file mode 100644
index 0000000000..e48dff15d0
--- /dev/null
+++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/TableCreationJob.scala
@@ -0,0 +1,134 @@
+package io.snappydata.benchmark.snappy.tpcds
+
+import java.io.{File, FileOutputStream, PrintStream}
+
+import com.typesafe.config.Config
+import io.snappydata.benchmark.{TPCHColumnPartitionedTable, TPCHReplicatedTable}
+
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{SnappyJobValid, SnappyJobValidation, SnappySQLJob, SnappySession}
+
+object TableCreationJob extends SnappySQLJob{
+
+ var sqlSparkProperties: Array[String] = _
+ var dataLocation: String = _
+ var buckets_ColumnTable: String = _
+
+ override def runSnappyJob(snSession: SnappySession, jobConfig: Config): Any = {
+ val snc = snSession.sqlContext
+ snc.sparkContext.hadoopConfiguration.set("fs.s3a.connection.maximum", "1000");
+ val isSnappy = true
+
+ val loadPerfFileStream: FileOutputStream = new FileOutputStream(new File("Snappy_LoadPerf.out"))
+ val loadPerfPrintStream: PrintStream = new PrintStream(loadPerfFileStream)
+
+ for (prop <- sqlSparkProperties) {
+ snc.sql(s"set $prop")
+ }
+
+ val tables = Seq("call_center", "catalog_page", "date_dim", "household_demographics",
+ "income_band", "promotion", "reason", "ship_mode", "store", "time_dim",
+ "warehouse", "web_page" , "web_site", "item", "customer_demographics")
+
+ tables.map { tableName =>
+ //println(s"Table Creation Started...$tableName")
+ val df = snSession.read.parquet(s"$dataLocation/$tableName")
+ snSession.createTable(tableName, "row",
+ new StructType(df.schema.map(_.copy(nullable = true)).toArray),
+ Map[String, String] ())
+ df.write.insertInto(tableName)
+ val cnt = df.collect().length;
+ // scalastyle:off println
+ println("-----------------------------------------------")
+ println(s"Table Created...$tableName with rows $cnt")
+
+ println("-----------------------------------------------")
+ }
+
+
+ var props = Map(("PARTITION_BY" -> "cr_order_number"), ("BUCKETS" -> buckets_ColumnTable))
+ var tableName = "catalog_returns"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+ props = Map(("PARTITION_BY" -> "cs_order_number"), ("BUCKETS" -> buckets_ColumnTable),
+ ("COLOCATE_WITH" -> "CATALOG_RETURNS"))
+ tableName = "catalog_sales"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+ props = Map(("PARTITION_BY" -> "wr_order_number"), ("BUCKETS" -> buckets_ColumnTable),
+ ("COLOCATE_WITH" -> "CATALOG_SALES"))
+ tableName = "web_returns"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+ props = Map(("PARTITION_BY" -> "ws_order_number"), ("BUCKETS" -> buckets_ColumnTable),
+ ("COLOCATE_WITH" -> "WEB_RETURNS"))
+ tableName = "web_sales"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+
+ props = Map(("PARTITION_BY" -> "inv_item_sk"), ("BUCKETS" -> buckets_ColumnTable))
+ tableName = "inventory"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+ props = Map(("PARTITION_BY" -> "sr_item_sk"), ("BUCKETS" -> buckets_ColumnTable),
+ ("COLOCATE_WITH" -> "INVENTORY"))
+ tableName = "store_returns"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+ props = Map(("PARTITION_BY" -> "ss_item_sk"), ("BUCKETS" -> buckets_ColumnTable),
+ ("COLOCATE_WITH" -> "STORE_RETURNS"))
+ tableName = "store_sales"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+
+ props = Map(("PARTITION_BY" -> "c_customer_sk"), ("BUCKETS" -> buckets_ColumnTable))
+ tableName = "customer"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+ props = Map(("PARTITION_BY" -> "ca_address_sk"), ("BUCKETS" -> buckets_ColumnTable))
+ tableName = "customer_address"
+ createColumnPartitionedTables(snSession, props, tableName)
+
+ val avgFileStream: FileOutputStream = new FileOutputStream(
+ new File(s"Snappy_Average.out"))
+ val avgPrintStream: PrintStream = new PrintStream(avgFileStream)
+
+ }
+
+ def createColumnPartitionedTables(snappy: SnappySession,
+ props: Map[String,String] , tableName: String): Unit = {
+
+ val df = snappy.read.parquet(s"$dataLocation/$tableName")
+ snappy.createTable(tableName, "column",
+ new StructType(df.schema.map(_.copy(nullable = false)).toArray), props)
+ df.write.insertInto(tableName)
+ val cnt = df.collect().length
+ // scalastyle:off println
+ println("-----------------------------------------------")
+ println(s"Table Created...$tableName with rows $cnt")
+ println("-----------------------------------------------")
+ }
+
+ override def isValidJob(snSession: SnappySession, config: Config): SnappyJobValidation = {
+
+ val sqlSparkProps = if (config.hasPath("sparkSqlProps")) {
+ config.getString("sparkSqlProps")
+ }
+ else " "
+ sqlSparkProperties = sqlSparkProps.split(",")
+
+ dataLocation = if (config.hasPath("dataDir")) {
+ config.getString("dataDir")
+ } else {
+ "/QASNAPPY/TPCH/DATA/1"
+ }
+
+ buckets_ColumnTable = if (config.hasPath("Buckets_ColumnTable")) {
+ config.getString("Buckets_ColumnTable")
+ } else {
+ "15"
+ }
+
+ SnappyJobValid()
+ }
+}
diff --git a/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala b/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala
index c54c21946c..8e40f4585a 100644
--- a/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala
+++ b/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala
@@ -18,11 +18,10 @@ package org.apache.spark.sql
import java.util.TimeZone
-import com.pivotal.gemfirexd.internal.engine.db.FabricDatabase
import io.snappydata.benchmark.TPCH_Queries
import io.snappydata.benchmark.snappy.tpch.QueryExecutor
import io.snappydata.benchmark.snappy.{SnappyAdapter, TPCH}
-import io.snappydata.{PlanTest, SnappyFunSuite}
+import io.snappydata.{PlanTest, Property, SnappyFunSuite}
import org.scalatest.BeforeAndAfterEach
import org.apache.spark.sql.catalyst.TableIdentifier
@@ -33,14 +32,11 @@ import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
import org.apache.spark.util.Benchmark
class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach {
- var existingSkipSPSCompile = false
override def beforeAll(): Unit = {
// System.setProperty("org.codehaus.janino.source_debugging.enable", "true")
System.setProperty("spark.sql.codegen.comments", "true")
System.setProperty("spark.testing", "true")
- existingSkipSPSCompile = FabricDatabase.SKIP_SPS_PRECOMPILE
- FabricDatabase.SKIP_SPS_PRECOMPILE = true
super.beforeAll()
}
@@ -48,8 +44,7 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach {
// System.clearProperty("org.codehaus.janino.source_debugging.enable")
System.clearProperty("spark.sql.codegen.comments")
System.clearProperty("spark.testing")
- System.clearProperty("DISABLE_PARTITION_PRUNING")
- FabricDatabase.SKIP_SPS_PRECOMPILE = existingSkipSPSCompile
+ Property.PartitionPruning.set(snc.conf, true)
super.afterAll()
}
@@ -203,8 +198,8 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach {
}
}
- private def togglePruning(onOff: Boolean) =
- System.setProperty("DISABLE_PARTITION_PRUNING", onOff.toString)
+ private def togglePruning(onOff: Boolean, snc: SnappyContext) =
+ Property.PartitionPruning.set(snc.conf, onOff)
def runBenchmark(queryString: String, tableSizes: Map[String, Long], numSecs: Int = 0): Unit = {
@@ -236,9 +231,9 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach {
val b = new Benchmark(s"JoinOrder optimization", size,
warmupTime = numSecs.seconds)
b.addCase("WithOut Partition Pruning",
- prepare = () => togglePruning(true))(_ => snc.sql(queryString).collect().foreach(_ => ()))
+ prepare = () => togglePruning(onOff = false, snc))(_ => snc.sql(queryString).collect())
b.addCase("With Partition Pruning",
- prepare = () => togglePruning(false))(_ => snc.sql(queryString).collect().foreach(_ => ()))
+ prepare = () => togglePruning(onOff = true, snc))(_ => snc.sql(queryString).collect())
b.run()
}
@@ -276,9 +271,9 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach {
// b.addCase(s"$qNum baseTPCH index = F", prepare = case1)(i => evalBaseTPCH)
// b.addCase(s"$qNum baseTPCH joinOrder = T", prepare = case2)(i => evalBaseTPCH)
b.addCase(s"$qNum without PartitionPruning",
- prepare = () => togglePruning(true))(_ => evalSnappyMods(false))
+ prepare = () => togglePruning(onOff = false, snc))(_ => evalSnappyMods(false))
b.addCase(s"$qNum with PartitionPruning",
- prepare = () => togglePruning(false))(_ => evalSnappyMods(false))
+ prepare = () => togglePruning(onOff = true, snc))(_ => evalSnappyMods(false))
/*
b.addCase(s"$qNum snappyMods joinOrder = T", prepare = case2)(i => evalSnappyMods(false))
b.addCase(s"$qNum baseTPCH index = T", prepare = case3)(i => evalBaseTPCH)
diff --git a/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala b/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala
index 47bfb3c938..8d87f07716 100644
--- a/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala
+++ b/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala
@@ -16,7 +16,6 @@
*/
package org.apache.spark.sql
-import com.pivotal.gemfirexd.internal.engine.db.FabricDatabase
import io.snappydata.benchmark.TPCHColumnPartitionedTable
import io.snappydata.{PlanTest, SnappyFunSuite}
import org.scalatest.BeforeAndAfterEach
@@ -26,14 +25,11 @@ import org.apache.spark.sql.collection.MultiBucketExecutorPartition
import org.apache.spark.sql.execution.columnar.ColumnTableScan
class SingleNodeTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach {
- var existingSkipSPSCompile = false
override def beforeAll(): Unit = {
// System.setProperty("org.codehaus.janino.source_debugging.enable", "true")
System.setProperty("spark.sql.codegen.comments", "true")
System.setProperty("spark.testing", "true")
- existingSkipSPSCompile = FabricDatabase.SKIP_SPS_PRECOMPILE
- FabricDatabase.SKIP_SPS_PRECOMPILE = true
super.beforeAll()
}
@@ -41,7 +37,6 @@ class SingleNodeTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEac
// System.clearProperty("org.codehaus.janino.source_debugging.enable")
System.clearProperty("spark.sql.codegen.comments")
System.clearProperty("spark.testing")
- FabricDatabase.SKIP_SPS_PRECOMPILE = existingSkipSPSCompile
super.afterAll()
}
@@ -87,7 +82,7 @@ object SingleNodeTest {
val tpchDataPath = TPCHColumnPartitionedTable.getClass.getResource("/TPCH").getPath
val buckets_Order_Lineitem = "5"
TPCHColumnPartitionedTable.createPopulateOrderTable(snc, tpchDataPath,
- true, buckets_Order_Lineitem, null)
+ isSnappy = true, buckets_Order_Lineitem, null)
def validateSinglePartition(df: DataFrame, bucketId: Int): Unit = {
val scanRDD = df.queryExecution.executedPlan.collectFirst {
diff --git a/cluster/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala b/cluster/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala
index 7f9b97b219..a6a427bd6e 100644
--- a/cluster/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala
+++ b/cluster/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala
@@ -18,7 +18,6 @@ package org.apache.spark.sql
import io.snappydata.SnappyFunSuite
-import org.apache.spark.sql.catalyst.util.stackTraceToString
import org.apache.spark.sql.test.SQLTestData.TestData2
class SnappySQLQuerySuite extends SnappyFunSuite {
@@ -49,30 +48,6 @@ class SnappySQLQuerySuite extends SnappyFunSuite {
)
}
- protected def checkAnswer(df: => DataFrame, expectedAnswer: Seq[Row]): Unit = {
- val analyzedDF = try df catch {
- case ae: AnalysisException =>
- if (ae.plan.isDefined) {
- fail(
- s"""
- |Failed to analyze query: $ae
- |${ae.plan.get}
- |
- |${stackTraceToString(ae)}
- |""".stripMargin)
- } else {
- throw ae
- }
- }
-
- assertEmptyMissingInput(analyzedDF)
-
- QueryTest.checkAnswer(analyzedDF, expectedAnswer) match {
- case Some(errorMessage) => fail(errorMessage)
- case None =>
- }
- }
-
test("SNAP-1884 Join with temporary table not returning rows") {
val df = snc.createDataFrame(snc.sparkContext.parallelize(
LowerCaseData(1, "a") ::
@@ -187,19 +162,6 @@ class SnappySQLQuerySuite extends SnappyFunSuite {
session.dropTable("l", ifExists = true)
session.dropTable("r", ifExists = true)
}
-
- /**
- * Asserts that a given [[Dataset]] does not have missing inputs in all the analyzed plans.
- */
- def assertEmptyMissingInput(query: Dataset[_]): Unit = {
- assert(query.queryExecution.analyzed.missingInput.isEmpty,
- s"The analyzed logical plan has missing inputs:\n${query.queryExecution.analyzed}")
- assert(query.queryExecution.optimizedPlan.missingInput.isEmpty,
- s"The optimized logical plan has missing inputs:\n${query.queryExecution.optimizedPlan}")
- assert(query.queryExecution.executedPlan.missingInput.isEmpty,
- s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}")
- }
-
}
case class LowerCaseData(n: Int, l: String)
diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnCacheBenchmark.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnCacheBenchmark.scala
index 2735ec3976..0170819208 100644
--- a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnCacheBenchmark.scala
+++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnCacheBenchmark.scala
@@ -40,7 +40,6 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl
import io.snappydata.SnappyFunSuite
import org.apache.spark.SparkConf
-import org.apache.spark.memory.SnappyUnifiedMemoryManager
import org.apache.spark.sql._
import org.apache.spark.sql.execution.benchmark.ColumnCacheBenchmark.addCaseWithCleanup
import org.apache.spark.sql.internal.SQLConf
@@ -62,22 +61,8 @@ class ColumnCacheBenchmark extends SnappyFunSuite {
}
override protected def newSparkConf(
- addOn: SparkConf => SparkConf = null): SparkConf = {
- val conf = new SparkConf()
- .setIfMissing("spark.master", s"local[$cores]")
- .setAppName("microbenchmark")
- conf.set("snappydata.store.critical-heap-percentage", "95")
- if (SnappySession.isEnterpriseEdition) {
- conf.set("snappydata.store.memory-size", "1200m")
- }
- conf.set("spark.memory.manager", classOf[SnappyUnifiedMemoryManager].getName)
- conf.set("spark.serializer", "org.apache.spark.serializer.PooledKryoSerializer")
- conf.set("spark.closure.serializer", "org.apache.spark.serializer.PooledKryoSerializer")
- if (addOn != null) {
- addOn(conf)
- }
- conf
- }
+ addOn: SparkConf => SparkConf = null): SparkConf =
+ TAQTest.newSparkConf(addOn)
private lazy val sparkSession = new SparkSession(sc)
private lazy val snappySession = snc.snappySession
@@ -340,6 +325,14 @@ class ColumnCacheBenchmark extends SnappyFunSuite {
val max = counts.max
assert(max - min <= 800000, "Unexpectedly large data skew: " +
results.map(r => s"${r.getInt(1)}=${r.getLong(0)}").mkString(","))
+ // check for SNAP-2200 by forcing overflow with updates
+ snappySession.sql("update test set id = id + 1")
+ snappySession.sql("update test set k = k + 1.0")
+ ColumnCacheBenchmark.collect(snappySession.sql(
+ "select max(id), min(id) from test"), Seq(Row(size, 1L)))
+ // repopulate for the benchmark test
+ snappySession.sql("truncate table test")
+ testDF2.write.insertInto("test")
ColumnCacheBenchmark.collect(snappySession.sql(query), expectedAnswer2)
} else {
ColumnCacheBenchmark.collect(sparkSession.sql(query), expectedAnswer)
diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala
index a75643959d..9fe53a4f46 100644
--- a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala
+++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala
@@ -62,7 +62,7 @@ class TAQTest extends SnappyFunSuite {
op = CreateOp.Trade, runSparkCaching = false)
}
- test("select queries with random data (eviction) - query") {
+ test("select queries with random data - query") {
val quoteSize = 3400000L
val tradeSize = 500000L
val numDays = 1
@@ -273,7 +273,7 @@ object TAQTest extends Logging with Assertions {
}
def newSparkConf(addOn: SparkConf => SparkConf = null): SparkConf = {
- val cores = math.min(8, Runtime.getRuntime.availableProcessors())
+ val cores = math.min(16, Runtime.getRuntime.availableProcessors())
val conf = new SparkConf()
.setIfMissing("spark.master", s"local[$cores]")
.setAppName("microbenchmark")
diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQuerySnappyBenchmark.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQuerySnappyBenchmark.scala
new file mode 100644
index 0000000000..48e33efda1
--- /dev/null
+++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQuerySnappyBenchmark.scala
@@ -0,0 +1,154 @@
+/*
+ * Copyright (c) 2017 SnappyData, Inc. All rights reserved.
+ *
+ * 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. See accompanying
+ * LICENSE file.
+ */
+
+package org.apache.spark.sql.execution.benchmark
+
+import java.io.{File, FileOutputStream, PrintStream}
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.util.fileToString
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Benchmark
+import org.apache.spark.{SparkConf, SparkContext}
+
+import scala.collection.mutable.ArrayBuffer
+
+
+object TPCDSQuerySnappyBenchmark {
+
+ var spark: SparkSession = _
+ var snappy: SnappySession = _
+ var ds: DataFrame = _
+
+ val tables = Seq("catalog_page", "catalog_returns", "customer", "customer_address",
+ "customer_demographics", "date_dim", "household_demographics", "inventory", "item",
+ "promotion", "store", "store_returns", "catalog_sales", "web_sales", "store_sales",
+ "web_returns", "web_site", "reason", "call_center", "warehouse", "ship_mode", "income_band",
+ "time_dim", "web_page")
+
+ var snappyRS: FileOutputStream = new FileOutputStream(
+ new File(s"Snappy_Results.out"))
+ var sparkRS : FileOutputStream = new FileOutputStream(
+ new File(s"Spark_Results.out"))
+
+ var snappyPS: PrintStream = new PrintStream(snappyRS)
+ var sparkPS: PrintStream = new PrintStream(sparkRS)
+
+ def setupTables(dataLocation: String, isSnappy: Boolean): Map[String, Long] = {
+ val props = Map("BUCKETS" -> "7")
+
+ tables.map { tableName =>
+ if (isSnappy) {
+
+ val df = snappy.read.parquet(s"$dataLocation/$tableName")
+ snappy.createTable(tableName, "column",
+ new StructType(df.schema.map(_.copy(nullable = false)).toArray), props)
+ df.write.insertInto(tableName)
+
+ // scalastyle:off println
+ println("Table Created..."+ tableName)
+ tableName -> snappy.table(tableName).count()
+ }
+ else {
+ spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName)
+ spark.sqlContext.cacheTable(tableName)
+ tableName -> spark.table(tableName).count()
+ }
+ }.toMap
+ }
+
+ def execute(dataLocation: String, queries: Seq[String], isSnappy: Boolean = false,
+ queryPath: String = ""): Unit = {
+
+ require(dataLocation.nonEmpty,
+ "please modify the value of dataLocation to point to your local TPCDS data")
+ val tableSizes = setupTables(dataLocation, isSnappy)
+
+ queries.foreach { name =>
+
+ val path: String = s"$queryPath/$name.sql"
+ val queryString = fileToString(new File(path))
+
+ // This is an indirect hack to estimate the size of each query's input by traversing the
+ // logical plan and adding up the sizes of all tables that appear in the plan. Note that this
+ // currently doesn't take WITH subqueries into account which might lead to fairly inaccurate
+ // per-row processing time for those cases.
+ try {
+ val queryRelations = scala.collection.mutable.HashSet[String]()
+
+ if (isSnappy) {
+ ds = snappy.sqlContext.sql(queryString)
+ //println("Plan..."+ ds.queryExecution.executedPlan)
+ }
+ else
+ ds = spark.sql(queryString)
+
+ ds.queryExecution.logical.map {
+ case ur@UnresolvedRelation(t: TableIdentifier, _) =>
+ queryRelations.add(t.table)
+ case lp: LogicalPlan =>
+ lp.expressions.foreach {
+ _ foreach {
+ case subquery: SubqueryExpression =>
+ subquery.plan.foreach {
+ case ur@UnresolvedRelation(t: TableIdentifier, _) =>
+ queryRelations.add(t.table)
+ case _ =>
+ }
+ case _ =>
+ }
+ }
+ case _ =>
+ }
+
+ val numRows = queryRelations.map(tableSizes.getOrElse(_, 0L)).sum
+ val benchmark = new Benchmark(s"TPCDS Snappy", numRows, 5)
+
+ benchmark.addCase(name) { i =>
+
+ if (isSnappy) {
+ val rs = snappy.sqlContext.sql(queryString).collect()
+ // snappyPS = new PrintStream(new FileOutputStream(new File(s"Snappy_$name.out")))
+ // normalizeRows(rs, snappyPS)
+ }
+ else {
+ val rs = spark.sql(queryString).collect()
+ //sparkPS = new PrintStream(new FileOutputStream(new File(s"Spark_$name.out")))
+ //normalizeRows(rs, sparkPS)
+ }
+ }
+ benchmark.run()
+
+ } catch {
+ case e: Exception => println(s"Failed $name " + e.printStackTrace())
+ }
+ }
+ }
+
+ private def normalizeRows(resultSet: Array[Row], printStream: PrintStream): Unit = {
+ for (row <- resultSet) {
+ printStream.println(row.toSeq.map {
+ // case d: Double => "%18.4f".format(d).trim()
+ case v => v
+ }.mkString("|"))
+ }
+ }
+}
diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala
index c6437707a9..8e2385e806 100644
--- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala
+++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala
@@ -74,6 +74,10 @@ class ColumnUpdateDeleteTest extends ColumnTablesTestBase {
ColumnUpdateDeleteTests.testConcurrentOps(this.snc.snappySession)
}
+ test("SNAP-2124 update missed") {
+ ColumnUpdateDeleteTests.testSNAP2124(this.snc.snappySession, checkPruning = true)
+ }
+
ignore("test update for all types") {
val session = this.snc.snappySession
// reduced size to ensure both column table and row buffer have data
diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTest.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTest.scala
index fd2795a245..fb89390445 100644
--- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTest.scala
+++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTest.scala
@@ -18,7 +18,7 @@ package io.snappydata.cluster
import java.io.PrintWriter
import java.nio.file.{Files, Paths}
-import java.sql.{Blob, Clob, Connection, DriverManager, ResultSet, Statement, Timestamp}
+import java.sql.{Blob, Clob, Connection, DriverManager, ResultSet, SQLException, Statement, Timestamp}
import java.util.Properties
import scala.collection.JavaConverters._
@@ -26,6 +26,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.language.{implicitConversions, postfixOps}
import scala.sys.process._
import scala.util.Random
+
import com.fasterxml.jackson.databind.ObjectMapper
import com.pivotal.gemfirexd.Attribute
import com.pivotal.gemfirexd.snappy.ComplexTypeSerializer
@@ -33,6 +34,7 @@ import io.snappydata.Constant
import io.snappydata.test.dunit.{AvailablePortHelper, DistributedTestBase, Host, VM}
import io.snappydata.util.TestUtils
import org.junit.Assert
+
import org.apache.spark.sql.SnappyContext
import org.apache.spark.sql.types.Decimal
import org.apache.spark.util.collection.OpenHashSet
@@ -209,6 +211,30 @@ object SplitClusterDUnitTest extends SplitClusterDUnitTestObject {
selectFromTableUsingJDBC("splitModeTable1", 1005, stmt)
}
+ // check for SNAP-2156/2164
+ var updateSql = "update splitModeTable1 set col1 = 100 where exists " +
+ s"(select 1 from splitModeTable1 t where t.col1 = splitModeTable1.col1 and t.col1 = 1234)"
+ assert(!stmt.execute(updateSql))
+ assert(stmt.getUpdateCount >= 0) // random value can be 1234
+ assert(stmt.executeUpdate(updateSql) == 0)
+ updateSql = "update splitModeTable1 set col1 = 100 where exists " +
+ s"(select 1 from splitModeTable1 t where t.col1 = splitModeTable1.col1 and t.col1 = 1)"
+ assert(!stmt.execute(updateSql))
+ assert(stmt.getUpdateCount >= 1)
+ assert(stmt.executeUpdate(updateSql) == 0)
+ updateSql = "update splitModeTable1 set col1 = 1 where exists " +
+ s"(select 1 from splitModeTable1 t where t.col1 = splitModeTable1.col1 and t.col1 = 100)"
+ assert(stmt.executeUpdate(updateSql) >= 1)
+ assert(!stmt.execute(updateSql))
+ assert(stmt.getUpdateCount == 0)
+
+ // check exception should be proper (SNAP-1423/1386)
+ try {
+ stmt.execute("call sys.rebalance_all_bickets()")
+ } catch {
+ case sqle: SQLException if sqle.getSQLState == "42Y03" => // ignore
+ }
+
stmt.execute("drop table if exists embeddedModeTable1")
stmt.execute("drop table if exists embeddedModeTable2")
stmt.execute("drop table if exists splitModeTable1")
diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala
index 4b7f45bba5..0b769d693f 100644
--- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala
+++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala
@@ -173,6 +173,7 @@ trait SplitClusterDUnitTestBase extends Logging {
ColumnUpdateDeleteTests.testSNAP1925(session)
ColumnUpdateDeleteTests.testSNAP1926(session)
ColumnUpdateDeleteTests.testConcurrentOps(session)
+ ColumnUpdateDeleteTests.testSNAP2124(session, checkPruning = false)
}
})
}
diff --git a/core/src/main/scala/io/snappydata/Literals.scala b/core/src/main/scala/io/snappydata/Literals.scala
index 5cf09283e2..01301235b7 100644
--- a/core/src/main/scala/io/snappydata/Literals.scala
+++ b/core/src/main/scala/io/snappydata/Literals.scala
@@ -277,6 +277,10 @@ object Property extends Enumeration {
"scalability of queries in the interest of reduced memory usage for " +
"secondary buckets. Default is false.", Some(false), Constant.SPARK_PREFIX)
+ val PartitionPruning: SQLValue[Boolean] = SQLVal[Boolean](
+ s"${Constant.PROPERTY_PREFIX}sql.partitionPruning",
+ "Property to set/unset partition pruning of queries", Some(true))
+
val PlanCaching: SQLValue[Boolean] = SQLVal[Boolean](
s"${Constant.PROPERTY_PREFIX}sql.planCaching",
"Property to set/unset plan caching", Some(true))
diff --git a/core/src/main/scala/io/snappydata/impl/SparkConnectorRDDHelper.scala b/core/src/main/scala/io/snappydata/impl/SparkConnectorRDDHelper.scala
index bdc407662b..c08701e3af 100644
--- a/core/src/main/scala/io/snappydata/impl/SparkConnectorRDDHelper.scala
+++ b/core/src/main/scala/io/snappydata/impl/SparkConnectorRDDHelper.scala
@@ -74,23 +74,22 @@ final class SparkConnectorRDDHelper {
s"${ColumnFormatEntry.STATROW_COL_INDEX}", fetchColString)
}
- def executeQuery(conn: Connection, tableName: String,
- split: Partition, query: String, relDestroyVersion: Int): (Statement, ResultSet, String) = {
- val partition = split.asInstanceOf[SmartExecutorBucketPartition]
+ def executeQuery(conn: Connection, tableName: String, partition: SmartExecutorBucketPartition,
+ query: String, relDestroyVersion: Int): (Statement, ResultSet, String) = {
val statement = conn.createStatement()
val txId = SparkConnectorRDDHelper.snapshotTxIdForRead.get() match {
- case "null" => null
+ case "" => null
case id => id
}
statement match {
case clientStmt: ClientStatement =>
- val bucketSet = Collections.singleton(Int.box(partition.index))
+ val bucketSet = Collections.singleton(Int.box(partition.bucketId))
clientStmt.setLocalExecutionBucketIds(bucketSet, tableName, true)
clientStmt.setMetadataVersion(relDestroyVersion)
clientStmt.setSnapshotTransactionId(txId)
case _ =>
statement.execute("call sys.SET_BUCKETS_FOR_LOCAL_EXECUTION(" +
- s"'$tableName', '${partition.index}', $relDestroyVersion)")
+ s"'$tableName', '${partition.bucketId}', $relDestroyVersion)")
if (txId ne null) {
statement.execute(s"call sys.USE_SNAPSHOT_TXID('$txId')")
}
@@ -101,9 +100,8 @@ final class SparkConnectorRDDHelper {
}
def getConnection(connectionProperties: ConnectionProperties,
- split: Partition): Connection = {
- val urlsOfNetServerHost = split.asInstanceOf[
- SmartExecutorBucketPartition].hostList
+ part: SmartExecutorBucketPartition): Connection = {
+ val urlsOfNetServerHost = part.hostList
useLocatorURL = SparkConnectorRDDHelper.useLocatorUrl(urlsOfNetServerHost)
createConnection(connectionProperties, urlsOfNetServerHost)
}
@@ -153,7 +151,7 @@ object SparkConnectorRDDHelper {
val numPartitions = bucketToServerList.length
val partitions = new Array[Partition](numPartitions)
for (p <- 0 until numPartitions) {
- partitions(p) = new SmartExecutorBucketPartition(p, bucketToServerList(p))
+ partitions(p) = new SmartExecutorBucketPartition(p, p, bucketToServerList(p))
}
partitions
}
diff --git a/core/src/main/scala/org/apache/spark/memory/DefaultMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/DefaultMemoryManager.scala
index bd48c55c90..7803ab3af6 100644
--- a/core/src/main/scala/org/apache/spark/memory/DefaultMemoryManager.scala
+++ b/core/src/main/scala/org/apache/spark/memory/DefaultMemoryManager.scala
@@ -137,4 +137,9 @@ class DefaultMemoryManager extends StoreUnifiedManager with Logging {
override def clear(): Unit = {
memoryForObject.clear()
}
+
+ /**
+ * Initializes the memoryManager
+ */
+ override def init(): Unit = {}
}
diff --git a/core/src/main/scala/org/apache/spark/memory/StoreUnifiedManager.scala b/core/src/main/scala/org/apache/spark/memory/StoreUnifiedManager.scala
index 64b7e0d7cb..261544af60 100644
--- a/core/src/main/scala/org/apache/spark/memory/StoreUnifiedManager.scala
+++ b/core/src/main/scala/org/apache/spark/memory/StoreUnifiedManager.scala
@@ -86,6 +86,11 @@ trait StoreUnifiedManager {
* Closes the memory manager.
*/
def close()
+
+ /**
+ * Initializes the memoryManager
+ */
+ def init()
}
diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyBaseParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappyBaseParser.scala
index 3cb7f8b9b5..a146d2832d 100644
--- a/core/src/main/scala/org/apache/spark/sql/SnappyBaseParser.scala
+++ b/core/src/main/scala/org/apache/spark/sql/SnappyBaseParser.scala
@@ -481,11 +481,12 @@ object SnappyParserConsts {
final val WEEK: Keyword = nonReservedKeyword("week")
final val YEAR: Keyword = nonReservedKeyword("year")
- // cube, rollup, grouping sets are not reserved
+ // cube, rollup, grouping sets etc are not reserved
final val CUBE: Keyword = nonReservedKeyword("cube")
final val ROLLUP: Keyword = nonReservedKeyword("rollup")
final val GROUPING: Keyword = nonReservedKeyword("grouping")
final val SETS: Keyword = nonReservedKeyword("sets")
+ final val LATERAL: Keyword = nonReservedKeyword("lateral")
// datatypes are not reserved
final val ARRAY: Keyword = nonReservedKeyword("array")
diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala
index d753c1ca5a..849d958a91 100644
--- a/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala
+++ b/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala
@@ -183,11 +183,12 @@ abstract class SnappyDDLParser(session: SparkSession)
final def WEEK: Rule0 = rule { intervalUnit(Consts.WEEK) }
final def YEAR: Rule0 = rule { intervalUnit(Consts.YEAR) }
- // cube, rollup, grouping sets
+ // cube, rollup, grouping sets etc
final def CUBE: Rule0 = rule { keyword(Consts.CUBE) }
final def ROLLUP: Rule0 = rule { keyword(Consts.ROLLUP) }
final def GROUPING: Rule0 = rule { keyword(Consts.GROUPING) }
final def SETS: Rule0 = rule { keyword(Consts.SETS) }
+ final def LATERAL: Rule0 = rule { keyword(Consts.LATERAL) }
// DDLs, SET, SHOW etc
diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala
index 55fdd3c574..c51e978171 100644
--- a/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala
+++ b/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala
@@ -752,11 +752,15 @@ class SnappyParser(session: SnappySession) extends SnappyDDLParser(session) {
}
protected final def relations: Rule1[LogicalPlan] = rule {
- (relation + commaSep) ~> ((joins: Seq[LogicalPlan]) =>
- if (joins.size == 1) joins.head
+ (relation + commaSep) ~ lateralView.* ~> ((joins: Seq[LogicalPlan], views: Any) => {
+ val from = if (joins.size == 1) joins.head
else joins.tail.foldLeft(joins.head) {
case (lhs, rel) => Join(lhs, rel, Inner, None)
- })
+ }
+ views.asInstanceOf[Seq[LogicalPlan => LogicalPlan]].foldLeft(from) {
+ case (child, view) => view(child)
+ }
+ })
}
protected final def keyWhenThenElse: Rule1[WhenElseType] = rule {
@@ -963,8 +967,24 @@ class SnappyParser(session: SnappySession) extends SnappyDDLParser(session) {
).*
}
+ protected final def lateralView: Rule1[LogicalPlan => LogicalPlan] = rule {
+ LATERAL ~ VIEW ~ (OUTER ~ push(true)).? ~ functionIdentifier ~ '(' ~ ws ~
+ (expression * commaSep) ~ ')' ~ ws ~ identifier ~ (AS.? ~ (identifier + commaSep)).? ~>
+ ((o: Any, functionName: FunctionIdentifier, e: Any, tableName: String,
+ cols: Any) => (child: LogicalPlan) => {
+ val expressions = e.asInstanceOf[Seq[Expression]]
+ val columnNames = cols.asInstanceOf[Option[Seq[String]]] match {
+ case Some(s) => s.map(UnresolvedAttribute.apply)
+ case None => Seq.empty
+ }
+ Generate(UnresolvedGenerator(functionName, expressions), join = true,
+ outer = o.asInstanceOf[Option[Boolean]].isDefined, Some(tableName),
+ columnNames, child)
+ })
+ }
+
protected final def insert: Rule1[LogicalPlan] = rule {
- INSERT ~ ((OVERWRITE ~> (() => true)) | (INTO ~> (() => false))) ~
+ INSERT ~ ((OVERWRITE ~ push(true)) | (INTO ~ push(false))) ~
TABLE.? ~ relationFactor ~ subSelectQuery ~> ((o: Boolean, r: LogicalPlan,
s: LogicalPlan) => new Insert(r, Map.empty[String,
Option[String]], s, OverwriteOptions(o), ifNotExists = false))
@@ -978,8 +998,8 @@ class SnappyParser(session: SnappySession) extends SnappyDDLParser(session) {
protected final def update: Rule1[LogicalPlan] = rule {
UPDATE ~ relationFactor ~ SET ~ TOKENIZE_BEGIN ~ (((identifier + ('.' ~ ws)) ~
'=' ~ ws ~ expression ~> ((cols: Seq[String], e: Expression) =>
- UnresolvedAttribute(cols) -> e)) + commaSep) ~
- (FROM ~ relations).? ~ (WHERE ~ expression).? ~ TOKENIZE_END ~>
+ UnresolvedAttribute(cols) -> e)) + commaSep) ~ TOKENIZE_END ~
+ (FROM ~ relations).? ~ (WHERE ~ TOKENIZE_BEGIN ~ expression ~ TOKENIZE_END).? ~>
((t: Any, updateExprs: Seq[(UnresolvedAttribute,
Expression)], relations : Any, whereExpr: Any) => {
val table = t.asInstanceOf[LogicalPlan]
@@ -1027,8 +1047,7 @@ class SnappyParser(session: SnappySession) extends SnappyDDLParser(session) {
private var canTokenize = false
protected final def TOKENIZE_BEGIN: Rule0 = rule {
- MATCH ~> (() => tokenize = session.planCaching &&
- SnappySession.tokenize && canTokenize && session.wholeStageEnabled)
+ MATCH ~> (() => tokenize = SnappySession.tokenize && canTokenize && session.wholeStageEnabled)
}
protected final def TOKENIZE_END: Rule0 = rule {
diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala
index b7cee213da..a3a8e16036 100644
--- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala
+++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql
import java.sql.SQLException
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicInteger
+import java.util.function.Consumer
import scala.collection.JavaConverters._
import scala.collection.mutable
@@ -224,6 +225,9 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) {
@transient
private[sql] var planCaching: Boolean = Property.PlanCaching.get(sessionState.conf)
+ @transient
+ private[sql] var partitionPruning: Boolean = Property.PartitionPruning.get(sessionState.conf)
+
@transient
private[sql] var wholeStageEnabled: Boolean = sessionState.conf.wholeStageEnabled
@@ -282,6 +286,9 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) {
.getOrElse(false)
}
+ private[sql] def preferPrimaries: Boolean =
+ Property.PreferPrimariesInQuery.get(sessionState.conf)
+
private[sql] def addFinallyCode(ctx: CodegenContext, code: String): Int = {
val depth = getContextObject[Int](ctx, "D", "depth").getOrElse(0) + 1
addContextObject(ctx, "D", "depth", depth)
@@ -2004,8 +2011,12 @@ object SnappySession extends Logging {
// add profile listener for all regions that are using cached
// partitions of their "leader" region
if (rdd.getNumPartitions > 0) {
- session.sessionState.leaderPartitions.keysIterator.foreach(
- addBucketProfileListener)
+ session.sessionState.leaderPartitions.keySet().forEach(
+ new Consumer[PartitionedRegion] {
+ override def accept(pr: PartitionedRegion): Unit = {
+ addBucketProfileListener(pr)
+ }
+ })
}
}
(rdd, findShuffleDependencies(rdd).toArray, rdd.id, false,
@@ -2192,7 +2203,7 @@ object SnappySession extends Logging {
val key = CachedKey(session, lp, sqlText, currentWrappedConstants)
try {
var cachedDF = planCache.getUnchecked(key)
- if (!key.valid) {
+ if (!key.valid || !session.planCaching) {
logDebug(s"Invalidating cached plan for sql: ${key.sqlText}")
planCache.invalidate(key)
}
@@ -2201,7 +2212,7 @@ object SnappySession extends Logging {
val df = session.executeSQL(sqlText)
cachedDF = evaluatePlan(df, session, sqlText)
// default is enable caching
- if (!java.lang.Boolean.getBoolean("DISABLE_PLAN_CACHING")) {
+ if (session.planCaching) {
planCache.put(key, cachedDF)
}
}
diff --git a/core/src/main/scala/org/apache/spark/sql/catalyst/expressions/ParamLiteral.scala b/core/src/main/scala/org/apache/spark/sql/catalyst/expressions/ParamLiteral.scala
index 01a875cf1a..5033f75e00 100644
--- a/core/src/main/scala/org/apache/spark/sql/catalyst/expressions/ParamLiteral.scala
+++ b/core/src/main/scala/org/apache/spark/sql/catalyst/expressions/ParamLiteral.scala
@@ -19,14 +19,17 @@ package org.apache.spark.sql.catalyst.expressions
import java.util.Objects
+import scala.collection.mutable.ArrayBuffer
+
import com.esotericsoftware.kryo.io.{Input, Output}
import com.esotericsoftware.kryo.{Kryo, KryoSerializable}
+import com.gemstone.gemfire.internal.shared.ClientResolverUtils
import org.apache.spark.memory.{MemoryConsumer, MemoryMode, TaskMemoryManager}
import org.apache.spark.serializer.StructTypeSerializer
import org.apache.spark.sql.catalyst.CatalystTypeConverters._
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, SubExprEliminationState}
import org.apache.spark.sql.collection.Utils
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.Platform
@@ -43,18 +46,20 @@ trait DynamicReplacableConstant {
// whereever ParamLiteral case matching is required, it must match
// for DynamicReplacableConstant and use .eval(..) for code generation.
// see SNAP-1597 for more details.
-class ParamLiteral(_value: Any, _dataType: DataType, val pos: Int)
- extends Literal(_value, _dataType) with DynamicReplacableConstant {
+final class ParamLiteral(override val value: Any, _dataType: DataType, val pos: Int)
+ extends Literal(null, _dataType) with DynamicReplacableConstant {
// override def toString: String = s"ParamLiteral ${super.toString}"
private[this] var _foldable = false
private[this] var literalValueRef: String = _
-
private[this] val literalValue: LiteralValue = LiteralValue(value, dataType, pos)()
- private[this] def lv(ctx: CodegenContext) = if (ctx.references.exists(_ equals literalValue)) {
+ private[this] var isNull: String = _
+ private[this] var valueTerm: String = _
+
+ private[this] def lv(ctx: CodegenContext) = if (ctx.references.contains(literalValue)) {
assert(literalValueRef != null)
literalValueRef
} else {
@@ -74,14 +79,15 @@ class ParamLiteral(_value: Any, _dataType: DataType, val pos: Int)
// override def toString: String = s"pl[${super.toString}]"
- override def hashCode(): Int = {
- 31 * (31 * Objects.hashCode(dataType)) + pos
- }
+ override def hashCode(): Int = ClientResolverUtils.fastHashLong(
+ Objects.hashCode(dataType).toLong << 32L | (pos & 0xffffffffL))
+
+ // Literal cannot equal ParamLiteral since runtime value can be different
+ override def canEqual(that: Any): Boolean = that.isInstanceOf[ParamLiteral]
override def equals(obj: Any): Boolean = obj match {
case a: AnyRef if this eq a => true
- case pl: ParamLiteral =>
- pl.dataType == dataType && pl.pos == pos
+ case pl: ParamLiteral => pl.pos == pos && pl.dataType == dataType
case _ => false
}
@@ -90,9 +96,8 @@ class ParamLiteral(_value: Any, _dataType: DataType, val pos: Int)
if (n < parentFields) {
super.productElement(n)
} else {
- n match {
- case v if v == parentFields => pos
- }
+ assert (n == parentFields, s"unexpected n = $n but expected $parentFields")
+ pos
}
}
@@ -101,13 +106,25 @@ class ParamLiteral(_value: Any, _dataType: DataType, val pos: Int)
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
// change the isNull and primitive to consts, to inline them
val value = this.value
- val isNull = ctx.freshName("isNull")
- val valueTerm = ctx.freshName("value")
- val isNullLocal = s"${isNull}Local"
- val valueLocal = s"${valueTerm}Local"
- val valueRef = lv(ctx)
+ val addMutableState = (isNull eq null) || !ctx.mutableStates.exists(_._2 == isNull)
+ if (addMutableState) {
+ isNull = ctx.freshName("isNullTerm")
+ valueTerm = ctx.freshName("valueTerm")
+ }
+ val isNullLocal = ev.isNull
+ val valueLocal = ev.value
val dataType = Utils.getSQLDataType(this.dataType)
val javaType = ctx.javaType(dataType)
+ val initCode =
+ s"""
+ |final boolean $isNullLocal = $isNull;
+ |final $javaType $valueLocal = $valueTerm;
+ """.stripMargin
+ if (!addMutableState) {
+ // use the already added fields
+ return ev.copy(initCode, isNullLocal, valueLocal)
+ }
+ val valueRef = lv(ctx)
val box = ctx.boxedType(javaType)
val unbox = dataType match {
@@ -170,11 +187,7 @@ class ParamLiteral(_value: Any, _dataType: DataType, val pos: Int)
|$valueTerm = $isNull ? ${ctx.defaultValue(dataType)} : (($box)$valueRef.value())$unbox;
""".stripMargin)
}
- ev.copy(
- s"""
- |final boolean $isNullLocal = $isNull;
- |final $javaType $valueLocal = $valueTerm;
- """.stripMargin, isNullLocal, valueLocal)
+ ev.copy(initCode, isNullLocal, valueLocal)
}
}
@@ -243,7 +256,7 @@ case class LiteralValue(var value: Any, var dataType: DataType, var position: In
*
* Expressions like select c from tab where
* case col2 when 1 then col3 else 'y' end = 22
- * like queries doesn't converts literal evaluation into init method.
+ * like queries don't convert literal evaluation into init method.
*
* @param expr minimal expression tree that can be evaluated only once and turn into a constant.
*/
@@ -256,12 +269,19 @@ case class DynamicFoldableExpression(expr: Expression) extends Expression
def convertedLiteral: Any = createToScalaConverter(dataType)(eval(null))
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- // skip subexpression elimination for this because actual values
- // at runtime can be different
- val subExprs = ctx.subExprEliminationExprs.toMap
- ctx.subExprEliminationExprs.clear()
+ // subexpression elimination can lead to local variables but this expression
+ // needs in mutable state, so disable it (extra evaluations only once in init in any case)
+ val numSubExprs = ctx.subExprEliminationExprs.size
+ val oldSubExprs = if (numSubExprs != 0) {
+ val exprs = new ArrayBuffer[(Expression, SubExprEliminationState)](numSubExprs)
+ exprs ++= ctx.subExprEliminationExprs
+ ctx.subExprEliminationExprs.clear()
+ exprs
+ } else null
val eval = expr.genCode(ctx)
- ctx.subExprEliminationExprs ++= subExprs
+ if (oldSubExprs ne null) {
+ ctx.subExprEliminationExprs ++= oldSubExprs
+ }
val newVar = ctx.freshName("paramLiteralExpr")
val newVarIsNull = ctx.freshName("paramLiteralExprIsNull")
val comment = ctx.registerComment(expr.toString)
@@ -272,6 +292,8 @@ case class DynamicFoldableExpression(expr: Expression) extends Expression
s"$comment\n${eval.code}\n$newVar = ${eval.value};\n" +
s"$newVarIsNull = ${eval.isNull};")
ctx.addMutableState("boolean", newVarIsNull, "")
+ // allow sub-expression elimination of this expression itself
+ ctx.subExprEliminationExprs += this -> SubExprEliminationState(newVarIsNull, newVar)
ev.copy(code = "", value = newVar, isNull = newVarIsNull)
}
@@ -279,16 +301,6 @@ case class DynamicFoldableExpression(expr: Expression) extends Expression
override def children: Seq[Expression] = Seq(expr)
- // object reference equality for this class since values can change at runtime
- // so this should never be considered for subexpression elimination, for example
-
- override def hashCode(): Int = System.identityHashCode(this)
-
- override def equals(that: Any): Boolean = that match {
- case thatExpr: DynamicFoldableExpression => thatExpr eq this
- case _ => false
- }
-
override def nodeName: String = "DynamicExpression"
override def prettyName: String = "DynamicExpression"
diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala
index 9ab22788bc..37561be5c3 100644
--- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala
@@ -953,14 +953,17 @@ private[spark] class CoGroupExecutorLocalPartition(
override def hashCode(): Int = idx
}
-final class SmartExecutorBucketPartition(private var _index: Int,
+final class SmartExecutorBucketPartition(private var _index: Int, private var _bucketId: Int,
var hostList: mutable.ArrayBuffer[(String, String)])
extends Partition with KryoSerializable {
override def index: Int = _index
+ def bucketId: Int = _bucketId
+
override def write(kryo: Kryo, output: Output): Unit = {
output.writeVarInt(_index, true)
+ output.writeVarInt(_bucketId, true)
val numHosts = hostList.length
output.writeVarInt(numHosts, true)
for ((host, url) <- hostList) {
@@ -971,6 +974,7 @@ final class SmartExecutorBucketPartition(private var _index: Int,
override def read(kryo: Kryo, input: Input): Unit = {
_index = input.readVarInt(true)
+ _bucketId = input.readVarInt(true)
val numHosts = input.readVarInt(true)
hostList = new mutable.ArrayBuffer[(String, String)](numHosts)
for (_ <- 0 until numHosts) {
@@ -981,7 +985,7 @@ final class SmartExecutorBucketPartition(private var _index: Int,
}
override def toString: String =
- s"SmartExecutorBucketPartition($index, $hostList)"
+ s"SmartExecutorBucketPartition($index, $bucketId, $hostList)"
}
object ToolsCallbackInit extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala
index 9113a38282..7eab9ffe2a 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala
@@ -106,7 +106,8 @@ private[sql] abstract class PartitionedPhysicalScan(
val session = sqlContext.sparkSession.asInstanceOf[SnappySession]
callbacks.getOrderlessHashPartitioning(partitionColumns,
partitionColumnAliases, numPartitions,
- if (session.hasLinkPartitionsToBuckets) 0 else numBuckets, numBuckets)
+ if (session.hasLinkPartitionsToBuckets || session.preferPrimaries) 0 else numBuckets,
+ numBuckets)
} else {
HashPartitioning(partitionColumns, numPartitions)
}
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala
index 7d7146dbf4..ea1572a211 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala
@@ -17,16 +17,15 @@
package org.apache.spark.sql.execution
import com.gemstone.gemfire.internal.cache.PartitionedRegion
+
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression}
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.collection.{SmartExecutorBucketPartition, Utils}
-import org.apache.spark.sql.execution.columnar.JDBCAppendableRelation
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.hive.ConnectorCatalog
-import org.apache.spark.sql.row.JDBCMutableRelation
import org.apache.spark.sql.sources.DestroyRelation
import org.apache.spark.sql.store.StoreUtils
import org.apache.spark.sql.types.{LongType, StructType}
@@ -143,16 +142,12 @@ trait TableExec extends UnaryExecNode with CodegenSupportOnExecutor {
})
locations
}
- relation.get match {
- case m: JDBCMutableRelation =>
- inputRDDs.map { rdd =>
- new DelegateRDD(sparkContext, rdd, preferredLocations(m.table))
- }
- case JDBCAppendableRelation(table, _, _, _, _, _, _) =>
- inputRDDs.map { rdd =>
- new DelegateRDD(sparkContext, rdd, preferredLocations(table))
- }
- case _ => inputRDDs
+ inputRDDs.map { rdd =>
+ // if the two are different then its partition pruning case
+ if (numBuckets == rdd.getNumPartitions) {
+ val table = relation.get.asInstanceOf[PartitionedDataSourceScan].table
+ new DelegateRDD(sparkContext, rdd, preferredLocations(table))
+ } else rdd
}
}
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala
index 675f0109b0..f8e0f3dc75 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala
@@ -35,6 +35,8 @@ trait ColumnExec extends RowExec {
override def resolvedName: String = externalStore.tableName
+ protected def delayRollover: Boolean = false
+
override protected def connectionCodes(ctx: CodegenContext): (String, String, String) = {
val connectionClass = classOf[Connection].getName
val externalStoreTerm = ctx.addReferenceObj("externalStore", externalStore)
@@ -49,7 +51,7 @@ trait ColumnExec extends RowExec {
val initCode =
s"""
- |$taskListener = new $listenerClass(($storeClass)$externalStoreTerm);
+ |$taskListener = new $listenerClass(($storeClass)$externalStoreTerm, $delayRollover);
|$connTerm = $taskListener.getConn();
|if ($getContext() != null) {
| $getContext().addTaskCompletionListener($taskListener);
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala
index 3cc2e49f2f..e7588fe115 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala
@@ -663,14 +663,14 @@ case class ColumnInsertExec(child: SparkPlan, partitionColumns: Seq[String],
ctx.addNewFunction(beginSnapshotTx,
s"""
|private final Object[] $beginSnapshotTx() {
- | return $externalStoreTerm.beginTx();
+ | return $externalStoreTerm.beginTx(false);
|}
""".stripMargin)
commitSnapshotTx = ctx.freshName("commitSnapshotTx")
ctx.addNewFunction(commitSnapshotTx,
s"""
|private final void $commitSnapshotTx(String $txId, scala.Option $conn) {
- | $externalStoreTerm.commitTx($txId, $conn);
+ | $externalStoreTerm.commitTx($txId, false, $conn);
|}
""".stripMargin)
rollbackSnapshotTx = ctx.freshName("rollbackSnapshotTx")
@@ -812,14 +812,14 @@ case class ColumnInsertExec(child: SparkPlan, partitionColumns: Seq[String],
ctx.addNewFunction(beginSnapshotTx,
s"""
|private final Object[] $beginSnapshotTx() {
- | return $externalStoreTerm.beginTx();
+ | return $externalStoreTerm.beginTx(false);
|}
""".stripMargin)
commitSnapshotTx = ctx.freshName("commitSnapshotTx")
ctx.addNewFunction(commitSnapshotTx,
s"""
|private final void $commitSnapshotTx(String $txId, scala.Option $conn) {
- | $externalStoreTerm.commitTx($txId, $conn);
+ | $externalStoreTerm.commitTx($txId, false, $conn);
|}
""".stripMargin)
rollbackSnapshotTx = ctx.freshName("rollbackSnapshotTx")
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala
index 97e9a64a53..72563331aa 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala
@@ -161,16 +161,36 @@ private[sql] final case class ColumnTableScan(
statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
case EqualTo(l: DynamicReplacableConstant, a: AttributeReference) =>
statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+ case EqualTo(a: AttributeReference, l: Literal) =>
+ statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+ case EqualTo(l: Literal, a: AttributeReference) =>
+ statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+ case LessThan(a: AttributeReference, l: DynamicReplacableConstant) =>
+ statsFor(a).lowerBound < l
+ case LessThan(l: DynamicReplacableConstant, a: AttributeReference) =>
+ l < statsFor(a).upperBound
case LessThan(a: AttributeReference, l: Literal) => statsFor(a).lowerBound < l
case LessThan(l: Literal, a: AttributeReference) => l < statsFor(a).upperBound
+ case LessThanOrEqual(a: AttributeReference, l: DynamicReplacableConstant) =>
+ statsFor(a).lowerBound <= l
+ case LessThanOrEqual(l: DynamicReplacableConstant, a: AttributeReference) =>
+ l <= statsFor(a).upperBound
case LessThanOrEqual(a: AttributeReference, l: Literal) => statsFor(a).lowerBound <= l
case LessThanOrEqual(l: Literal, a: AttributeReference) => l <= statsFor(a).upperBound
+ case GreaterThan(a: AttributeReference, l: DynamicReplacableConstant) =>
+ l < statsFor(a).upperBound
+ case GreaterThan(l: DynamicReplacableConstant, a: AttributeReference) =>
+ statsFor(a).lowerBound < l
case GreaterThan(a: AttributeReference, l: Literal) => l < statsFor(a).upperBound
case GreaterThan(l: Literal, a: AttributeReference) => statsFor(a).lowerBound < l
+ case GreaterThanOrEqual(a: AttributeReference, l: DynamicReplacableConstant) =>
+ l <= statsFor(a).upperBound
+ case GreaterThanOrEqual(l: DynamicReplacableConstant, a: AttributeReference) =>
+ statsFor(a).lowerBound <= l
case GreaterThanOrEqual(a: AttributeReference, l: Literal) => l <= statsFor(a).upperBound
case GreaterThanOrEqual(l: Literal, a: AttributeReference) => statsFor(a).lowerBound <= l
@@ -617,8 +637,9 @@ private[sql] final case class ColumnTableScan(
val filterFunction = generateStatPredicate(ctx, numBatchRows)
val unsafeRow = ctx.freshName("unsafeRow")
val colNextBytes = ctx.freshName("colNextBytes")
- val numColumnsInStatBlob =
- relationSchema.size * ColumnStatsSchema.NUM_STATS_PER_COLUMN + 1
+ val numTableColumns = if (ordinalIdTerm eq null) relationSchema.size
+ else relationSchema.size - ColumnDelta.mutableKeyNames.length // for update/delete
+ val numColumnsInStatBlob = numTableColumns * ColumnStatsSchema.NUM_STATS_PER_COLUMN + 1
val incrementBatchOutputRows = if (numOutputRows ne null) {
s"$numOutputRows.${metricAdd(s"$numBatchRows - $deletedCount")};"
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnUpdateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnUpdateExec.scala
index a644611feb..e06b1fe159 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnUpdateExec.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnUpdateExec.scala
@@ -90,6 +90,8 @@ case class ColumnUpdateExec(child: SparkPlan, columnTable: String,
@transient private var updateMetric: String = _
@transient protected var txId: String = _
+ override protected def delayRollover: Boolean = true
+
override protected def doProduce(ctx: CodegenContext): String = {
val sql = new StringBuilder
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStore.scala
index 0382f1d873..f2604c11a0 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStore.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStore.scala
@@ -42,7 +42,7 @@ trait ExternalStore extends Serializable {
def getColumnBatchRDD(tableName: String, rowBuffer: String, requiredColumns: Array[String],
projection: Array[Int], fullScan: Boolean, prunePartitions: => Int,
- session: SparkSession, schema: StructType): RDD[Any]
+ session: SparkSession, schema: StructType, delayRollover: Boolean): RDD[Any]
def getConnectedExternalStore(tableName: String,
onExecutor: Boolean): ConnectedExternalStore
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala
index 6400588706..88ad4a5afe 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala
@@ -109,10 +109,9 @@ object ExternalStoreUtils {
} else tableName
}
- private def addProperty(props: Map[String, String], key: String,
- default: String): Map[String, String] = {
- if (props.contains(key)) props
- else props + (key -> default)
+ private def addProperty(props: mutable.Map[String, String], key: String,
+ default: String): Unit = {
+ if (!props.contains(key)) props.put(key, default)
}
private def defaultMaxExternalPoolSize: String =
@@ -125,24 +124,29 @@ object ExternalStoreUtils {
poolProps: Map[String, String], hikariCP: Boolean,
isEmbedded: Boolean): Map[String, String] = {
// setup default pool properties
- var props = poolProps
+ val props = new mutable.HashMap[String, String]()
+ if (poolProps.nonEmpty) props ++= poolProps
if (driver != null && !driver.isEmpty) {
- props = addProperty(props, "driverClassName", driver)
+ addProperty(props, "driverClassName", driver)
}
val defaultMaxPoolSize = if (isEmbedded) defaultMaxEmbeddedPoolSize
else defaultMaxExternalPoolSize
if (hikariCP) {
- props = props + ("jdbcUrl" -> url)
- props = addProperty(props, "maximumPoolSize", defaultMaxPoolSize)
- props = addProperty(props, "minimumIdle", "10")
- props = addProperty(props, "idleTimeout", "120000")
+ props.put("jdbcUrl", url)
+ addProperty(props, "maximumPoolSize", defaultMaxPoolSize)
+ addProperty(props, "minimumIdle", "10")
+ addProperty(props, "idleTimeout", "120000")
} else {
- props = props + ("url" -> url)
- props = addProperty(props, "maxActive", defaultMaxPoolSize)
- props = addProperty(props, "maxIdle", defaultMaxPoolSize)
- props = addProperty(props, "initialSize", "4")
+ props.put("url", url)
+ addProperty(props, "maxActive", defaultMaxPoolSize)
+ addProperty(props, "maxIdle", defaultMaxPoolSize)
+ addProperty(props, "initialSize", "4")
+ addProperty(props, "testOnBorrow", "true")
+ // embedded validation check is cheap
+ if (isEmbedded) addProperty(props, "validationInterval", "0")
+ else addProperty(props, "validationInterval", "10000")
}
- props
+ props.toMap
}
def getDriver(url: String, dialect: JdbcDialect): String = {
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala
index 863bbbdb5f..2dfa933f15 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala
@@ -75,6 +75,8 @@ abstract case class JDBCAppendableRelation(
val resolvedName: String = table
+ protected var delayRollover = false
+
def numBuckets: Int = -1
def isPartitioned: Boolean = true
@@ -133,7 +135,7 @@ abstract case class JDBCAppendableRelation(
readLock {
externalStore.getColumnBatchRDD(tableName, rowBuffer = table,
requestedColumns, projection, (filters eq null) || filters.length == 0,
- prunePartitions, sqlContext.sparkSession, schema)
+ prunePartitions, sqlContext.sparkSession, schema, delayRollover)
}
}
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnEncoding.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnEncoding.scala
index faadece53e..b132d2e019 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnEncoding.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnEncoding.scala
@@ -634,7 +634,7 @@ trait ColumnEncoder extends ColumnEncoding {
}
val baseTypeOffset = offset(position).toInt
// initialize the null bytes to zeros
- allocator.clearBuffer(columnData, baseTypeOffset, numNullBytes)
+ allocator.fill(columnData, 0, baseTypeOffset, numNullBytes)
this.baseTypeOffset = baseTypeOffset
this.baseDataOffset = baseTypeOffset + numNullBytes
if (writeNumElements) {
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala
index e0e13b1bfe..1d37d68877 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala
@@ -167,7 +167,8 @@ final class ColumnFormatIterator(baseRegion: LocalRegion, projection: Array[Int]
}
}
- override def getColumnValue(column: Int): AnyRef = {
+ override def getColumnValue(columnIndex: Int): AnyRef = {
+ val column = columnIndex & 0xffffffffL
if (entryIterator ne null) inMemoryBatches.get(inMemoryBatchIndex).get(column)
else currentDiskBatch.entryMap.get(column)
}
@@ -187,7 +188,7 @@ final class ColumnFormatIterator(baseRegion: LocalRegion, projection: Array[Int]
// try once more
v = entry.getValue(currentRegion)
}
- if (v ne null) uuidMap.justPut(columnIndex, v)
+ if (v ne null) uuidMap.justPut(columnIndex & 0xffffffffL, v)
}
def advanceToNextBatchSet(): Boolean = {
@@ -333,7 +334,7 @@ private final class DiskMultiColumnBatch(_region: LocalRegion, _readerId: Int,
case _ => v
} else v
} else re.getValueInVMOrDiskWithoutFaultIn(region)
- map.justPut(re.getRawKey.asInstanceOf[ColumnFormatKey].columnIndex, v)
+ map.justPut(re.getRawKey.asInstanceOf[ColumnFormatKey].columnIndex & 0xffffffffL, v)
i += 1
}
diskEntries = null
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala
index 229e141e63..cbeaeb8a47 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala
@@ -22,6 +22,7 @@ import scala.util.control.NonFatal
import com.gemstone.gemfire.internal.cache.{ExternalTableMetaData, PartitionedRegion}
import com.pivotal.gemfirexd.internal.engine.Misc
+import com.pivotal.gemfirexd.internal.engine.ddl.catalog.GfxdSystemProcedures
import com.pivotal.gemfirexd.internal.engine.store.GemFireContainer
import io.snappydata.Constant
@@ -171,7 +172,7 @@ abstract class BaseColumnFormatRelation(
val rdd = scanTable(table, columns, filters, -1)
val partitionEvaluator = rdd match {
case c: ColumnarStorePartitionedRDD => c.getPartitionEvaluator
- case r => () => r.partitions
+ case s => s.asInstanceOf[SmartConnectorColumnRDD].getPartitionEvaluator
}
// select the rowId from row buffer for update/delete keys
val numColumns = columns.length
@@ -214,7 +215,7 @@ abstract class BaseColumnFormatRelation(
Array.empty[Filter],
// use same partitions as the column store (SNAP-1083)
partitionEvaluator,
- commitTx = false)
+ commitTx = false, delayRollover)
case _ =>
new SmartConnectorRowRDD(
session,
@@ -226,8 +227,7 @@ abstract class BaseColumnFormatRelation(
// use same partitions as the column store (SNAP-1083)
partitionEvaluator,
relInfo.embdClusterRelDestroyVersion,
- _commitTx = false
- )
+ _commitTx = false, delayRollover)
}
}
@@ -486,10 +486,10 @@ abstract class BaseColumnFormatRelation(
case SnappyEmbeddedMode(_, _) | LocalMode(_, _) =>
// force flush all the buckets into the column store
Utils.mapExecutors(sqlContext, () => {
- ColumnFormatRelation.flushLocalBuckets(resolvedName)
+ GfxdSystemProcedures.flushLocalBuckets(resolvedName, true)
Iterator.empty
}).count()
- ColumnFormatRelation.flushLocalBuckets(resolvedName)
+ GfxdSystemProcedures.flushLocalBuckets(resolvedName, true)
case _ =>
}
}
@@ -529,9 +529,11 @@ class ColumnFormatRelation(
}
val cr = relation.relation.asInstanceOf[ColumnFormatRelation]
val schema = StructType(cr.schema ++ ColumnDelta.mutableKeyFields)
- relation.copy(relation = new ColumnFormatRelation(cr.table, cr.provider,
+ val newRelation = new ColumnFormatRelation(cr.table, cr.provider,
cr.mode, schema, cr.schemaExtensions, cr.ddlExtensionForShadowTable,
- cr.origOptions, cr.externalStore, cr.partitioningColumns, cr.sqlContext),
+ cr.origOptions, cr.externalStore, cr.partitioningColumns, cr.sqlContext)
+ newRelation.delayRollover = true
+ relation.copy(relation = newRelation,
expectedOutputAttributes = Some(relation.output ++ ColumnDelta.mutableKeyAttributes))
}
@@ -718,9 +720,11 @@ class IndexColumnFormatRelation(
keyColumns: Seq[String]): LogicalRelation = {
val cr = relation.relation.asInstanceOf[IndexColumnFormatRelation]
val schema = StructType(cr.schema ++ ColumnDelta.mutableKeyFields)
- relation.copy(relation = new IndexColumnFormatRelation(cr.table, cr.provider,
+ val newRelation = new IndexColumnFormatRelation(cr.table, cr.provider,
cr.mode, schema, cr.schemaExtensions, cr.ddlExtensionForShadowTable, cr.origOptions,
- cr.externalStore, cr.partitioningColumns, cr.sqlContext, baseTableName),
+ cr.externalStore, cr.partitioningColumns, cr.sqlContext, baseTableName)
+ newRelation.delayRollover = true
+ relation.copy(relation = newRelation,
expectedOutputAttributes = Some(relation.output ++ ColumnDelta.mutableKeyAttributes))
}
@@ -740,23 +744,6 @@ object ColumnFormatRelation extends Logging with StoreCallback {
type IndexUpdateStruct = ((PreparedStatement, InternalRow) => Int, PreparedStatement)
- // register the call backs with the JDBCSource so that
- // bucket region can insert into the column table
-
- def flushLocalBuckets(resolvedName: String): Unit = {
- val pr = Misc.getRegionForTable(resolvedName, false)
- .asInstanceOf[PartitionedRegion]
- if (pr != null) {
- val ds = pr.getDataStore
- if (ds != null) {
- val itr = ds.getAllLocalPrimaryBucketRegions.iterator()
- while (itr.hasNext) {
- itr.next().createAndInsertColumnBatch(true)
- }
- }
- }
- }
-
final def columnBatchTableName(table: String): String = {
val schemaDot = table.indexOf('.')
if (schemaDot > 0) {
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala
index 62b22ff8f1..4e831532a9 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala
@@ -27,10 +27,12 @@ import scala.util.control.NonFatal
import com.esotericsoftware.kryo.io.{Input, Output}
import com.esotericsoftware.kryo.{Kryo, KryoSerializable}
+import com.gemstone.gemfire.cache.IsolationLevel
import com.gemstone.gemfire.internal.cache.{BucketRegion, CachePerfStats, GemFireCacheImpl, LocalRegion, PartitionedRegion, TXManagerImpl}
import com.gemstone.gemfire.internal.shared.unsafe.UnsafeHolder
import com.gemstone.gemfire.internal.shared.{BufferAllocator, SystemProperties}
import com.pivotal.gemfirexd.internal.engine.Misc
+import com.pivotal.gemfirexd.internal.engine.ddl.catalog.GfxdSystemProcedures
import com.pivotal.gemfirexd.internal.iapi.services.context.ContextService
import com.pivotal.gemfirexd.internal.impl.jdbc.{EmbedConnection, EmbedConnectionContext}
import io.snappydata.impl.SparkConnectorRDDHelper
@@ -104,7 +106,7 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie
}
// begin should decide the connection which will be used by insert/commit/rollback
- def beginTx(): Array[_ <: Object] = {
+ def beginTx(delayRollover: Boolean): Array[_ <: Object] = {
val conn = self.getConnection(tableName, onExecutor = true)
assert(!conn.isClosed)
@@ -116,7 +118,9 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie
if (context == null ||
(context.getSnapshotTXState == null && context.getTXState == null)) {
val txMgr = Misc.getGemFireCache.getCacheTransactionManager
- txMgr.begin(com.gemstone.gemfire.cache.IsolationLevel.SNAPSHOT, null)
+ val tx = txMgr.beginTX(TXManagerImpl.getOrCreateTXContext(),
+ IsolationLevel.SNAPSHOT, null, null)
+ tx.setColumnRolloverDisabled(delayRollover)
Array(conn, txMgr.getTransactionId.stringFormat())
} else {
Array(conn, null)
@@ -125,10 +129,11 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie
val txId = SparkConnectorRDDHelper.snapshotTxIdForWrite.get
if (txId == null) {
logDebug(s"Going to start the transaction on server on conn $conn ")
- val startAndGetSnapshotTXId = conn.prepareCall(s"call sys.START_SNAPSHOT_TXID (?)")
- startAndGetSnapshotTXId.registerOutParameter(1, java.sql.Types.VARCHAR)
+ val startAndGetSnapshotTXId = conn.prepareCall(s"call sys.START_SNAPSHOT_TXID(?,?)")
+ startAndGetSnapshotTXId.setBoolean(1, delayRollover)
+ startAndGetSnapshotTXId.registerOutParameter(2, java.sql.Types.VARCHAR)
startAndGetSnapshotTXId.execute()
- val txid: String = startAndGetSnapshotTXId.getString(1)
+ val txid = startAndGetSnapshotTXId.getString(2)
startAndGetSnapshotTXId.close()
SparkConnectorRDDHelper.snapshotTxIdForWrite.set(txid)
logDebug(s"The snapshot tx id is $txid and tablename is $tableName")
@@ -136,7 +141,7 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie
} else {
logDebug(s"Going to use the transaction $txId on server on conn $conn ")
// it should always be not null.
- if (!txId.equals("null")) {
+ if (!txId.isEmpty) {
val statement = conn.createStatement()
statement.execute(
s"call sys.USE_SNAPSHOT_TXID('$txId')")
@@ -148,18 +153,23 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie
}(Some(conn))
}
- def commitTx(txId: String, conn: Option[Connection]): Unit = {
+ def commitTx(txId: String, delayRollover: Boolean, conn: Option[Connection]): Unit = {
tryExecute(tableName, closeOnSuccessOrFailure = false, onExecutor = true)(conn => {
var success = false
try {
connectionType match {
case ConnectionType.Embedded =>
+ // if rollover was marked as delayed, then do the rollover before commit
+ if (delayRollover) {
+ GfxdSystemProcedures.flushLocalBuckets(tableName, false)
+ }
// if(SparkConnectorRDDHelper.snapshotTxIdForRead.get)
Misc.getGemFireCache.getCacheTransactionManager.commit()
case _ =>
logDebug(s"Going to commit $txId the transaction on server conn is $conn")
- val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?)")
- ps.setString(1, if (txId == null) "null" else txId)
+ val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)")
+ ps.setString(1, if (txId ne null) txId else "")
+ ps.setString(2, if (delayRollover) tableName else "")
try {
ps.executeUpdate()
logDebug(s"The txid being committed is $txId")
@@ -194,7 +204,7 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie
case _ =>
logDebug(s"Going to rollback $txId the transaction on server on wconn $conn ")
val ps = conn.prepareStatement(s"call sys.ROLLBACK_SNAPSHOT_TXID(?)")
- ps.setString(1, if (txId == null) "null" else txId)
+ ps.setString(1, if (txId ne null) txId else "")
try {
ps.executeUpdate()
logDebug(s"The txid being rolledback is $txId")
@@ -509,7 +519,8 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie
fullScan: Boolean,
prunePartitions: => Int,
session: SparkSession,
- schema: StructType): RDD[Any] = {
+ schema: StructType,
+ delayRollover: Boolean): RDD[Any] = {
val snappySession = session.asInstanceOf[SnappySession]
connectionType match {
case ConnectionType.Embedded =>
@@ -535,7 +546,9 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie
tableName, requiredColumns, ConnectionProperties(connProperties.url,
connProperties.driver, connProperties.dialect, poolProps,
connProperties.connProps, connProperties.executorConnProps,
- connProperties.hikariCP), schema, this, parts, embdClusterRelDestroyVersion)
+ connProperties.hikariCP),
+ // TODO: correct to proper pruner as part of SNAP-2194
+ schema, this, parts, -1, embdClusterRelDestroyVersion, delayRollover)
}
}
@@ -684,7 +697,7 @@ final class ColumnarStorePartitionedRDD(
region.asInstanceOf[PartitionedRegion])
allPartitions
case bucketId: Int =>
- if (java.lang.Boolean.getBoolean("DISABLE_PARTITION_PRUNING")) {
+ if (!session.partitionPruning) {
allPartitions = session.sessionState.getTablePartitions(
region.asInstanceOf[PartitionedRegion])
allPartitions
@@ -768,21 +781,23 @@ final class SmartConnectorColumnRDD(
private var connProperties: ConnectionProperties,
private var schema: StructType,
@transient private val store: ExternalStore,
- private val parts: Array[Partition],
- private var relDestroyVersion: Int = -1)
- extends RDDKryo[Any](session.sparkContext, Nil)
- with KryoSerializable {
+ @transient private val allParts: Array[Partition],
+ @(transient @param) partitionPruner: => Int,
+ private var relDestroyVersion: Int,
+ private var delayRollover: Boolean)
+ extends RDDKryo[Any](session.sparkContext, Nil) with KryoSerializable {
override def compute(split: Partition,
context: TaskContext): Iterator[ByteBuffer] = {
val helper = new SparkConnectorRDDHelper
- val conn: Connection = helper.getConnection(connProperties, split)
+ val part = split.asInstanceOf[SmartExecutorBucketPartition]
+ val conn: Connection = helper.getConnection(connProperties, part)
- val partitionId = split.index
+ val partitionId = part.bucketId
val (fetchStatsQuery, fetchColQuery) = helper.getSQLStatement(tableName,
partitionId, requiredColumns, schema)
// fetch the stats
- val (statement, rs, txId) = helper.executeQuery(conn, tableName, split,
+ val (statement, rs, txId) = helper.executeQuery(conn, tableName, part,
fetchStatsQuery, relDestroyVersion)
val itr = new ColumnBatchIteratorOnRS(conn, requiredColumns, statement, rs,
context, partitionId, fetchColQuery)
@@ -792,8 +807,9 @@ final class SmartConnectorColumnRDD(
logDebug(s"The txid going to be committed is $txId " + tableName)
// if ((txId ne null) && !txId.equals("null")) {
- val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?)")
- ps.setString(1, if (txId eq null) "null" else txId)
+ val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)")
+ ps.setString(1, if (txId ne null) txId else "")
+ ps.setString(2, if (delayRollover) tableName else "")
ps.executeUpdate()
logDebug(s"The txid being committed is $txId")
ps.close()
@@ -816,7 +832,14 @@ final class SmartConnectorColumnRDD(
.hostList.map(_._1.asInstanceOf[String])
}
- override def getPartitions: Array[Partition] = parts
+ def getPartitionEvaluator: () => Array[Partition] = () => partitionPruner match {
+ case -1 => allParts
+ case bucketId =>
+ val part = allParts(bucketId).asInstanceOf[SmartExecutorBucketPartition]
+ Array(new SmartExecutorBucketPartition(0, bucketId, part.hostList))
+ }
+
+ override def getPartitions: Array[Partition] = getPartitionEvaluator()
override def write(kryo: Kryo, output: Output): Unit = {
super.write(kryo, output)
@@ -829,6 +852,7 @@ final class SmartConnectorColumnRDD(
ConnectionPropertiesSerializer.write(kryo, output, connProperties)
StructTypeSerializer.write(kryo, output, schema)
output.writeVarInt(relDestroyVersion, false)
+ output.writeBoolean(delayRollover)
}
override def read(kryo: Kryo, input: Input): Unit = {
@@ -840,6 +864,7 @@ final class SmartConnectorColumnRDD(
connProperties = ConnectionPropertiesSerializer.read(kryo, input)
schema = StructTypeSerializer.read(kryo, input, c = null)
relDestroyVersion = input.readVarInt(false)
+ delayRollover = input.readBoolean()
}
}
@@ -848,13 +873,13 @@ class SmartConnectorRowRDD(_session: SnappySession,
_isPartitioned: Boolean,
_columns: Array[String],
_connProperties: ConnectionProperties,
- _filters: Array[Filter] = Array.empty[Filter],
- _partEval: () => Array[Partition] = () => Array.empty[Partition],
- _relDestroyVersion: Int = -1,
- _commitTx: Boolean)
+ _filters: Array[Filter],
+ _partEval: () => Array[Partition],
+ _relDestroyVersion: Int,
+ _commitTx: Boolean, _delayRollover: Boolean)
extends RowFormatScanRDD(_session, _tableName, _isPartitioned, _columns,
pushProjections = true, useResultSet = true, _connProperties,
- _filters, _partEval, _commitTx) {
+ _filters, _partEval, _commitTx, _delayRollover) {
override def commitTxBeforeTaskCompletion(conn: Option[Connection],
@@ -863,8 +888,9 @@ class SmartConnectorRowRDD(_session: SnappySession,
val txId = SparkConnectorRDDHelper.snapshotTxIdForRead.get
logDebug(s"The txid going to be committed is $txId " + tableName)
// if ((txId ne null) && !txId.equals("null")) {
- val ps = conn.get.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?)")
- ps.setString(1, if (txId == null) "null" else txId)
+ val ps = conn.get.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)")
+ ps.setString(1, if (txId ne null) txId else "")
+ ps.setString(2, if (delayRollover) tableName else "")
ps.executeUpdate()
logDebug(s"The txid being committed is $txId")
ps.close()
@@ -877,7 +903,7 @@ class SmartConnectorRowRDD(_session: SnappySession,
thePart: Partition, context: TaskContext): (Connection, Statement, ResultSet) = {
val helper = new SparkConnectorRDDHelper
val conn: Connection = helper.getConnection(
- connProperties, thePart)
+ connProperties, thePart.asInstanceOf[SmartExecutorBucketPartition])
if (context ne null) {
val partitionId = context.partitionId()
context.addTaskCompletionListener { _ =>
@@ -932,7 +958,7 @@ class SmartConnectorRowRDD(_session: SnappySession,
if (thriftConn ne null) {
stmt.asInstanceOf[ClientPreparedStatement].setSnapshotTransactionId(txId)
} else if (txId != null) {
- if (!txId.equals("null")) {
+ if (!txId.isEmpty) {
statement.execute(
s"call sys.USE_SNAPSHOT_TXID('$txId')")
}
@@ -941,8 +967,9 @@ class SmartConnectorRowRDD(_session: SnappySession,
val rs = stmt.executeQuery()
// get the txid which was used to take the snapshot.
- if (!_commitTx) {
- val getSnapshotTXId = conn.prepareStatement("values sys.GET_SNAPSHOT_TXID()")
+ if (!commitTx) {
+ val getSnapshotTXId = conn.prepareStatement("values sys.GET_SNAPSHOT_TXID(?)")
+ getSnapshotTXId.setBoolean(1, delayRollover)
val rs = getSnapshotTXId.executeQuery()
rs.next()
val txId = rs.getString(1)
@@ -972,8 +999,9 @@ class SmartConnectorRowRDD(_session: SnappySession,
}
-class SnapshotConnectionListener(store: JDBCSourceAsColumnarStore) extends TaskCompletionListener {
- val connAndTxId: Array[_ <: Object] = store.beginTx()
+class SnapshotConnectionListener(store: JDBCSourceAsColumnarStore,
+ delayRollover: Boolean) extends TaskCompletionListener {
+ val connAndTxId: Array[_ <: Object] = store.beginTx(delayRollover)
var isSuccess = false
override def onTaskCompletion(context: TaskContext): Unit = {
@@ -981,7 +1009,7 @@ class SnapshotConnectionListener(store: JDBCSourceAsColumnarStore) extends TaskC
val conn = connAndTxId(0).asInstanceOf[Connection]
if (connAndTxId(1) != null) {
if (success()) {
- store.commitTx(txId, Some(conn))
+ store.commitTx(txId, delayRollover, Some(conn))
}
else {
store.rollbackTx(txId, Some(conn))
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala
index 20279f4cdb..fc9e7ee985 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala
@@ -24,7 +24,7 @@ import com.gemstone.gemfire.internal.cache.{NonLocalRegionEntry, PartitionedRegi
import com.pivotal.gemfirexd.internal.engine.distributed.GfxdListResultCollector.ListResultCollectorValue
import com.pivotal.gemfirexd.internal.engine.distributed.message.GetAllExecutorMessage
import com.pivotal.gemfirexd.internal.engine.sql.execute.GemFireResultSet
-import io.snappydata.collection.LongObjectHashMap
+import io.snappydata.collection.IntObjectHashMap
import org.apache.spark.sql.execution.columnar.impl.ColumnFormatEntry.{DELETE_MASK_COL_INDEX, STATROW_COL_INDEX}
@@ -67,7 +67,7 @@ final class RemoteEntriesIterator(bucketId: Int, projection: Array[Int],
}
private var currentStatsKey: ColumnFormatKey = _
- private val currentValueMap = LongObjectHashMap.withExpectedSize[AnyRef](8)
+ private val currentValueMap = IntObjectHashMap.withExpectedSize[AnyRef](8)
private def fetchUsingGetAll(keys: Array[AnyRef]): Seq[(AnyRef, AnyRef)] = {
val msg = new GetAllExecutorMessage(pr, keys, null, null, null, null,
@@ -76,7 +76,6 @@ final class RemoteEntriesIterator(bucketId: Int, projection: Array[Int],
allMemberResults.flatMap { case v: ListResultCollectorValue =>
msg.getKeysPerMember(v.memberID).asScala.zip(
v.resultOfSingleExecution.asInstanceOf[java.util.List[AnyRef]].asScala)
-
}
}
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatRelation.scala
index 1838c776c3..798c5237cc 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatRelation.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatRelation.scala
@@ -145,8 +145,7 @@ class RowFormatRelation(
useResultSet = false,
connProperties,
handledFilters,
- commitTx = true
- )
+ commitTx = true, delayRollover = false)
case _ =>
new SmartConnectorRowRDD(
@@ -158,8 +157,7 @@ class RowFormatRelation(
handledFilters,
_partEval = () => relInfo.partitions,
relInfo.embdClusterRelDestroyVersion,
- _commitTx = true
- )
+ _commitTx = true, _delayRollover = false)
}
(rdd, Nil)
}
diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala
index 71708c0370..abd9883cb9 100644
--- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala
+++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala
@@ -29,6 +29,7 @@ import com.gemstone.gemfire.cache.IsolationLevel
import com.gemstone.gemfire.internal.cache._
import com.gemstone.gemfire.internal.shared.ClientSharedData
import com.pivotal.gemfirexd.internal.engine.Misc
+import com.pivotal.gemfirexd.internal.engine.ddl.catalog.GfxdSystemProcedures
import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils
import com.pivotal.gemfirexd.internal.engine.store.{AbstractCompactExecRow, GemFireContainer, RegionEntryUtils}
import com.pivotal.gemfirexd.internal.iapi.types.RowLocation
@@ -53,11 +54,12 @@ class RowFormatScanRDD(@transient val session: SnappySession,
protected var isPartitioned: Boolean,
@transient private val columns: Array[String],
var pushProjections: Boolean,
- var useResultSet: Boolean,
+ protected var useResultSet: Boolean,
protected var connProperties: ConnectionProperties,
@transient private val filters: Array[Filter] = Array.empty[Filter],
@transient protected val partitionEvaluator: () => Array[Partition] = () =>
- Array.empty[Partition], var commitTx: Boolean)
+ Array.empty[Partition],
+ protected var commitTx: Boolean, protected var delayRollover: Boolean)
extends RDDKryo[Any](session.sparkContext, Nil) with KryoSerializable {
protected var filterWhereArgs: ArrayBuffer[Any] = _
@@ -226,6 +228,13 @@ class RowFormatScanRDD(@transient val session: SnappySession,
embedConn.getTR.setupContextStack()
rs.pushStatementContext(lcc, true)
*/
+ // set the delayRollover flag on current transaction
+ if (delayRollover) {
+ val tx = TXManagerImpl.getCurrentTXState
+ if (tx ne null) {
+ tx.getProxy.setColumnRolloverDisabled(true)
+ }
+ }
(conn, stmt, rs)
}
@@ -234,6 +243,10 @@ class RowFormatScanRDD(@transient val session: SnappySession,
Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => {
val tx = TXManagerImpl.getCurrentSnapshotTXState
if (tx != null /* && !(tx.asInstanceOf[TXStateProxy]).isClosed() */ ) {
+ // if rollover was marked as delayed, then do the rollover before commit
+ if (delayRollover) {
+ GfxdSystemProcedures.flushLocalBuckets(tableName, false)
+ }
val txMgr = tx.getTxMgr
txMgr.masqueradeAs(tx)
txMgr.commit()
@@ -322,11 +335,12 @@ class RowFormatScanRDD(@transient val session: SnappySession,
override def write(kryo: Kryo, output: Output): Unit = {
super.write(kryo, output)
- output.writeBoolean(commitTx)
output.writeString(tableName)
output.writeBoolean(isPartitioned)
output.writeBoolean(pushProjections)
output.writeBoolean(useResultSet)
+ output.writeBoolean(commitTx)
+ output.writeBoolean(delayRollover)
output.writeString(columnList)
val filterArgs = filterWhereArgs
@@ -350,11 +364,12 @@ class RowFormatScanRDD(@transient val session: SnappySession,
override def read(kryo: Kryo, input: Input): Unit = {
super.read(kryo, input)
- commitTx = input.readBoolean()
tableName = input.readString()
isPartitioned = input.readBoolean()
pushProjections = input.readBoolean()
useResultSet = input.readBoolean()
+ commitTx = input.readBoolean()
+ delayRollover = input.readBoolean()
columnList = input.readString()
val numFilters = input.readVarInt(true)
diff --git a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala
index eea6d70a76..a04e53204f 100644
--- a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala
+++ b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala
@@ -18,8 +18,8 @@
package org.apache.spark.sql.internal
import java.util.Properties
+import java.util.concurrent.ConcurrentHashMap
-import scala.collection.concurrent.TrieMap
import scala.reflect.{ClassTag, classTag}
import com.gemstone.gemfire.internal.cache.{CacheDistributionAdvisee, ColocationHelper, PartitionedRegion}
@@ -220,8 +220,8 @@ class SnappySessionState(snappySession: SnappySession)
* The partition mapping selected for the lead partitioned region in
* a collocated chain for current execution
*/
- private[spark] val leaderPartitions = new TrieMap[PartitionedRegion,
- Array[Partition]]()
+ private[spark] val leaderPartitions = new ConcurrentHashMap[PartitionedRegion,
+ Array[Partition]](16, 0.7f, 1)
/**
* Replaces [[UnresolvedRelation]]s with concrete relations from the catalog.
@@ -436,17 +436,21 @@ class SnappySessionState(snappySession: SnappySession)
def getTablePartitions(region: PartitionedRegion): Array[Partition] = {
val leaderRegion = ColocationHelper.getLeaderRegion(region)
- leaderPartitions.getOrElseUpdate(leaderRegion, {
- val linkPartitionsToBuckets = snappySession.hasLinkPartitionsToBuckets
- if (linkPartitionsToBuckets) {
- // also set the default shuffle partitions for this execution
- // to minimize exchange
- snappySession.sessionState.conf.setExecutionShufflePartitions(
- region.getTotalNumberOfBuckets)
- }
- StoreUtils.getPartitionsPartitionedTable(snappySession, leaderRegion,
- linkPartitionsToBuckets)
- })
+ leaderPartitions.computeIfAbsent(leaderRegion,
+ new java.util.function.Function[PartitionedRegion, Array[Partition]] {
+ override def apply(pr: PartitionedRegion): Array[Partition] = {
+ val linkPartitionsToBuckets = snappySession.hasLinkPartitionsToBuckets
+ val preferPrimaries = snappySession.preferPrimaries
+ if (linkPartitionsToBuckets || preferPrimaries) {
+ // also set the default shuffle partitions for this execution
+ // to minimize exchange
+ snappySession.sessionState.conf.setExecutionShufflePartitions(
+ region.getTotalNumberOfBuckets)
+ }
+ StoreUtils.getPartitionsPartitionedTable(snappySession, pr,
+ linkPartitionsToBuckets, preferPrimaries)
+ }
+ })
}
def getTablePartitions(region: CacheDistributionAdvisee): Array[Partition] =
@@ -501,6 +505,11 @@ class SnappyConf(@transient val session: SnappySession)
case Some(pool) => throw new IllegalArgumentException(s"Invalid Pool $pool")
}
+ case Property.PartitionPruning.name => value match {
+ case Some(b) => session.partitionPruning = b.toString.toBoolean
+ case None => session.partitionPruning = Property.PartitionPruning.defaultValue.get
+ }
+
case Property.PlanCaching.name =>
value match {
case Some(boolVal) =>
diff --git a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala
index 0c681cba1c..545cb630f0 100644
--- a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala
+++ b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala
@@ -179,12 +179,10 @@ object StoreUtils {
}
private[sql] def getPartitionsPartitionedTable(session: SnappySession,
- region: PartitionedRegion,
- linkBucketsToPartitions: Boolean): Array[Partition] = {
+ region: PartitionedRegion, linkBucketsToPartitions: Boolean,
+ preferPrimaries: Boolean): Array[Partition] = {
val callbacks = ToolsCallbackInit.toolsCallback
- val preferPrimaries = Property.PreferPrimariesInQuery.get(
- session.sessionState.conf)
if (!linkBucketsToPartitions && callbacks != null) {
allocateBucketsToPartitions(session, region, preferPrimaries)
} else {
diff --git a/core/src/test/java/io/snappydata/TestClass.java b/core/src/test/java/io/snappydata/TestClass.java
new file mode 100644
index 0000000000..ee3f65ee83
--- /dev/null
+++ b/core/src/test/java/io/snappydata/TestClass.java
@@ -0,0 +1,110 @@
+package io.snappydata;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.function.Supplier;
+
+import com.gemstone.gemfire.DataSerializable;
+import com.gemstone.gemfire.internal.shared.ClientResolverUtils;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.hash.Murmur3_x86_32;
+
+public class TestClass {
+
+ private final HashMap> registeredTypes =
+ new HashMap<>();
+
+ private void register(String key, Supplier f) {
+ registeredTypes.put(key, f);
+ }
+
+ private void invoke(String key) throws IOException, ClassNotFoundException {
+ DataSerializable v = registeredTypes.get(key).get();
+ v.fromData(null);
+ System.out.println(v);
+ }
+
+ public static void main(String[] args) throws Exception {
+ /*
+ TestClass test = new TestClass();
+ System.out.println("Registering");
+ test.register("T1", T1::new);
+ System.out.println("Registering Child");
+ test.register("Child", T1.Child::new);
+ System.out.println("Invoking");
+ test.invoke("T1");
+ System.out.println("Invoking");
+ test.invoke("T1");
+ System.out.println("Invoking Child");
+ test.invoke("Child");
+ System.out.println("Invoking Child");
+ test.invoke("Child");
+ */
+ byte[] bytes = ClientResolverUtils.class.getName().getBytes(StandardCharsets.UTF_8);
+ final int warmups = 100;
+ final int measure = 1000;
+
+ long total = 0;
+ for (int i = 0; i < warmups; i++) {
+ total += ClientResolverUtils.addBytesToHash(bytes, 42);
+ }
+ System.out.println("1: Total=" + total);
+
+ total = 0;
+ long start = System.nanoTime();
+ for (int i = 0; i < measure; i++) {
+ total += ClientResolverUtils.addBytesToHash(bytes, 42);
+ }
+ long end = System.nanoTime();
+ System.out.println("1: Time taken = " + (end - start) + "ns, total=" + total);
+
+ total = 0;
+ for (int i = 0; i < warmups; i++) {
+ total += Murmur3_x86_32.hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET,
+ bytes.length, 42);
+ }
+ System.out.println("2: Total=" + total);
+
+ total = 0;
+ start = System.nanoTime();
+ for (int i = 0; i < measure; i++) {
+ total += Murmur3_x86_32.hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET,
+ bytes.length, 42);
+ }
+ end = System.nanoTime();
+ System.out.println("2: Time taken = " + (end - start) + "ns, total=" + total);
+ }
+}
+
+final class T1 implements DataSerializable {
+
+ static {
+ System.out.println("T1.cinit");
+ }
+
+ static final class Child implements DataSerializable {
+
+ static {
+ System.out.println("Child.cinit");
+ }
+
+ @Override
+ public void toData(DataOutput out) throws IOException {
+ }
+
+ @Override
+ public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+ }
+ }
+
+ @Override
+ public void toData(DataOutput out) throws IOException {
+ }
+
+ @Override
+ public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+ }
+}
diff --git a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala
index c34c2ce85f..c95d55141c 100644
--- a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala
+++ b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala
@@ -523,4 +523,32 @@ object ColumnUpdateDeleteTests extends Assertions with Logging {
"select * from updateTable EXCEPT select * from checkTable2").collect()
assert(res.length === 0)
}
+
+ def testSNAP2124(session: SnappySession, checkPruning: Boolean): Unit = {
+ val filePath = getClass.getResource("/sample_records.json").getPath
+ session.sql("CREATE TABLE domaindata (cntno_l string,cntno_m string," +
+ "day1 string,day2 string,day3 string,day4 string,day5 string," +
+ "day6 string,day7 string,dr string,ds string,email string," +
+ "id BIGINT NOT NULL,idinfo_1 string,idinfo_2 string,idinfo_3 string," +
+ "idinfo_4 string,lang_1 string,lang_2 string,lang_3 string,name string) " +
+ "USING COLUMN OPTIONS (PARTITION_BY 'id',BUCKETS '40', COLUMN_BATCH_SIZE '10')")
+ session.read.json(filePath).write.insertInto("domaindata")
+
+ var ds = session.sql("select ds, dr from domaindata where id = 40L")
+ SnappyFunSuite.checkAnswer(ds, Seq(Row("['cbcinewsemail.com']", "[]")))
+
+ ds = session.sql("UPDATE domaindata SET ds = '[''cbcin.com'']', dr = '[]' WHERE id = 40")
+ if (checkPruning) {
+ // below checks both the result and partition pruning (only one row)
+ SnappyFunSuite.checkAnswer(ds, Seq(Row(1)))
+ } else {
+ // check the result but expect no pruning (missing in smart connector)
+ SnappyFunSuite.checkAnswer(ds, Row(1) :: (1 until 40).map(_ => Row(0)).toList)
+ }
+
+ ds = session.sql("select ds, dr from domaindata where id = 40")
+ // pruning for selects is available in all modes (SNAP-2194)
+ assert(ds.rdd.getNumPartitions === 1)
+ SnappyFunSuite.checkAnswer(ds, Seq(Row("['cbcin.com']", "[]")))
+ }
}
diff --git a/core/src/test/scala/io/snappydata/SnappyFunSuite.scala b/core/src/test/scala/io/snappydata/SnappyFunSuite.scala
index d9e203d881..352614d4aa 100644
--- a/core/src/test/scala/io/snappydata/SnappyFunSuite.scala
+++ b/core/src/test/scala/io/snappydata/SnappyFunSuite.scala
@@ -25,10 +25,12 @@ import io.snappydata.core.{FileCleaner, LocalSparkConf}
import io.snappydata.test.dunit.DistributedTestBase
import io.snappydata.test.dunit.DistributedTestBase.{InitializeRun, WaitCriterion}
import io.snappydata.util.TestUtils
+import org.scalatest.Assertions
import org.apache.spark.sql.catalyst.expressions.{Alias, And, AttributeReference, EqualNullSafe, EqualTo, Exists, ExprId, Expression, ListQuery, PredicateHelper, PredicateSubquery, ScalarSubquery}
import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, OneRowRelation, Sample}
-import org.apache.spark.sql.catalyst.util.sideBySide
+import org.apache.spark.sql.catalyst.util.{sideBySide, stackTraceToString}
+import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row}
// scalastyle:off
import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome, Retries}
// scalastyle:on
@@ -195,6 +197,47 @@ abstract class SnappyFunSuite
println(msg)
// scalastyle:on
}
+
+ def checkAnswer(df: => DataFrame, expectedAnswer: Seq[Row]): Unit =
+ SnappyFunSuite.checkAnswer(df, expectedAnswer)
+}
+
+object SnappyFunSuite extends Assertions {
+ def checkAnswer(df: => DataFrame, expectedAnswer: Seq[Row]): Unit = {
+ val analyzedDF = try df catch {
+ case ae: AnalysisException =>
+ if (ae.plan.isDefined) {
+ fail(
+ s"""
+ |Failed to analyze query: $ae
+ |${ae.plan.get}
+ |
+ |${stackTraceToString(ae)}
+ |""".stripMargin)
+ } else {
+ throw ae
+ }
+ }
+
+ assertEmptyMissingInput(analyzedDF)
+
+ QueryTest.checkAnswer(analyzedDF, expectedAnswer) match {
+ case Some(errorMessage) => fail(errorMessage)
+ case None =>
+ }
+ }
+
+ /**
+ * Asserts that a given [[Dataset]] does not have missing inputs in all the analyzed plans.
+ */
+ def assertEmptyMissingInput(query: Dataset[_]): Unit = {
+ assert(query.queryExecution.analyzed.missingInput.isEmpty,
+ s"The analyzed logical plan has missing inputs:\n${query.queryExecution.analyzed}")
+ assert(query.queryExecution.optimizedPlan.missingInput.isEmpty,
+ s"The optimized logical plan has missing inputs:\n${query.queryExecution.optimizedPlan}")
+ assert(query.queryExecution.executedPlan.missingInput.isEmpty,
+ s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}")
+ }
}
/**
diff --git a/core/src/test/scala/io/snappydata/core/LocalTestData.scala b/core/src/test/scala/io/snappydata/core/LocalTestData.scala
index 869080069c..59607c6d1a 100644
--- a/core/src/test/scala/io/snappydata/core/LocalTestData.scala
+++ b/core/src/test/scala/io/snappydata/core/LocalTestData.scala
@@ -78,9 +78,10 @@ object FileCleaner {
object LocalSparkConf {
def newConf(addOn: (SparkConf) => SparkConf = null): SparkConf = {
- val conf = new SparkConf().
- setIfMissing("spark.master", "local[4]").
- setAppName(getClass.getName)
+ val conf = new SparkConf()
+ .setIfMissing("spark.master", "local[4]")
+ .setIfMissing("spark.memory.debugFill", "true")
+ .setAppName(getClass.getName)
if (addOn != null) {
addOn(conf)
}
diff --git a/core/src/test/scala/org/apache/spark/sql/execution/AggregationSuite.scala b/core/src/test/scala/org/apache/spark/sql/execution/AggregationSuite.scala
index 1cbcaec4ad..80d9bba96f 100644
--- a/core/src/test/scala/org/apache/spark/sql/execution/AggregationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/sql/execution/AggregationSuite.scala
@@ -18,20 +18,13 @@ package org.apache.spark.sql.execution
import io.snappydata.{Property, SnappyFunSuite}
-import org.apache.spark.sql.{DataFrame, QueryTest, Row, SnappySession, SparkSession}
+import org.apache.spark.sql.{Row, SnappySession, SparkSession}
class AggregationSuite extends SnappyFunSuite {
- private def checkAnswer(result: DataFrame, expected: Seq[Row]): Unit = {
- QueryTest.checkAnswer(result, expected) match {
- case Some(errMessage) => throw new RuntimeException(errMessage)
- case None => // all good
- }
- }
-
test("AVG plan failure for nullables") {
val spark = new SparkSession(sc)
- val snappy = new SnappySession(sc)
+ val snappy = snc.snappySession
snappy.sql(s"set ${Property.ColumnBatchSize.name}=5000")
val checkDF = spark.range(10000).selectExpr("id", "(id * 12) as k",
@@ -73,4 +66,18 @@ class AggregationSuite extends SnappyFunSuite {
result = snappy.sql(query)
checkAnswer(result, expectedAnswer)
}
+
+ test("support for LATERAL VIEW in SnappyParser") {
+ val snappy = snc.snappySession
+ val json =
+ """{ "id": 1, "name": "A green door", "price": 12.50, "tags": ["home", "green"],
+ | "parts" : [ { "lock" : "One lock", "key" : "single key" },
+ | { "lock" : "2 lock", "key" : "2 key" } ] }""".stripMargin
+ val rdd = sc.makeRDD(Seq(json))
+ val ds = snappy.read.json(rdd)
+ ds.createOrReplaceTempView("json")
+ val res = snappy.sql("SELECT id, part.lock, part.key FROM json " +
+ "LATERAL VIEW explode(parts) partTable AS part")
+ checkAnswer(res, Seq(Row(1, "One lock", "single key"), Row(1, "2 lock", "2 key")))
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/sql/store/CreateIndexTest.scala b/core/src/test/scala/org/apache/spark/sql/store/CreateIndexTest.scala
index 77bbb23793..9adcf4f366 100644
--- a/core/src/test/scala/org/apache/spark/sql/store/CreateIndexTest.scala
+++ b/core/src/test/scala/org/apache/spark/sql/store/CreateIndexTest.scala
@@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.joins.HashJoinExec
import org.apache.spark.sql.execution.row.RowFormatRelation
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SnappyContext}
+import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode, SnappyContext}
class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach {
self =>
@@ -234,10 +234,10 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach {
val ds = snContext.createDataset(data)
ds.write.insertInto(tableName)
- QueryTest.checkAnswer(snContext.sql(s"select * from $tableName"), data)
- QueryTest.checkAnswer(snContext.sql(
+ checkAnswer(snContext.sql(s"select * from $tableName"), data)
+ checkAnswer(snContext.sql(
s"""select "value" from $tableName where `version`=111"""), Seq(Row("aaa")))
- QueryTest.checkAnswer(snContext.sql(s"""select * from $tableName where "version" >= 555"""),
+ checkAnswer(snContext.sql(s"""select * from $tableName where "version" >= 555"""),
Seq(Row(555, "ccc"), Row(666, "ccc")))
snContext.sql("drop table " + tableName)
@@ -261,10 +261,10 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach {
ds.write.insertInto(tableName)
- QueryTest.checkAnswer(snContext.table(tableName), data)
- QueryTest.checkAnswer(snContext.table(tableName).filter($"version" === 111)
+ checkAnswer(snContext.table(tableName), data)
+ checkAnswer(snContext.table(tableName).filter($"version" === 111)
.select($"value"), Seq(Row("aaa")))
- QueryTest.checkAnswer(snContext.table(tableName).filter($"version" >= 555),
+ checkAnswer(snContext.table(tableName).filter($"version" >= 555),
Seq(Row(555, "ccc"), Row(666, "ccc")))
snContext.setConf(SQLConf.CASE_SENSITIVE, false)
diff --git a/core/src/test/scala/org/apache/spark/sql/store/TokenizationTest.scala b/core/src/test/scala/org/apache/spark/sql/store/TokenizationTest.scala
index 0f43c6aca7..a6a43c80d7 100644
--- a/core/src/test/scala/org/apache/spark/sql/store/TokenizationTest.scala
+++ b/core/src/test/scala/org/apache/spark/sql/store/TokenizationTest.scala
@@ -54,6 +54,10 @@ class TokenizationTest
super.afterAll()
}
+ before {
+ SnappyTableStatsProviderService.suspendCacheInvalidation = true
+ }
+
after {
SnappyTableStatsProviderService.suspendCacheInvalidation = false
SnappySession.clearAllCache()
@@ -138,7 +142,6 @@ class TokenizationTest
}
test("like queries") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
val numRows = 100
createSimpleTableAndPoupulateData(numRows, s"$table", true)
@@ -150,11 +153,9 @@ class TokenizationTest
var result2 = snc.sql(q2).collect()
assert(!(result.sameElements(result2)) && result.length > 0)
}
- SnappyTableStatsProviderService.suspendCacheInvalidation = false
}
test("same session from different thread") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
val numRows = 2
createSimpleTableAndPoupulateData(numRows, s"$table", true)
@@ -184,7 +185,6 @@ class TokenizationTest
val cacheMap = SnappySession.getPlanCache.asMap()
assert( cacheMap.size() == 1)
}
- SnappyTableStatsProviderService.suspendCacheInvalidation = false
}
def getAllValidKeys(): Int = {
@@ -207,7 +207,6 @@ class TokenizationTest
}
test("Test some more foldable expressions and limit in where clause") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
val numRows = 10
createSimpleTableAndPoupulateData(numRows, s"$table", true)
val cacheMap = SnappySession.getPlanCache.asMap()
@@ -273,8 +272,7 @@ class TokenizationTest
snc.sql(s"drop table STAGING_AIRLINE")
}
- test("Test no tokenize if plan caching disabled in session") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
+ test("Test plan caching and tokenization disabled in session") {
val numRows = 10
createSimpleTableAndPoupulateData(numRows, s"$table", true)
@@ -300,15 +298,15 @@ class TokenizationTest
var query = s"select * from $table where a = 0"
newSession.sql(query).collect()
- assert(cacheMap.size() == 2)
+ assert(cacheMap.size() == 1)
query = s"select * from $table where a = 1"
newSession.sql(query).collect()
- assert(cacheMap.size() == 3)
+ assert(cacheMap.size() == 1)
query = s"select * from $table where b = 1"
var res2 = newSession.sql(query).collect()
- assert(cacheMap.size() == 4)
+ assert(cacheMap.size() == 1)
newSession.sql(s"set snappydata.sql.planCachingAll=false").collect()
assert(cacheMap.size() == 0)
@@ -320,7 +318,7 @@ class TokenizationTest
assert(r.get(0) == r.get(1) && r.get(2) == i)
})
}
- assert(cacheMap.size() == 10)
+ assert(cacheMap.size() == 0)
cacheMap.clear()
@@ -364,12 +362,10 @@ class TokenizationTest
snc.sql("set spark.sql.caseSensitive = false")
snc.sql("set schema = APP")
snc.sql(s"set snappydata.sql.tokenize=true").collect()
- SnappyTableStatsProviderService.suspendCacheInvalidation = false
}
}
test("Test tokenize and queryHints and noTokenize if limit or projection") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
val numRows = 10
createSimpleTableAndPoupulateData(numRows, s"$table", true)
@@ -519,7 +515,6 @@ class TokenizationTest
} finally {
snc.sql("set spark.sql.caseSensitive = false")
snc.sql("set schema = APP")
- SnappyTableStatsProviderService.suspendCacheInvalidation = false
}
logInfo("Successful")
}
@@ -529,7 +524,6 @@ class TokenizationTest
createAllTypeTableAndPoupulateData(numRows, s"$all_typetable")
try {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
val q = (0 until numRows).zipWithIndex.map { case (_, i) =>
s"select * from $all_typetable where s = 'abc$i'"
}
@@ -554,14 +548,12 @@ class TokenizationTest
} finally {
snc.sql("set spark.sql.caseSensitive = false")
snc.sql("set schema = APP")
- SnappyTableStatsProviderService.suspendCacheInvalidation = false
}
logInfo("Successful")
}
test("Test tokenize for sub-queries") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
snc.sql(s"set spark.sql.autoBroadcastJoinThreshold=1")
snc.sql(s"set spark.sql.crossJoin.enabled=true")
val numRows = 10
@@ -600,11 +592,9 @@ class TokenizationTest
assert( cacheMap.size() == 1)
logInfo("Successful")
- SnappyTableStatsProviderService.suspendCacheInvalidation = false
}
test("Test tokenize for joins and sub-queries") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
snc.sql(s"set spark.sql.autoBroadcastJoinThreshold=1")
val numRows = 10
createSimpleTableAndPoupulateData(numRows, s"$table", true)
@@ -625,7 +615,6 @@ class TokenizationTest
println(r.get(0) + ", " + r.get(1) + r.get(2) + ", " + r.get(3) + r.get(4) + ", " + r.get(5))
})
assert( cacheMap.size() == 1)
- SnappyTableStatsProviderService.suspendCacheInvalidation = false
assert(!result1.sameElements(result2))
assert(result1.length > 0)
assert(result2.length > 0)
@@ -669,14 +658,6 @@ class TokenizationTest
" then null else count(*) end as cnt from r where l.a = r.c) = 0").count == 4)
}
- test("Test tokenize for nulls") {
- logInfo("Successful")
- }
-
- test("Test tokenize for cast queries") {
- logInfo("Successful")
- }
-
private def createSimpleTableWithAStringColumnAndPoupulateData(numRows: Int, name: String,
dosleep: Boolean = false) = {
val strs = (0 to numRows).map(i => s"aa${i}bb")
@@ -746,7 +727,6 @@ class TokenizationTest
val colTableName = "airlineColTable"
test("Test broadcast hash joins and scalar sub-queries") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
try {
val ddlStr = "(YearI INT," + // NOT NULL
"MonthI INT," + // NOT NULL
@@ -908,12 +888,10 @@ class TokenizationTest
}
finally {
- SnappyTableStatsProviderService.suspendCacheInvalidation = false
}
}
test("Test BUG SNAP-1642") {
- SnappyTableStatsProviderService.suspendCacheInvalidation = true
val maxquery = s"select * from $table where a = (select max(a) from $table)"
val numRows = 10
createSimpleTableAndPoupulateData(numRows, s"$table", true)
diff --git a/docs/Images/kill_job.png b/docs/Images/kill_job.png
new file mode 100644
index 0000000000..48b05e2eb9
Binary files /dev/null and b/docs/Images/kill_job.png differ
diff --git a/docs/Images/snapshot_isolation.png b/docs/Images/snapshot_isolation.png
new file mode 100644
index 0000000000..ef15ac6ede
Binary files /dev/null and b/docs/Images/snapshot_isolation.png differ
diff --git a/docs/Images/transactions_read.png b/docs/Images/transactions_read.png
new file mode 100644
index 0000000000..1bcf1e4d6a
Binary files /dev/null and b/docs/Images/transactions_read.png differ
diff --git a/docs/Images/transactions_write.png b/docs/Images/transactions_write.png
new file mode 100644
index 0000000000..4c0440fb9e
Binary files /dev/null and b/docs/Images/transactions_write.png differ
diff --git a/docs/Images/vsd/vsd-connection-stats.png b/docs/Images/vsd/vsd-connection-stats.png
new file mode 100755
index 0000000000..e52d530350
Binary files /dev/null and b/docs/Images/vsd/vsd-connection-stats.png differ
diff --git a/docs/Images/vsd/vsd_applications.png b/docs/Images/vsd/vsd_applications.png
new file mode 100755
index 0000000000..452d78b6d0
Binary files /dev/null and b/docs/Images/vsd/vsd_applications.png differ
diff --git a/docs/Images/vsd/vsd_applications_2.png b/docs/Images/vsd/vsd_applications_2.png
new file mode 100755
index 0000000000..f87ab52099
Binary files /dev/null and b/docs/Images/vsd/vsd_applications_2.png differ
diff --git a/docs/Images/vsd/vsd_cpu.png b/docs/Images/vsd/vsd_cpu.png
new file mode 100755
index 0000000000..c9c00db571
Binary files /dev/null and b/docs/Images/vsd/vsd_cpu.png differ
diff --git a/docs/Images/vsd/vsd_memory.png b/docs/Images/vsd/vsd_memory.png
new file mode 100755
index 0000000000..03a433ff9a
Binary files /dev/null and b/docs/Images/vsd/vsd_memory.png differ
diff --git a/docs/Images/vsd/vsd_memory_2.png b/docs/Images/vsd/vsd_memory_2.png
new file mode 100755
index 0000000000..ed8322ebfe
Binary files /dev/null and b/docs/Images/vsd/vsd_memory_2.png differ
diff --git a/docs/Images/vsd/vsd_statements.png b/docs/Images/vsd/vsd_statements.png
new file mode 100755
index 0000000000..0243c9d48d
Binary files /dev/null and b/docs/Images/vsd/vsd_statements.png differ
diff --git a/docs/Images/vsd/vsd_tables.png b/docs/Images/vsd/vsd_tables.png
new file mode 100755
index 0000000000..b1b2435c45
Binary files /dev/null and b/docs/Images/vsd/vsd_tables.png differ
diff --git a/docs/Images/vsd/vsd_tables_2.png b/docs/Images/vsd/vsd_tables_2.png
new file mode 100755
index 0000000000..9ee5be9cfb
Binary files /dev/null and b/docs/Images/vsd/vsd_tables_2.png differ
diff --git a/docs/Images/vsd/vsd_tables_3.png b/docs/Images/vsd/vsd_tables_3.png
new file mode 100755
index 0000000000..d7c5ddebb7
Binary files /dev/null and b/docs/Images/vsd/vsd_tables_3.png differ
diff --git a/docs/Images/vsd/vsd_transactions.png b/docs/Images/vsd/vsd_transactions.png
new file mode 100755
index 0000000000..7926054fe7
Binary files /dev/null and b/docs/Images/vsd/vsd_transactions.png differ
diff --git a/docs/Images/vsd/vsd_transactions_2.png b/docs/Images/vsd/vsd_transactions_2.png
new file mode 100755
index 0000000000..bfb275579b
Binary files /dev/null and b/docs/Images/vsd/vsd_transactions_2.png differ
diff --git a/docs/best_practices/important_settings.md b/docs/best_practices/important_settings.md
index c7c43b7f8e..2578ce1c67 100644
--- a/docs/best_practices/important_settings.md
+++ b/docs/best_practices/important_settings.md
@@ -89,7 +89,7 @@ sudo swapon /var/swapfile.1
```
-## SnappyData Smart Connector mode and Local mode Settings
+## SnappyData Smart Connector mode and Local mode settings
### Managing Executor Memory
For efficient loading of data from a Smart Connector application or a Local Mode application, all the partitions of the input data are processed in parallel by making use of all the available cores. Further, to have better ingestion speed, small internal columnar storage structures are created in the Spark application's cluster itself, which is then directly inserted into the required buckets of the column table in the SnappyData cluster.
@@ -99,8 +99,24 @@ For example, if there are 32 cores for the Smart Connector application and the n
You can modify this setting in the `spark.executor.memory` property. For more information, refer to the [Spark documentation](https://spark.apache.org/docs/latest/configuration.html#available-properties).
### JVM settings for optimal performance
-The JVM setting is set by default and the following is recommended only for local mode:
+The following JVM settings are set by default and is recommended only for local mode.
+
+* -XX:+UseParNewGC
+* -XX:+UseConcMarkSweepGC
+* -XX:CMSInitiatingOccupancyFraction=50
+* -XX:+CMSClassUnloadingEnabled
+* -XX:-DontCompileHugeMethods
+* -XX:CompileThreshold=2000
+* -XX:+UnlockDiagnosticVMOptions
+* -XX:ParGCCardsPerStrideChunk=4k
+* -Djdk.nio.maxCachedBufferSize=131072
+
+**Example**:
+
+```
+-XX:-DontCompileHugeMethods -XX:+UnlockDiagnosticVMOptions -XX:ParGCCardsPerStrideChunk=4k
+```
+CMS collector with ParNew is used by default as above and recommended. GC settings set above have been seen to work best in representative workloads and can be tuned further as per application. For enterprise users `off-heap` is recommended for best performance.
-```-XX:-DontCompileHugeMethods -XX:+UnlockDiagnosticVMOptions -XX:ParGCCardsPerStrideChunk=4k```
Set in the **conf/locators**, **conf/leads**, and **conf/servers** file.
\ No newline at end of file
diff --git a/docs/best_practices/optimizing_query_latency.md b/docs/best_practices/optimizing_query_latency.md
index 89821f8a3e..feba8b1ce0 100644
--- a/docs/best_practices/optimizing_query_latency.md
+++ b/docs/best_practices/optimizing_query_latency.md
@@ -95,6 +95,9 @@ For persistent tables, setting this to 'true' will overflow the table evicted ro
Refer to [CREATE TABLE](../reference/sql_reference/create-table.md) link to understand how to configure [OVERFLOW](../reference/sql_reference/create-table.md#overflow) and [EVICTION_BY](../reference/sql_reference/create-table.md#eviction-by) clauses.
+!!! Tip:
+ By default eviction is set to `Overflow`.
+
## Known Limitation
### Change NOT IN queries to use NOT EXISTS if possible
diff --git a/docs/best_practices/setup_cluster.md b/docs/best_practices/setup_cluster.md
index da25345e90..4d3722f223 100644
--- a/docs/best_practices/setup_cluster.md
+++ b/docs/best_practices/setup_cluster.md
@@ -32,7 +32,7 @@ SnappyData uses an optimization method which clubs multiple partitions on a sing
In SnappyData, multiple queries can be executed concurrently, if they are submitted by different threads or different jobs. For concurrent queries, SnappyData uses fair scheduling to manage the available resources such that all the queries get a fair distribution of resources.
-For example: In the image below, 6 cores are available on 3 systems, and 2 jobs have 4 tasks each. Because of fair scheduling, both jobs get 3 cores and hence three tasks per job execute concurrently.
+For example, In the image below, 6 cores are available on 3 systems, and 2 jobs have 4 tasks each. Because of fair scheduling, both jobs get 3 cores and hence three tasks per job execute concurrently.
Pending tasks have to wait for completion of the current tasks and are assigned to the core that is first available.
@@ -47,7 +47,9 @@ SnappyData out of the box comes configured with two execution pools:
* Default pool: This is the pool that is used for the remaining requests.
-Two cores are statically assigned to the low latency pool. Also, the low latency pool has weight twice that of the default pool. Thus, if there are 30 cores available to an executor for a query that has 30 partitions, only 28 would be assigned to it and two cores would be reserved to not starve the low latency queries. When the system has both low latency and normal queries, 20 cores are used for the low latency queries as it has got higher priority (weight=2).
+Two cores are statically assigned to the low latency pool. Also, the low latency pool has weight twice that of the default pool. Thus, if there are 30 cores available to an executor for a query that has 30 partitions, only 28 would be assigned to it and two cores would be reserved to not starve the low latency queries. When the system has both low latency and normal queries, 20 cores are used for the low latency queries as it has higher priority (weight=2).
+If a query requires all 30 partitions and no low latency queries are running at that time, all 30 cores are assigned to the first query. However, when a low latency query is assigned, the scheduler does its best to allocate cores as soon as tasks from the earlier query finish.
+
Applications can explicitly configure to use a particular pool for the current session using a SQL configuration property, `snappydata.scheduler.pool`. For example, the `set snappydata.scheduler.pool=lowlatency` command sets the pool as low latency pool for the current session.
diff --git a/docs/best_practices/transactions_best_practices.md b/docs/best_practices/transactions_best_practices.md
new file mode 100755
index 0000000000..3066d28a39
--- /dev/null
+++ b/docs/best_practices/transactions_best_practices.md
@@ -0,0 +1,31 @@
+# SnappyData Distributed Transactions
+
+
+## Using Transactions
+
+- For high performance, mimimize the duration of transactions to avoid conflicts with other concurrent transactions. If atomicity for only single row updates is required, then completely avoid using transactions because SnappyData provides atomicity and isolation for single rows without transactions.
+
+- When using transactions, keep the number of rows involved in the transaction as low as possible. SnappyData acquires locks eagerly, and long-lasting transactions increase the probability of conflicts and transaction failures. Avoid transactions for large batch update statements or statements that effect a lot of rows.
+
+- Unlike in traditional databases, SnappyData transactions can fail with a conflict exception on writes instead of on commit. This choice makes sense given that the outcome of the transaction has been determined to fail.
+
+- To the extent possible, model your database so that most transactions operate on colocated data. When all transactional data is on a single member, then stricter isolation guarantees are provided.
+
+
+- DDL Statements in a transaction
+ SnappyData permits schema and data manipulation statements (DML) within a single transaction. A data definition statement (DDL) is not automatically committed when it is performed, but participates in the transaction within which it is issued.
+
+ Although the table itself becomes visible in the system immediately, it acquires exclusive locks on the system tables and the affected tables on all the members in the cluster, so that any DML operations in other transactions will block and wait for the table's locks.
+
+ For example, if a new index is created on a table in a transaction, then all other transactions that refer to that table wait for the transaction to commit or roll back. Because of this behavior, as a best practice you should keep transactions that involve DDL statements short (preferably in a single transaction by itself).
+
+
+## Using Snapshot Isolation
+
+To the extent possible, model your database so that most transactions operate on colocated data. When all transactional data is on a single member, then stricter isolation guarantees are provided. In case of failure, the rollback is complete and not partial.
+
+**More information**
+
+- [Overview of SnappyData Distributed Transactions](../consistency/transactions_about.md)
+
+- [How to use Transactions Isolation Levels](../howto/use_transactions_isolation_levels.md)
\ No newline at end of file
diff --git a/docs/concepts/network_partition.md b/docs/concepts/network_partition.md
new file mode 100755
index 0000000000..6d4c12c304
--- /dev/null
+++ b/docs/concepts/network_partition.md
@@ -0,0 +1,23 @@
+# Detecting and Handling Network Segmentation ("Split Brain")
+
+When network segmentation occurs, a distributed system that does not handle the partition condition properly allows multiple subgroups to form. This condition can lead to numerous problems, including distributed applications operating on inconsistent data.
+
+For example, because thin clients connecting to a server cluster are not tied into the membership system, a client might communicate with servers from multiple subgroups. Or, one set of clients might see one subgroup of servers while another set of clients cannot see that subgroup but can see another one.
+
+SnappyData handles this problem by allowing only one subgroup to form and survive. The distributed systems and caches of other subgroups are shut down as quickly as possible. Appropriate alerts are raised through the SnappyData logging system to alert administrators to take action.
+
+Network partition detection in SnappyData is based on the concept of a lead member and a group management coordinator. The coordinator is a member that manages entry and exit of other members of the distributed system. For network partition detection, the coordinator is always a SnappyData locator. The lead member is always the oldest member of the distributed system that does not have a locator running in the same process. Given this, two situations causes SnappyData to declare a network partition:
+
+* If both a locator and the lead member leave the distributed system abnormally within a configurable period of time, a network partition is declared and the caches of members who are unable to see the locator and the lead member are immediately closed and disconnected.
+
+ If a locator or lead member's distributed system is shut down normally, SnappyData automatically elects a new one and continues to operate.
+
+* If no locator can be contacted by a member, it declares a network partition has occurred, closes itself, and disconnects from the distributed system.
+
+You enable network partition detection by setting the [enable-network-partition-detection](/reference/configuration_parameters/enable-network-partition-detection.md) distributed system property to **true**. Enable network partition detection in all locators and in any other process that should be sensitive to network partitioning. Processes that do not have network partition detection enabled are not eligible to be the lead member, so their failure does not trigger declaration of a network partition.
+
+
+!!! Note:
+ The distributed system must contain locators to enable network partition detection.
+
+
diff --git a/docs/configuration.md b/docs/configuration.md
index 5f089b9d4f..22d6e0f1fe 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -27,3 +27,5 @@ The following topics are covered in this section:
* [SSL Setup for Client-Server](configuring_cluster/ssl_setup.md)
* [List of Properties](configuring_cluster/property_description.md)
+
+* [Firewalls and Connections](configuring_cluster/firewalls_connections.md)
\ No newline at end of file
diff --git a/docs/configuring_cluster/firewalls_connections.md b/docs/configuring_cluster/firewalls_connections.md
new file mode 100755
index 0000000000..3f1fa64a65
--- /dev/null
+++ b/docs/configuring_cluster/firewalls_connections.md
@@ -0,0 +1,65 @@
+# Firewalls and Connections
+
+You may face possible connection problems that can result from running a firewall on your machine.
+
+SnappyData is a network-centric distributed system, so if you have a firewall running on your machine it could cause connection problems. For example, your connections may fail if your firewall places restrictions on inbound or outbound permissions for Java-based sockets. You may need to modify your firewall configuration to permit traffic to Java applications running on your machine. The specific configuration depends on the firewall you are using.
+
+As one example, firewalls may close connections to SnappyData due to timeout settings. If a firewall senses no activity in a certain time period, it may close a connection and open a new connection when activity resumes, which can cause some confusion about which connections you have.
+
+## Firewall and Port Considerations
+
+You can configure and limit port usage for situations that involve firewalls, for example, between client-server or server-server connections.
+
+
+Make sure your port settings are configured correctly for firewalls. For each SnappyData member, there are two different port settings you may need to be concerned with regarding firewalls:
+
+- The port that the server or locator listens on for client connections. This is configurable using the `-client-port` option to the snappy server or snappy locator command.
+
+- The peer discovery port. SnappyData members connect to the locator for peer-to-peer messaging. The locator port is configurable using the `-peer-discovery-port` option to the snappy server or snappy locator command.
+
+ By default, SnappyData servers and locators discover each other on a pre-defined port (10334) on the localhost.
+
+### Limiting Ephemeral Ports for Peer-to-Peer Membership
+
+By default, SnappyData utilizes *ephemeral* ports for UDP messaging and TCP failure detection. Ephemeral ports are temporary ports assigned from a designated range, which can encompass a large number of possible ports. When a firewall is present, the ephemeral port range usually must be limited to a much smaller number, for example six. If you are configuring P2P communications through a firewall, you must also set each the tcp port for each process and ensure that UDP traffic is allowed through the firewall.
+
+### Properties for Firewall and Port Configuration
+
+#### Store Layer
+
+This following tables contain properties potentially involved in firewall behavior, with a brief description of each property. The [Configuration Properties](../reference/configuration_parameters/config_parameters.md) section contains detailed information for each property.
+
+| Configuration Area | Property or Setting | Definition |
+|--------|--------|--------|
+|peer-to-peer config|[locators](../reference/configuration_parameters/locators.md)|The list of locators used by system members. The list must be configured consistently for every member of the distributed system.|
+|peer-to-peer config|[membership-port-range](../reference/configuration_parameters/membership-port-range.md)|The range of ephemeral ports available for unicast UDP messaging and for TCP failure detection in the peer-to-peer distributed system.|
+|member config|[-J-Dgemfirexd.hostname-for-clients](../configuring_cluster/property_description.md#host-name)|Hostname or IP address to pass to the client as the location where the server is listening.|
+|member config|[client-port](../reference/command_line_utilities/store-run/) option to the [snappy server](../configuring_cluster/configuring_cluster.md#configuring-data-servers) and [snappy locator](../configuring_cluster/configuring_cluster.md#configuring-locators) commands|Port that the member listens on for client communication.|
+|Locator|[locator command](../configuring_cluster/configuring_cluster.md#configuring-locators)|10334|
+
+#### Spark Layer
+
+The following table lists the Spark properties you can set to configure the ports required for Spark infrastructure.Refer to [Spark Configuration](https://spark.apache.org/docs/latest/configuration.html) in the official documentation for detailed information.
+
+| Property | Default |Description|
+|--------|--------|--------|
+|spark.blockManager.port |random|Port for all block managers to listen on. These exist on both the driver and the executors.|
+|spark.driver.blockManager.port |(value of spark.blockManager.port)|Driver-specific port for the block manager to listen on, for cases where it cannot use the same configuration as executors.|
+|spark.driver.port |random | Port for the driver to listen on. This is used for communicating with the executors and the standalone Master.|
+|spark.port.maxRetries|16|Maximum number of retries when binding to a port before giving up. When a port is given a specific value (non 0), each subsequent retry will increment the port used in the previous attempt by 1 before retrying. This essentially allows it to try a range of ports from the start port specified to port + maxRetries.
+|spark.shuffle.service.port |7337|Port on which the external shuffle service will run.|
+|spark.ui.port |4040| Port for your application's dashboard, which shows memory and workload data.|
+|spark.ssl.[namespace].port |None|The port where the SSL service will listen on.
The port must be defined within a namespace configuration; see SSL Configuration for the available namespaces. When not set, the SSL port will be derived from the non-SSL port for the same service. A value of "0" will make the service bind to an ephemeral port.|
+|spark.history.ui.port|The port to which the web interface of the history server binds.|18080|
+|SPARK_MASTER_PORT |Start the master on a different port.|Default: 7077|
+|SPARK_WORKER_PORT |Start the Spark worker on a specific port.|(Default: random|
+
+## Locators and Ports
+
+The ephemeral port range and TCP port range for locators must be accessible to members through the firewall.
+
+Locators are used in the peer-to-peer cache to discover other processes. They can be used by clients to locate servers as an alternative to configuring clients with a collection of server addresses and ports.
+
+Locators have a TCP/IP port that all members must be able to connect to. They also start a distributed system and so need to have their ephemeral port range and TCP port accessible to other members through the firewall.
+
+Clients need only be able to connect to the locator's locator port. They don't interact with the locator's distributed system; clients get server names and ports from the locator and use these to connect to the servers. For more information, see [Using Locators](configuring_cluster.md#configuring-locators).
\ No newline at end of file
diff --git a/docs/configuring_cluster/property_description.md b/docs/configuring_cluster/property_description.md
index f1a6c8f812..54ee8bc1d2 100644
--- a/docs/configuring_cluster/property_description.md
+++ b/docs/configuring_cluster/property_description.md
@@ -13,7 +13,7 @@ Below is a list of properties that can be set to configure the cluster. These pr
|-eviction-off-heap-percentage|Sets the off-heap memory usage percentage threshold, 0-100, that the Resource Manager uses to start evicting data from off-heap memory. By default, the eviction threshold is 81% of whatever is set for `-critical-off-heap-percentage`. Use this switch to override the default.|Server|
|-heap-size| Sets the maximum heap size for the Java VM, using SnappyData default resource manager settings. For example, -heap-size=1024m. If you use the `-heap-size` option, by default SnappyData sets the critical-heap-percentage to 90% of the heap size, and the `eviction-heap-percentage` to 81% of the `critical-heap-percentage`. SnappyData also sets resource management properties for eviction and garbage collection if they are supported by the JVM. |ServerLeadLocator|
|-J|JVM option passed to the spawned SnappyData server JVM. For example, use -J-Xmx1024m to set the JVM heap to 1GB.|ServerLeadLocator|
-|-J-Dgemfirexd.hostname-for-clients|Hostname or IP address that is sent to clients so they can connect to the locator. The default is the `bind-address` of the locator.|Server|
+|-J-Dgemfirexd.hostname-for-clients|Hostname or IP address that is sent to clients so they can connect to the locator. The default is the `bind-address` of the locator.|Server|
|-locators|List of locators as comma-separated host:port values used to communicate with running locators in the system and thus discover other peers of the distributed system. The list must include all locators in use and must be configured consistently for every member of the distributed system.|ServerLeadLocator|
|-log-file|Path of the file to which this member writes log messages (default is snappyserver.log in the working directory)|ServerLeadLocator|
|-memory-size|Specifies the total memory that can be used by the node for column storage and execution in off-heap. The default value is 0 (OFF_HEAP is not used by default)|ServerLead|
diff --git a/docs/consistency/transactions_about.md b/docs/consistency/transactions_about.md
new file mode 100755
index 0000000000..570cabe31a
--- /dev/null
+++ b/docs/consistency/transactions_about.md
@@ -0,0 +1,22 @@
+# Overview of SnappyData Distributed Transactions
+
+SnappyData supports transaction characteristics of isolation and atomicity. Transactions are supported using JDBC/ODBC through statements such as SET [autocommit](../reference/interactive_commands/autocommit.md), [SET Isolation](../reference/sql_reference/set-isolation.md), [COMMIT](../reference/interactive_commands/commit.md), and [ROLLBACK](../reference/interactive_commands/rollback.md).
+
+
+!!! Note:
+ - Full distributed transactions (that is, multiple update SQL statements in one logical transaction) is currently supported only for row tables.
+
+ - Column tables only support single statement implicit transactions. That is, every DML (insert/update/delete) statement is executed in a implicit transaction. The DML statement can in-fact be a multi-row statement and is executed with "all or nothing" semantics.
+
+ - Transactions execution do not depend on a central locking facility and is highly scalable.
+
+ - SnappyData supports high concurrency for transactions. Readers (queries) do not acquire locks and isolated from concurrent transactions using an MVCC implementation.
+
+ - Currently, demarcated transactions (Commit, rollback) is only supported through the JDBC and ODBC API. Support for commit/rollback will be added to the Spark API will be added in a later release.
+
+**Additional Information**
+
+* [How to use Transactions Isolation Levels](../howto/use_transactions_isolation_levels.md)
+
+* [Best Practices for using Distributed Transactions and Snapshot Isolation](../best_practices/transactions_best_practices.md)
+
diff --git a/docs/consistency/using_snapshot_isolation_column.md b/docs/consistency/using_snapshot_isolation_column.md
new file mode 100644
index 0000000000..226e8bd0a4
--- /dev/null
+++ b/docs/consistency/using_snapshot_isolation_column.md
@@ -0,0 +1,58 @@
+# Lock-free Queries using MVCC (multi-version concurrency control) and Snapshot Isolation for Column Tables
+
+!!!Note:
+ Snapshot isolation is supported only for column tables.
+
+As the term suggests, all queries in the system operate on a snapshot view of the database. This is, even if concurrent updates are in progress, the querying system gets a non-changing view of the state of the database at the moment in time when the query is executed. The snapshot is partition wise. The snapshot of the partition is taken the moment the query accesses the partition. This behavior is set by default for column tables and cannot be modified.
+
+
+## How the Snapshot Model Works
+
+SnappyData maintains a version vector for each of the table on every node. The version information for each row of the table is also maintained.
+
+When a user query is executed, a snapshot is taken of the version vector of all the tables on the node on which the query is executed. The write operation modifies the row, increments its version while still maintaining a reference to the older version.
+
+At the time of commit, the version information is published under a lock so that all the changes of an operation is published atomically. Older rows are cleaned periodically once it is made sure that there are no operations that require these older rows.
+
+The read operations compare the version of each row to the ones in its snapshot and return the row whose version is same as the snapshot.
+
+In case of failure, the versions are not published, which makes the rows invisible to any future operations. A new node joining the cluster copies all the committed rows from the existing node making sure that any snapshot will see only committed data.
+
+The following image represents the functioning of read and write operations in the Snapshot isolation model:
+
+![Snapshot Isolation](../Images/snapshot_isolation.png)
+
+By default, all individual operations (read/write) on column table have snapshot isolation with `autocommit` set to `ON`. This means, in case of a failure the user operation fails and [rollback](../reference/interactive_commands/rollback.md) is triggered.
+You cannot set [autocommit](../reference/interactive_commands/autocommit.md) to `Off`. Snapshot isolation ensures that changes made, after the ongoing operation has taken a snapshot is not visible partially or totally.
+If there are concurrent updates in a row, then the last commit is used.
+
+!!! Note:
+ To get per statement transactional behavior, all the write operations can span only one partition.
+
+ However, if you have operations that span multiple partitions, then, ensure that:
+
+ * In case of failure on one partition, the operation is retried on another copy of the same partition. Set [redundancy](../reference/sql_reference/create-table.md) to more than 0, if transactional behavior with operations spanning more than one partition is required.
+
+ * If the operation fails on all the redundant copies of a partition and the same operation succeeds on some of the other partitions, then, partial rollback is initiated.
+ In this case, you can retry the operation at the application level.
+
+
+## Rollback Behavior and Member Failures
+
+In column tables, roll back is performed in case of low memory. If the operation fails due to low memory, automatic roll back is initiated.
+
+
+## Snapshot Limitations
+
+The following limitations have been reported:
+
+- For column tables, snapshot isolation is enabled by default, but the full range of fault tolerance is not yet implemented. It is assumed that at least one copy of a partition is always available (redundant members are available) in the event of member failures.
+
+- Write-write conflict is not detected. The last write option is applied.
+
+- Multi-statement is not supported.
+
+
+## Snapshot Isolation with SELECT FOR UPDATE
+
+The `SELECT FOR UPDATE` statement and other statements that implicitly place locks are not supported for column tables, and snapshot isolation is applied by default for updates. In case of multiple concurrent updates, the last update is applied.
diff --git a/docs/consistency/using_transactions_row.md b/docs/consistency/using_transactions_row.md
new file mode 100644
index 0000000000..e875779160
--- /dev/null
+++ b/docs/consistency/using_transactions_row.md
@@ -0,0 +1,122 @@
+# How Transactions Work for Row Tables
+
+!!! Note:
+ Distributed transaction is supported only for row tables.
+
+There is no centralized transaction coordinator in SnappyData. Instead, the member on which a transaction was started acts as the coordinator for the duration of the transaction. If the application updates one or more rows, the transaction coordinator determines which owning members are involved, and acquires local "write" locks on all of the copies of the rows. At commit time, all changes are applied to the local store and any redundant copies. If another concurrent transaction attempts to change one of the rows, the local "write" acquisition fails for the row, and that transaction is automatically rolled back.
+
+Unlike traditional distributed databases, SnappyData does not use write-ahead logging for transaction recovery in case the commit fails during replication or redundant updates to one or more members. The most likely failure scenario is one where the member is unhealthy and gets forced out of the distributed system, guaranteeing the consistency of the data. When the failed member comes back online, it automatically recovers the replicated/redundant data set and establishes coherency with the other members. If all copies of some data go down before the commit is issued, then this condition is detected using the group membership system, and the transaction is rolled back automatically on all members.
+
+!!! Note:
+ SnappyData does not support transactions is new data store members are added while in progress. If you add a new member to the cluster in the middle of a transaction and the new member is involved in the transaction (e.g. owns a partition of the data or is a replica), SnappyData implicitly rolls back the transaction and throws an SQLException (SQLState: "X0Z05").
+
+
+The following images represent the functioning of read and write operations in the transaction model:
+
+![Read Operations](../Images/transactions_read.png)
+
+![Write Operations](../Images/transactions_write.png)
+
+
+## Using Transactions for Row Tables
+
+Transactions specify an [isolation level](../reference/sql_reference/set-isolation.md) that defines the degree to which one transaction must be isolated from resource or data modifications made by other transactions. The transaction isolation levels define the type of locks acquired on read operations. Only one of the isolation level options can be set at a time, and it remains set for that connection until it is explicitly changed.
+
+!!! Note:
+
+ * If you set the isolation level to `READ_COMMITTED` or `REPEATABLE_READ`, queries on column table report an error if [autocommit](../reference/interactive_commands/autocommit.md) is set to off (false). Queries on column tables are supported when isolation level is set to `READ_COMMITTED` or `REPEATABLE_READ` and autocommit is set to **true**.
+
+ * DDL execution (for example [CREATE TABLE](../reference/sql_reference/create-table.md) /[DROP TABLE](../reference/sql_reference/drop-table.md)) is not allowed when `autocommit` is set to `false` and transaction isolation level is `READ_COMMITTED` or `REPEATABLE_READ`. DDL commands reports syntax error in such cases. DDL execution is allowed if `autocommit` is `true` for `READ_COMMITTED` or `REPEATABLE_READ` isolation levels.
+
+
+The following isolation levels are supported for row tables:
+
+| Isolation level | Description |
+|--------|--------|
+|NONE|Default isolation level. The Database Engine uses shared locks to prevent other transactions from modifying rows while the current transaction is running a read operation. |
+|READ_COMMITTED|SnappyData ensures that ongoing transactional as well as non-transactional (isolation-level NONE) operations never read uncommitted (dirty) data. SnappyData accomplishes this by maintaining transactional changes in a separate transaction state that are applied to the actual data-store for the table only at commit time. SnappyData detects only Write-Write conflicts while in READ_COMMITTED isolation level. In READ COMMITTED, a read view is created at the start of each statement and lasts only as long as each statement execution.|
+|REPEATABLE_READ|In this isolation level, a lock-based concurrency control DBMS implementation keeps read and write locks (acquired on selected data) until the end of the transaction. In REPEATABLE READ every lock acquired during a transaction is held for the duration of the transaction.|
+
+For more information, see, [SET ISOLATION](../reference/sql_reference/set-isolation.md)
+
+
+## Rollback Behavior and Member Failures
+
+Within the scope of a transaction, SnappyData automatically initiates a rollback if it encounters a constraint violation.
+
+Any errors that occur while parsing queries or while binding parameters in a SQL statement *do not* cause a rollback. For example, a syntax error that occurs while executing a SQL statement does not cause previous statements in the transaction to rollback. However, a column constraint violation would cause all previous SQL operations in the transaction to roll back.
+
+
+### Handling Member Failures
+
+The following steps describe specific events that can occur depending on which member fails and when the failure occurs during a transaction:
+
+1. If the transaction coordinator member fails before a commit is fired, then each of the cohort members aborts the ongoing transaction.
+
+2. If a participating member fails before a commit is fired, then it is simply ignored. If the copies/replicas go to zero for certain keys, then any subsequent update operations on those keys throw an exception as in the case of non-transactional updates. If a commit is fired in this state, then the whole transaction is aborted.
+
+3. If the transaction coordinator fails before completing the commit process (with or without sending the commit message to all cohorts), the surviving cohorts determine the outcome of the transaction.
+
+ If all of the cohorts are in the PREPARED state and successfully apply changes to the cache without any unique constraint violations, the transaction is committed on all cohorts. Otherwise, if any member reports failure or the last copy the associated rows go down during the PREPARED state, the transaction is rolled back on all cohorts.
+
+4. If a participating member fails before acknowledging to the client, then the transaction continues on other members without any interruption. However, if that member contains the last copy of a table or bucket, then the transaction is rolled back.
+
+5. The transaction coordinator might also fail while executing a rollback operation. In this case, the client would see such a failure as an SQLState error. If the client was performing a SELECT statement in a transaction, the member failure would result in SQLState error X0Z01::
+
+ ```
+ ERROR X0Z01: Node 'node-name' went down or data no longer available while iterating the results (method 'rollback()'). Please retry the operation.
+ ```
+
+ Clients that were performing a DML statement in the context of a transaction would fail with one of the SQLState errors: X0Z05, X0Z16, 40XD2, or 40XD0.
+
+ !!! Note:
+ Outside the scope of a transaction, a DML statement would not see an exception due to a member failure. Instead, the statement would be automatically retried on another SnappyData member. However, SELECT statements would receive the X0Z01 statement even outside of a transaction.
+
+If this type of failure occurs, the remaining members of the SnappyData distributed system clean-up the open transactions for the failed node, and no additional steps are needed to recover from the failure.
+
+!!! Note:
+ In this release of SnappyData, a transaction fails if any of the cohorts depart abnormally.
+
+
+### Other Rollback Scenarios
+
+SnappyData may cancel an executing statement due to low memory, a timeout, or a manual request to cancel the statement.
+
+If a statement that is being executed within the context of a transaction is canceled due to low memory or a manual cancellation request, then SnappyData rolls back the associated transaction.
+
+!!! Note:
+ SnappyData does not roll back a transaction if a statement is canceled due to a timeout.
+
+
+## Transaction Functionality and Limitations
+
+In this release of SnappyData, the scope for transactional functionality is:
+
+- The result set that is obtained from executing a query should either be completely consumed, or the result set is explicitly closed. Otherwise, DDL operations wait until the ResultSet is garbage-collected.
+
+- Transactions for persistent tables are enabled by default, but the full range of fault tolerance is not yet implemented. It is assumed that at least one copy of a row is always available (redundant members are available) in the event of member failures.
+
+- SQL statements that implicitly place locks, such as `select for update`, are not supported outside of transactions (default isolation level).
+
+- The supported isolation levels are 'READ COMMITTED' and 'READ UNCOMMITTED' where both behave as 'READ COMMITTED.' Autocommit is OFF by default in SnappyData, unlike in other JDBC drivers.
+
+- Transactions always do "write-write" conflict detection at operation or commit time. Applications do not need to use `select for update` or explicit locking to get this behavior, as compared to other databases. (`select for update` is not supported outside of a transaction.)
+
+- Nested transactions and savepoints are not supported.
+
+- SnappyData does not support transactions on partitioned tables that are configured with the DESTROY evict action. This restriction exists because the requirements of ACID transactions can conflict with the semantics of destroying evicted entries. For example, a transaction may need to update a number of entries that is greater than the amount allowed by the eviction setting. Transactions are supported with the OVERFLOW evict action because the required entries can be loaded into memory as necessary to support transaction semantics.
+
+- SnappyData does not restrict concurrent non-transactional clients from updating tables that may be involved in transactions. This is by design, to maintain very high performance when no transactions are in use. If an application uses transactions on a table, make sure the application consistently uses transactions when updating that table.
+
+- All DML on a single row is atomic in nature inside or outside of transactions.
+
+- There is a small window during a commit when the committed set is being applied to the underlying table and concurrent readers, which do not consult any transactional state, have visibility to the partially-committed state. The larger the transaction, the larger the window. Also, transaction state is maintained in a memory-based buffer. The shorter and smaller the transaction, the less likely the transaction manager will run short on memory.
+
+
+## Transactions with SELECT FOR UPDATE
+
+The `SELECT FOR UPDATE` statement and other statements that implicitly place locks are not supported outside of a transaction (default isolation level).
+
+A SELECT FOR UPDATE begins by obtaining a read lock, which allows other transactions to possibly obtain read locks on the same data. A transaction's read lock is immediately upgraded to an exclusive write lock after a row is qualified for the SELECT FOR UPDATE statement. At this point, any other transactions that obtained a read lock on the data receive a conflict exception and can roll back and release their locks.
+
+The transaction that has the exclusive lock can successfully commit only after all other read locks on the table have been released. In some cases, it is possible for one transaction to obtain an exclusive lock for data on one SnappyData member, while another transaction obtains an exclusive lock on a different member. In this case, both transactions will fail during the commit.
\ No newline at end of file
diff --git a/docs/extra.css b/docs/extra.css
index 36a4f0b46c..699f4bf916 100644
--- a/docs/extra.css
+++ b/docs/extra.css
@@ -74,10 +74,10 @@ pre .cs, pre .c {
}
*/
-/* Formatting for Note
+/*Formatting for Note
note-sub {
background-color: #e8f7ff;
- color: green;
+ color: 6ab0de;
margin: 10px 0px;
border: #e8f7ff 1px solid;
border-radius: 5px;
diff --git a/docs/howto/connect_using_jdbc_driver.md b/docs/howto/connect_using_jdbc_driver.md
index e3f6dd0057..983754e510 100644
--- a/docs/howto/connect_using_jdbc_driver.md
+++ b/docs/howto/connect_using_jdbc_driver.md
@@ -7,6 +7,24 @@ You can connect to and execute queries against SnappyData cluster using JDBC dri
Where the `` is the hostname of the node on which the locator is started and `` is the port on which the locator accepts client connections (default 1527).
+**Dependencies**: Use the Maven/SBT dependencies for the latest released version of SnappyData.
+
+**Example: Maven dependency:**
+```
+
+
+ io.snappydata
+ snappydata-store-client
+ 1.6.0
+
+```
+
+**Example: SBT dependency:**
+```
+// https://mvnrepository.com/artifact/io.snappydata/snappydata-store-client
+libraryDependencies += "io.snappydata" % "snappydata-store-client" % "1.6.0"
+```
+
**Code Example:**
**Connect to a SnappyData cluster using JDBC on default client port**
diff --git a/docs/howto/load_data_into_snappydata_tables.md b/docs/howto/load_data_into_snappydata_tables.md
index 52c2c9bab9..f9e3f31b29 100644
--- a/docs/howto/load_data_into_snappydata_tables.md
+++ b/docs/howto/load_data_into_snappydata_tables.md
@@ -22,8 +22,10 @@ Next, use it in any SQL query or DDL. For example,
```scala
CREATE EXTERNAL TABLE STAGING_CUSTOMER USING parquet OPTIONS(path 'quickstart/src/main/resources/customerparquet')
+
CREATE TABLE CUSTOMER USING column OPTIONS(buckets '8') AS ( SELECT * FROM STAGING_CUSTOMER)
```
+>>>>>>> origin/master
**Example - Load from CSV**
diff --git a/docs/howto/use_apache_zeppelin_with_snappydata.md b/docs/howto/use_apache_zeppelin_with_snappydata.md
index b01414c075..237720cbd2 100644
--- a/docs/howto/use_apache_zeppelin_with_snappydata.md
+++ b/docs/howto/use_apache_zeppelin_with_snappydata.md
@@ -75,8 +75,8 @@
| Property | Value | Description |
|--------|--------| -------- |
- |default.ur|jdbc:snappydata://localhost:1527/ | Specify the JDBC URL for SnappyData cluster in the format `jdbc:snappydata://:1527` |
- |default.driver|com.pivotal.gemfirexd.jdbc.ClientDriver| Specify the JDBC driver for SnappyData|
+ |default.url|jdbc:snappydata://localhost:1527/ | Specify the JDBC URL for SnappyData cluster in the format `jdbc:snappydata://:1527` |
+ |default.driver|io.snappydata.jdbc.ClientDriver| Specify the JDBC driver for SnappyData|
|snappydata.connection|localhost:1527| Specify the `host:clientPort` combination of the locator for the JDBC connection |
|master|local[*]| Specify the URI of the spark master (only local/split mode) |
|zeppelin.jdbc.concurrent.use|true| Specify the Zeppelin scheduler to be used. Select **True** for Fair and **False** for FIFO |
diff --git a/docs/howto/use_transactions_isolation_levels.md b/docs/howto/use_transactions_isolation_levels.md
index 4c518e1ec9..c8e127a6f8 100644
--- a/docs/howto/use_transactions_isolation_levels.md
+++ b/docs/howto/use_transactions_isolation_levels.md
@@ -2,7 +2,7 @@
SnappyData supports transaction isolation levels when using JDBC or ODBC connections. The default transaction level in SnappyData is set to NONE, which ensures that other transactions are prevented from modifying rows when the current transaction is running an operation on the same rows.
-SnappyData also supports `READ_COMMITTED` and `REPEATABLE_READ` transaction isolation levels. A detailed description of the transaction's semantics in SnappyData can be found in the [Using Transactions](../programming_guide/using_transactions.md) section.
+SnappyData also supports `READ_COMMITTED` and `REPEATABLE_READ` transaction isolation levels. A detailed description of the transaction's semantics in SnappyData can be found in the [Overview of SnappyData Distributed Transactions](../consistency/transactions_about.md) section.
!!! Note:
If you set the isolation level to `READ_COMMITTED` or `REPEATABLE_READ`, queries on column table report an error if [autocommit](../reference/interactive_commands/autocommit.md) is set to **off** (**false**). Queries on column tables are supported when isolation level is set to `READ_COMMITTED` or `REPEATABLE_READ` and autocommit is set to **true**.
@@ -111,3 +111,9 @@ stmt2.execute("SELECT * FROM APP.CUSTOMER")
// the above statement throws an error as given below
EXCEPTION: java.sql.SQLException: (SQLState=XJ218 Severity=20000) (Server=localhost/127.0.0.1[25299] Thread=pool-14-thread-3) Operations on column tables are not supported when query routing is disabled or autocommit is false
```
+
+**More information**
+
+- [Overview of SnappyData Distributed Transactions](../consistency/transactions_about.md)
+
+- [Best Practices for SnappyData Distributed Transactions](../best_practices/transactions_best_practices.md)
diff --git a/docs/install/setting_up_cluster_on_amazon_web_services.md b/docs/install/setting_up_cluster_on_amazon_web_services.md
index e7fed1e33a..4f492bfa37 100644
--- a/docs/install/setting_up_cluster_on_amazon_web_services.md
+++ b/docs/install/setting_up_cluster_on_amazon_web_services.md
@@ -117,7 +117,7 @@ The names and details of the members are automatically derived from the provide
**Example**
```
-./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem --stores=2 --with-zeppelin=embedded --region=us-west-1 launch my-cluster
+./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem --stores=2 --with-zeppelin --region=us-west-1 launch my-cluster
```
In the above example, you are launching a SnappyData cluster named **my-cluster** with 2 stores (or servers). The locator is available in security group named **my-cluster-locator** and the store/server are available in **my-cluster-store**.
@@ -128,8 +128,6 @@ The example assumes that you have the key file (my-ec2-key.pem) in your home dir
!!! Note:
By default, the cluster is launched in the US East (N. Virginia) region on AWS. To launch the cluster in a specific region ensure that you set the region property `--region=`.
-To start Apache Zeppelin on a separate instance, use `--with-zeppelin=non-embedded`.
-
#### Specifying properties
If you want to configure each of the locator, lead, or server with specific properties, you can do so by specifying them in files named **locators**, **leads** or **servers**, respectively and placing these under **aws/ec2/deploy/home/ec2-user/snappydata/**. Refer to [this SnappyData documentation page](../configuring_cluster/configuring_cluster.md#configuration-files) for example on how to configure these configuration files.
@@ -202,10 +200,10 @@ This retains the security groups created for this cluster. To delete this, use i
#### Starting cluster with Apache Zeppelin
-Optionally, you can start an instance of Apache Zeppelin server with the cluster. [Apache Zeppelin](https://zeppelin.apache.org/) is a web-based notebook that enables interactive notebook. You can start it either on a lead node's instance (`--with-zeppelin=embedded`) or on a separate instance (`--with-zeppelin=non-embedded`).
+Optionally, you can start an instance of Apache Zeppelin server with the cluster. [Apache Zeppelin](https://zeppelin.apache.org/) is a web-based notebook that enables interactive notebook. The Zeppelin server is launched on the same EC2 instance where the lead node is running.
````
-./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem --with-zeppelin=embedded launch cluster-name
+./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem --with-zeppelin launch cluster-name
````
### More options
@@ -251,10 +249,10 @@ Options:
-v SNAPPYDATA_VERSION, --snappydata-version=SNAPPYDATA_VERSION
Version of SnappyData to use: 'X.Y.Z' (default:
LATEST)
- --with-zeppelin=WITH_ZEPPELIN
- Launch Apache Zeppelin server with the cluster. Use
- 'embedded' to launch it on lead node and 'non-
- embedded' to launch it on a separate instance.
+ --with-zeppelin
+ Launch Apache Zeppelin server with the cluster. It'll
+ be launched on the same instance where lead node will
+ be running.
--deploy-root-dir=DEPLOY_ROOT_DIR
A directory to copy into/on the first master. Must
be absolute. Note that a trailing slash is handled as
diff --git a/docs/install/system_requirements.md b/docs/install/system_requirements.md
index 0289f492e2..6cb781fb99 100644
--- a/docs/install/system_requirements.md
+++ b/docs/install/system_requirements.md
@@ -18,15 +18,16 @@ SnappyData turns Apache Spark into a mission-critical, elastic scalable in-memor
| Operating System| Version |
|--------|--------|
-|Red Hat Enterprise Linux|RHEL 6.0 and 7.0|
-|Ubuntu|Ubuntu Server 14.04 and later|
-|CentOS|CentOS 6, 7|
+|Red Hat Enterprise Linux|- RHEL 6.0 - RHEL 7.0 (Mininum recommended kernel version: 3.10.0-693.2.2.el7.x86_64)|
+|Ubuntu|Ubuntu Server 14.04 and later||
+|CentOS|CentOS 6, 7 (Minimum recommended kernel version: 3.10.0-693.2.2.el7.x86_64)|
+
## Host Machine Requirements
Requirements for each host:
-* A supported [Java SE installation](http://www.oracle.com/technetwork/java/javase/downloads). SnappyData does not support Java SE 9.
+* A supported [Oracle Java SE 8](http://www.oracle.com/technetwork/java/javase/downloads) installation. We recommend minimum version: 1.8.0_144 (see [SNAP-2017](https://jira.snappydata.io/browse/SNAP-2017), [SNAP-1999](https://jira.snappydata.io/browse/SNAP-1999), [SNAP-1911](https://jira.snappydata.io/browse/SNAP-1911), [SNAP-1375](https://jira.snappydata.io/browse/SNAP-1375) for crashes reported with earlier versions).
* A file system that supports long file names.
diff --git a/docs/isight/quick_start_steps.md b/docs/isight/quick_start_steps.md
index a63f700ea6..594f42d3a8 100644
--- a/docs/isight/quick_start_steps.md
+++ b/docs/isight/quick_start_steps.md
@@ -204,10 +204,8 @@ For example, using the following command, you can start a SnappyData cluster nam
The examples below assume that you have the key file (my-ec2-key.pem) in your home directory for EC2 Key Pair named 'my-ec2-key'.
```
-./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem --stores=2 --with-zeppelin=embedded --region=us-west-1 launch snappydata-cluster
+./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem --stores=2 --with-zeppelin --region=us-west-1 launch snappydata-cluster
```
-To start Apache Zeppelin on a separate instance, use `--with-zeppelin=non-embedded`.
-
For a comprehensive list of command options, simply run `./snappy-ec2` in the command prompt.
diff --git a/docs/monitoring/configure_logging.md b/docs/monitoring/configure_logging.md
index 2fe8e598e7..d8e8968965 100755
--- a/docs/monitoring/configure_logging.md
+++ b/docs/monitoring/configure_logging.md
@@ -111,5 +111,5 @@ If you need to set a trace flag in a running system, use the [SYS.SET_TRACE_FLAG
snappy> call sys.set_trace_flag('traceindex', 'true');
Statement executed.
```
-
-
+!!! Note:
+ Trace flags work only for `snappy` and `jdbc` and not for `snappy-sql`.
\ No newline at end of file
diff --git a/docs/monitoring/monitor-manage.md b/docs/monitoring/monitor-manage.md
new file mode 100755
index 0000000000..53a909a80f
--- /dev/null
+++ b/docs/monitoring/monitor-manage.md
@@ -0,0 +1,213 @@
+# Getting Information from SnappyData System Tables
+
+You can monitor many common aspects of SnappyData by using SQL commands (system procedures and simple queries) to collect and analyze data in SnappyData system tables.
+
+## Distributed System Membership Information
+
+The SYS.MEMBERS table provides information about all peers and servers that make up the SnappyData system. You can use different queries to obtain details about individual members and their role in the cluster.
+
+
+### Determining Cluster Membership
+
+To display a list of all members that participate in a given cluster, simply query all ID entries in sys.members. For example:
+
+``` pre
+snappy> select ID from SYS.MEMBERS;
+ID
+------------------------------------------------------------------------------
+localhost(10898):55213
+localhost(11131):47059
+localhost(10739):65055
+
+3 rows selected
+```
+
+The number of rows returned corresponds to the total number of peers, servers, and locators in the cluster.
+
+To determine each member's role in the system, include the KIND column in the query:
+
+``` pre
+snappy> select ID, KIND from SYS.MEMBERS;
+ID |KIND
+------------------------------------------------------------------------------
+localhost(10898):55213 |datastore(normal)
+localhost(11131):47059 | accessor(normal)
+localhost(10739):65055 |locator(normal)
+
+3 rows selected
+```
+
+To view the members of cluster, query:
+
+``` pre
+snappy> show members;
+ID |HOST |KIND |STATUS |THRIFTSERVERS |SERVERGROUPS
+------------------------------------------------------------------------------------------------------------
+localhost(10739):65055 |localhost |locator(normal) |RUNNING|localhost/127.0.0.1[1527]|
+localhost(10898):55213 |localhost |datastore(normal) |RUNNING|localhost/127.0.0.1[1528]|
+localhost(11131):47059 |localhost |accessor(normal) |RUNNING|IMPLICIT_LEADER_SERVERGROUP|
+
+3 rows selected
+```
+Data store members host data in the cluster, while accessor members do not host data. This role is determined by the `host-data` boot property. If a cluster contains only a single data store, its KIND is listed as "datastore(loner)."
+
+## Table and Data Storage Information
+
+The SYS.SYSTABLES table provides information about all tables that are created in the SnappyData system. You can use different queries to obtain details about tables and the server groups that host data for those tables.
+
+- [Displaying a List of Tables](#display-list-of-tables)
+- [Determining Whether a Table Is Replicated or Partitioned](#determine-replica-partition)
+- [Determining How Persistent Data Is Stored](#determine-peristent-data)
+- [Displaying Eviction Settings](#display-eviction-setting)
+- [Displaying Indexes](#display-indexes)
+
+
+### Displaying a List of Tables
+
+To display a list of all tables in the cluster:
+
+``` bash
+select TABLESCHEMANAME, TABLENAME from SYS.SYSTABLES order by TABLESCHEMANAME;
+TABLESCHEMANAME |TABLENAME
+-----------------------------------------------
+APP |PIZZA_ORDER_PIZZAS
+APP |PIZZA
+APP |TOPPING
+APP |PIZZA_ORDER
+APP |HIBERNATE_SEQUENCES
+APP |HOTEL
+SNAPPY_HIVE_METASTORE |TBLS
+SNAPPY_HIVE_METASTORE |PARTITION_PARAMS
+SNAPPY_HIVE_METASTORE |SKEWED_STRING_LIST_VALUES
+SNAPPY_HIVE_METASTORE |FUNCS
+SNAPPY_HIVE_METASTORE |SDS
+SNAPPY_HIVE_METASTORE |SD_PARAMS
+SNAPPY_HIVE_METASTORE |PART_COL_STATS
+SNAPPY_HIVE_METASTORE |SKEWED_STRING_LIST
+SNAPPY_HIVE_METASTORE |DBS
+SNAPPY_HIVE_METASTORE |PARTITIONS
+SNAPPY_HIVE_METASTORE |SKEWED_COL_VALUE_LOC_MAP
+SNAPPY_HIVE_METASTORE |FUNC_RU
+SNAPPY_HIVE_METASTORE |SORT_COLS
+SNAPPY_HIVE_METASTORE |ROLES
+SNAPPY_HIVE_METASTORE |BUCKETING_COLS
+SNAPPY_HIVE_METASTORE |SKEWED_COL_NAMES
+SNAPPY_HIVE_METASTORE |TAB_COL_STATS
+SNAPPY_HIVE_METASTORE |GLOBAL_PRIVS
+SNAPPY_HIVE_METASTORE |SKEWED_VALUES
+SNAPPY_HIVE_METASTORE |TABLE_PARAMS
+SNAPPY_HIVE_METASTORE |VERSION
+SNAPPY_HIVE_METASTORE |CDS
+SNAPPY_HIVE_METASTORE |SEQUENCE_TABLE
+SNAPPY_HIVE_METASTORE |PARTITION_KEYS
+SNAPPY_HIVE_METASTORE |COLUMNS_V2
+SNAPPY_HIVE_METASTORE |DATABASE_PARAMS
+SNAPPY_HIVE_METASTORE |SERDE_PARAMS
+SNAPPY_HIVE_METASTORE |SERDES
+SNAPPY_HIVE_METASTORE |PARTITION_KEY_VALS
+SYS |GATEWAYSENDERS
+SYS |SYSSTATEMENTS
+SYS |SYSKEYS
+SYS |SYSROLES
+SYS |SYSFILES
+SYS |SYSROUTINEPERMS
+SYS |SYSCONSTRAINTS
+SYS |SYSCOLPERMS
+SYS |SYSHDFSSTORES
+SYS |SYSDEPENDS
+SYS |SYSALIASES
+SYS |SYSTABLEPERMS
+SYS |SYSTABLES
+SYS |SYSVIEWS
+SYS |ASYNCEVENTLISTENERS
+SYS |SYSCHECKS
+SYS |SYSSTATISTICS
+SYS |SYSCONGLOMERATES
+SYS |GATEWAYRECEIVERS
+SYS |SYSDISKSTORES
+SYS |SYSTRIGGERS
+SYS |SYSSCHEMAS
+SYS |SYSFOREIGNKEYS
+SYS |SYSCOLUMNS
+SYSIBM |SYSDUMMY1
+SYSSTAT |SYSXPLAIN_RESULTSETS
+SYSSTAT |SYSXPLAIN_STATEMENTS
+
+62 rows selected
+
+```
+
+
+
+### Determining Whether a Table Is Replicated or Partitioned
+
+The DATAPOLICY column specifies whether a table is replicated or partitioned, and whether a table is persisted to a disk store. For example:
+
+``` pre
+select TABLENAME, DATAPOLICY from SYS.SYSTABLES where TABLESCHEMANAME = 'APP';
+TABLENAME |DATAPOLICY
+--------------------------------------------------
+PIZZA |PERSISTENT_REPLICATE
+PIZZA_ORDER_PIZZAS |PARTITION
+BASE |REPLICATE
+TOPPING |REPLICATE
+PIZZA_TOPPINGS |PARTITION
+PIZZA_ORDER |PERSISTENT_PARTITION
+
+6 rows selected
+```
+
+
+### Determining How Persistent Data Is Stored
+
+For persistent tables, you can also display the disk store that persists the table's data, and whether the table uses synchronous or asynchronous persistence:
+
+``` pre
+select TABLENAME, DISKATTRS from SYS.SYSTABLES where TABLESCHEMANAME = 'APP';
+TABLENAME |DISKATTRS
+----------------------------------------------
+PIZZA |DiskStore is GFXD-DEFAULT-DISKSTORE; Synchronous writes to disk
+PIZZA_ORDER_PIZZAS |DiskStore is OVERFLOWDISKSTORE;Asynchronous writes to disk
+BASE |NULL
+TOPPING |NULL
+PIZZA_TOPPINGS |DiskStore is GFXD-DEFAULT-DISKSTORE; Synchronous writes to disk
+PIZZA_ORDER |DiskStore is GFXD-DEFAULT-DISKSTORE; Synchronous writes to disk
+
+6 rows selected
+```
+
+
+### Displaying Eviction Settings
+
+Use the EVICTIONATTRS column to determine if a table uses eviction settings and whether a table is configured to overflow to disk. For example:
+
+``` pre
+select TABLENAME, EVICTIONATTRS from SYS.SYSTABLES where TABLESCHEMANAME = 'APP';
+TABLENAME |EVICTIONATTRS
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+PIZZA |NULL
+PIZZA_ORDER_PIZZAS | algorithm=lru-entry-count; action=overflow-to-disk; maximum=100
+BASE |NULL
+TOPPING |NULL
+PIZZA_TOPPINGS | algorithm=lru-entry-count; action=overflow-to-disk; maximum=100
+PIZZA_ORDER |NULL
+
+6 rows selected
+```
+
+
+### Displaying Indexes
+
+Join SYSTABLES with CONGLOMERATENAME to determine if a table has an index and display the indexed columns. For example:
+
+``` pre
+select CONGLOMERATENAME from SYS.SYSCONGLOMERATES c, SYS.SYSTABLES t
+ where c.ISINDEX = 1 and c.TABLEID = t.TABLEID and t.TABLESCHEMANAME = 'APP'
+ and t.TABLENAME = 'PIZZA';
+CONGLOMERATENAME
+----------------
+2__PIZZA__ID
+6__PIZZA__BASE
+
+2 rows selected
+```
diff --git a/docs/monitoring/monitoring.md b/docs/monitoring/monitoring.md
index e0127682aa..65033001c4 100644
--- a/docs/monitoring/monitoring.md
+++ b/docs/monitoring/monitoring.md
@@ -9,6 +9,9 @@ To access the SnappyData Pulse, start your cluster and open http:``:50
The top-right side of the page displays the date and time when the Dashboard was last updated. Click on the product version number to view details like the build number, source revision, underlying spark version, etc.
+!!! Note:
+ - When using Smart Connector with upstream Spark, the **Dashboard** and **Member Details** sections are not displayed. Only the **SQL**, **Jobs** and **Stages** related information is displayed.
+
The following topics are covered in this section:
* [Dashboard](#dashboard)
@@ -193,7 +196,10 @@ The top-right side of the page displays the last updated date and time for the m
* **Status**: Displays the status of the job.
-* **Stages**: Click on the Stage to view its details. The table displays the time taken for completion of each stage.
+* **Stages**: Click on the stage to view its details. The table displays the time taken for completion of each stage.
+
+!!! Tip:
+ You can cancel a long running job, using the **Kill** option. ![kill](../Images/kill_job.png)
## Stages Page
diff --git a/docs/old_files/snappyOnAWS.md b/docs/old_files/snappyOnAWS.md
index 0bc249f857..4e5afa57cd 100644
--- a/docs/old_files/snappyOnAWS.md
+++ b/docs/old_files/snappyOnAWS.md
@@ -105,9 +105,9 @@ This will retain the security groups created for this cluster. To delete them as
**Starting cluster with Apache Zeppelin**
-Optionally, you can start an instance of Apache Zeppelin server with the cluster. [Apache Zeppelin](https://zeppelin.apache.org/) is a web-based notebook that enables interactive notebook. You can start it in two modes: embedded and non-embedded. In the former mode, the Apache Zeppelin server would be launched on lead instance while in the latter, it would be launched on a separate EC2 instance.
+Optionally, you can start an instance of Apache Zeppelin server with the cluster. [Apache Zeppelin](https://zeppelin.apache.org/) is a web-based notebook that enables interactive notebook. The Zeppelin server is launched on the same EC2 instance where the lead node is running.
````
-./snappy-ec2 -k ec2-keypair-name -i /path/to/keypair/private/key/file --with-zeppelin=embedded launch cluster-name
+./snappy-ec2 -k ec2-keypair-name -i /path/to/keypair/private/key/file --with-zeppelin launch cluster-name
````
**More options**
@@ -152,10 +152,10 @@ Options:
-v SNAPPYDATA_VERSION, --snappydata-version=SNAPPYDATA_VERSION
Version of SnappyData to use: 'X.Y.Z' (default:
LATEST)
- --with-zeppelin=WITH_ZEPPELIN
- Launch Apache Zeppelin server with the cluster. Use
- 'embedded' to launch it on lead node and 'non-
- embedded' to launch it on a separate instance.
+ --with-zeppelin
+ Launch Apache Zeppelin server with the cluster. It'll
+ be launched on the same instance where lead node will
+ be running.
--deploy-root-dir=DEPLOY_ROOT_DIR
A directory to copy into / on the first master. Must
be absolute. Note that a trailing slash is handled as
diff --git a/docs/programming_guide.md b/docs/programming_guide.md
index be2b1a9126..e232ea6279 100644
--- a/docs/programming_guide.md
+++ b/docs/programming_guide.md
@@ -21,8 +21,6 @@ The following topics are covered in this section:
* [Working with Hadoop YARN Cluster Manager](programming_guide/working_with_hadoop_yarn_cluster_manager.md)
-* [Using Transactions](programming_guide/using_transactions.md)
-
* [Using JDBC with SnappyData](programming_guide/using_jdbc_with_snappydata.md)
* [Multiple Language Binding using Thrift Protocol](programming_guide/multiple_language_binding_using_thrift_protocol.md)
diff --git a/docs/programming_guide/snappydata_jobs.md b/docs/programming_guide/snappydata_jobs.md
index 016a6994eb..8f458d0464 100644
--- a/docs/programming_guide/snappydata_jobs.md
+++ b/docs/programming_guide/snappydata_jobs.md
@@ -182,6 +182,7 @@ $ bin/snappy-job.sh submit \
--lead localhost:8090 \
--app-name airlineApp \
--class io.snappydata.examples.TwitterPopularTagsJob \
+ --conf streaming.batch_interval=5000 \
--app-jar $SNAPPY_HOME/examples/jars/quickstart.jar \
--stream
diff --git a/docs/programming_guide/tables_in_snappydata.md b/docs/programming_guide/tables_in_snappydata.md
index 544e893a99..430f663bae 100644
--- a/docs/programming_guide/tables_in_snappydata.md
+++ b/docs/programming_guide/tables_in_snappydata.md
@@ -186,6 +186,7 @@ Usage SnappySession.update(): Update all rows in table that match passed filter
snappy.update(tableName, "ITEMREF = 3" , Row(99) , "ITEMREF" )
```
+
**Usage SnappySession.delete()**: Delete all rows in table that match passed filter expression
```scala
@@ -219,15 +220,12 @@ CREATE TABLE tableName (Col1 char(25), Col2 varchar(100)) using row;
// create the table
snappy.createTable(tableName, "row", schema, Map.empty[String, String])
```
-
-
!!! Note:
STRING columns are handled differently when queried over a JDBC connection.
To ensure optimal performance for SELECT queries executed over JDBC connection (more specifically, those that get routed to lead node), the data of STRING columns is returned in VARCHAR format, by default. This also helps the data visualization tools to render the data effectively.
However, if the STRING column size is larger than VARCHAR limit (32768), you can enforce the returned data format to be in CLOB in following ways:
-
Using the system property `spark-string-as-clob` when starting the lead node(s). This applies to all the STRING columns in all the tables in cluster.
```
diff --git a/docs/programming_guide/using_transactions.md b/docs/programming_guide/using_transactions.md
deleted file mode 100644
index 163ae30336..0000000000
--- a/docs/programming_guide/using_transactions.md
+++ /dev/null
@@ -1,41 +0,0 @@
-# Using Transactions
-
-## Using Transaction Isolation levels
-Transactions specify an [isolation level](../reference/sql_reference/set-isolation.md) that defines the degree to which one transaction must be isolated from resource or data modifications made by other transactions. The transaction isolation levels define the type of locks acquired on read operations. Only one of the isolation level options can be set at a time, and it remains set for that connection until it is explicitly changed.
-
-!!! Note:
-
- * If you set the isolation level to `READ_COMMITTED` or `REPEATABLE_READ`, queries on column table report an error if [autocommit](../reference/interactive_commands/autocommit.md) is set to off (false). Queries on column tables are supported when isolation level is set to `READ_COMMITTED` or `REPEATABLE_READ` and autocommit is set to **true**.
-
- * DDL execution (for example [CREATE TABLE](../reference/sql_reference/create-table.md) /[DROP TABLE](../reference/sql_reference/drop-table.md)) is not allowed when `autocommit` is set to `false` and transaction isolation level is `READ_COMMITTED` or `REPEATABLE_READ`. DDL commands reports syntax error in such cases. DDL execution is allowed if `autocommit` is `true` for `READ_COMMITTED` or `REPEATABLE_READ` isolation levels.
-
-
-The following isolation levels are supported for row tables:
-
-| Isolation level | Description |
-|--------|--------|
-|NONE|Default isolation level. The Database Engine uses shared locks to prevent other transactions from modifying rows while the current transaction is running a read operation. |
-|READ_COMMITTED|SnappyData ensures that ongoing transactional as well as non-transactional (isolation-level NONE) operations never read uncommitted (dirty) data. SnappyData accomplishes this by maintaining transactional changes in a separate transaction state that are applied to the actual data-store for the table only at commit time. SnappyData detects only Write-Write conflicts while in READ_COMMITTED isolation level. In READ COMMITTED, a read view is created at the start of each statement and lasts only as long as each statement execution.|
-|REPEATABLE_READ|In this isolation level, a lock-based concurrency control DBMS implementation keeps read and write locks (acquired on selected data) until the end of the transaction. In REPEATABLE READ every lock acquired during a transaction is held for the duration of the transaction.|
-
-For more information, see, [SET ISOLATION](../reference/sql_reference/set-isolation.md)
-
-## Using Snapshot Isolation for Column Tables
-
-Multi-Statement transactions are not supported on column tables. Instead, we provide snapshot isolation by default. Snapshot ensures that all queries see the same version (snapshot), of the database, based on the state of the database at the moment in time when the query is executed. The snapshot is taken per statement for each partition, which means, the snapshot of the partition is taken the moment the query accesses the partition. This behavior is set by default for column tables and cannot be modified.
-
-By default, all individual operations (read/write) on column table have snapshot isolation with `autocommit` set to `ON`. This means, in case of a failure the user operation fails and [rollback](../reference/interactive_commands/rollback.md) is triggered.
-You cannot set [autocommit](../reference/interactive_commands/autocommit.md) to `Off`. Snapshot isolation ensures that changes made, after the ongoing operation has taken a snapshot is not visible partially or totally.
-If there are concurrent updates in a row, then the last commit is used.
-
-!!! Note:
- To get per statement transactional behavior, all the write operations should span only one partition.
-
- However, if you have operations that span multiple partitions, then, ensure that:
-
- * In case of failure on one partition, the operation is retried on another copy of the same partition. Set [redundancy](../reference/sql_reference/create-table.md) to more than 0, if transactional behavior with operations spanning more than one partition is required.
-
- * If the operation fails on all the redundant copies of a partition and the same operation succeeds on some of the other partitions, then, partial rollback is initiated.
- In this case, you can retry the operation at the application level.
-
-
diff --git a/docs/reference/configuration_parameters/config_parameters.md b/docs/reference/configuration_parameters/config_parameters.md
index 47d0ca1d8e..8785d89ef0 100644
--- a/docs/reference/configuration_parameters/config_parameters.md
+++ b/docs/reference/configuration_parameters/config_parameters.md
@@ -81,10 +81,14 @@ Below is the list of all the configuration properties and links for each propert
- [enable-time-statistics](enable-time-statistics.md)
+- [enable-timestats](enable-timestats.md)
+
- [enforce-unique-host](enforce-unique-host.md)
- [init-scripts](init-scripts.md)
+- [load-balance](load-balance.md)
+
- [locators](locators.md)
- [log-file](log-file.md)
@@ -93,8 +97,20 @@ Below is the list of all the configuration properties and links for each propert
- [member-timeout](member-timeout.md)
+- [membership-port-range](membership-port-range.md)
+
+- [password](password.md)
+
+- [read-timeout](read-timeout.md)
+
- [redundancy-zone](redundancy-zone.md)
+- [secondary-locators](secondary-locators.md)
+
+- [skip-constraint-checks](skip-constraint-checks.md)
+
+- [ssl](ssl.md)
+
- [start-locator](start-locator.md)
- [statistic-archive-file](statistic-archive-file.md)
@@ -103,6 +119,8 @@ Below is the list of all the configuration properties and links for each propert
- [statistic-sampling-enabled](statistic-sampling-enabled.md)
+- [sync-commits](sync-commits.md)
+
- [sys-disk-dir](sys-disk-dir.md)
- [user](user.md)
diff --git a/docs/reference/configuration_parameters/enable-timestats.md b/docs/reference/configuration_parameters/enable-timestats.md
new file mode 100644
index 0000000000..3d4d1e2cdf
--- /dev/null
+++ b/docs/reference/configuration_parameters/enable-timestats.md
@@ -0,0 +1,17 @@
+# enable-timestats
+
+## Description
+
+Boolean instructing the system to track time-based statistics for the current connection. Disabled by default for performance. See [Evaluating Statistics for the System and Applications](../../vsd/system_performance.md).
+
+## Default Value
+
+false
+
+## Property Type
+
+connection
+
+## Prefix
+
+gemfire.
diff --git a/docs/reference/configuration_parameters/hostname-for-clients.md b/docs/reference/configuration_parameters/hostname-for-clients.md
new file mode 100644
index 0000000000..8d11a66b03
--- /dev/null
+++ b/docs/reference/configuration_parameters/hostname-for-clients.md
@@ -0,0 +1,4 @@
+# hostname-for-clients
+Hostname given to clients that ask the locator for the location of a JMX Manager. By default the IP address of the JMX Manager is used. However, for clients on a different network, you can configure a different hostname to be given to clients. Ignored if jmx-manager is false or if jmx-manager-port is zero.
+
+##
\ No newline at end of file
diff --git a/docs/reference/configuration_parameters/load-balance.md b/docs/reference/configuration_parameters/load-balance.md
new file mode 100644
index 0000000000..65b5f51a1d
--- /dev/null
+++ b/docs/reference/configuration_parameters/load-balance.md
@@ -0,0 +1,19 @@
+# load-balance
+
+## Description
+Specifies whether load balancing is performed for the JDBC client connection. Note that load balancing is provided only for SnappyData distributed systems that use locators for member discovery. With the default value ("true") clients are automatically connected to a less-loaded server if locators are used for member discovery. Note that when load balancing is enabled, clients may not be able to connect to a specific server even if they provide that server's unique port number for client connections. As a best practice, clients should always request connections using a locator address and port when load balancing is enabled.
+If a JDBC client needs to connect to a specific member, set load-balance to "false" in the connection string and specify the connection details for a specific SnappyData member, rather than a locator. For example:
+```
+snappy> connect client 'server_hostname:server_port/;load-balance=false'
+```
+## Default Value
+
+true
+
+## Property Type
+
+connection
+
+## Prefix
+
+n/a
diff --git a/docs/reference/configuration_parameters/password.md b/docs/reference/configuration_parameters/password.md
new file mode 100644
index 0000000000..d28539ba81
--- /dev/null
+++ b/docs/reference/configuration_parameters/password.md
@@ -0,0 +1,17 @@
+# password
+
+## Description
+
+A password for the user name given at boot or connection time.
+Use this attribute in conjunction with the user attribute.
+
+## Default Value
+not set
+
+## Property Type
+
+connection
+
+## Prefix
+
+n/a
diff --git a/docs/reference/configuration_parameters/ssl.md b/docs/reference/configuration_parameters/ssl.md
new file mode 100644
index 0000000000..db763bb036
--- /dev/null
+++ b/docs/reference/configuration_parameters/ssl.md
@@ -0,0 +1,17 @@
+# ssl
+
+## Description
+
+Determines whether this connection is encrypted or not, and whether certificate-based peer authentication is enabled. Possible values are Off, Basic, and peerAuthentication.
+
+## Default Value
+
+Off
+
+## Property Type
+
+Connection
+
+## Prefix
+
+
diff --git a/docs/reference/configuration_parameters/sync-commits.md b/docs/reference/configuration_parameters/sync-commits.md
new file mode 100644
index 0000000000..2b263061ae
--- /dev/null
+++ b/docs/reference/configuration_parameters/sync-commits.md
@@ -0,0 +1,14 @@
+# sync-commits
+
+## Description
+Determines whether second-phase commit actions occur in the background for the current connection, or whether the connection waits for second-phase commit actions to complete. By default (sync-commits=false) SnappyData performs second-phase commits in the background, but ensures that the connection that issued the transaction only sees completed results. This means that other threads or connections may see different results until the second-phase commit actions complete.
+Using `sync-commits=true` ensures that the current thin client or peer client connection waits until all second-phase commit actions complete.
+
+## Default Value
+false
+
+## Property Type
+connection (boot)
+
+## Prefix
+n/a
\ No newline at end of file
diff --git a/docs/reference/interactive_commands/store_command_reference.md b/docs/reference/interactive_commands/store_command_reference.md
index 6db3cad660..93d3d94577 100755
--- a/docs/reference/interactive_commands/store_command_reference.md
+++ b/docs/reference/interactive_commands/store_command_reference.md
@@ -25,6 +25,12 @@ Commands can span multiple lines without using any special escape character for
`snappy` treats any command that it does not recognize as a SQL command that is passed to the underlying connection. This means that any syntactic errors in `snappy` commands are handed to the SQL engine and generally result in SQL parsing errors.
+- **[autocommit](../../reference/interactive_commands/autocommit.md)**
+ Turns the connection's auto-commit mode on or off.
+
+- **[commit](../../reference/interactive_commands/commit.md)**
+ Issues a `java.sql.Connection.commit` request.
+
- **[connect client](../../reference/interactive_commands/connect_client.md)**
Using the JDBC SnappyData thin client driver, connects to a SnappyData member indicated by the *host:port* values.
@@ -46,6 +52,9 @@ Commands can span multiple lines without using any special escape character for
- **[MaximumDisplayWidth](../../reference/interactive_commands/maximumdisplaywidth.md)**
Sets the largest display width for columns to the specified value.
+- **[rollback](../../reference/interactive_commands/rollback.md)**
+ Issues a `java.sql.Connection.rollback` request.
+
- **[run](../../reference/interactive_commands/run.md)**
Treats the value of the string as a valid file name, and redirects `snappy` processing to read from that file until it ends or an exit command is executed.
diff --git a/docs/reference/misc/del_supported_datatypes.md b/docs/reference/misc/del_supported_datatypes.md
new file mode 100644
index 0000000000..dd9bcfb786
--- /dev/null
+++ b/docs/reference/misc/del_supported_datatypes.md
@@ -0,0 +1,235 @@
+# Data Types
+The SQL type system determines the compile-time and runtime type of an expression. Each type has a certain range of permissible values that can be assigned to a column or value of that type.
+
+The special value NULL, denotes an unassigned or missing value of any of the types (columns that have been assigned as non-nullable using NOT NULL clause or the primary key columns cannot have a NULL value). The supported types are given below:
+
+**Data Types Supported for Column and Row Tables**
+
+| Data Type | Description |
+|--------|--------|
+| [BIGINT](#bigint)|Provides 8-byte integer for long integer values|
+| [BINARY](#binary)|Binary-encoded strings|
+| [BLOB](#blob)|Carying-length binary string that can be up to 2,147,483,647 characters long|
+| [BOOLEAN](#boolean)|Logical Boolean values (true/false)|
+| [BYTE](#byte)|Binary data ("byte array")|
+| [CLOB](#clob)|Text data in random-access chunks|
+| [DATE](#date)|Calendar dates (year, month, day)|
+| [DECIMAL](#decimal)|DECIMAL(p) floating point and DECIMAL(p,s) fixed point|
+| [DOUBLE](#double)|A double-precision floating point value|
+| [FLOAT](#float)|Stores floating point value: that is a number with a fractional part|
+| [INT](#int)|Stores integer: a whole number|
+| [INTEGER](#integer)|Stores signed four-byte integer|
+| [LONG](#long)|Stores character strings with a maximum length of 32,700 characters|
+| [NUMERIC](#numeric)|Stores exact numeric of selectable precision|
+| [REAL](#real)|Stores single precision floating-point number (4 bytes)|
+| [SHORT](#short)|The size of the short type is 2 bytes (16 bits) |
+| [SMALLINT](#smallint)|Stores signed two-byte integer|
+| [STRING](#string)|Stores are sequences of characters|
+| [TIMESTAMP](#timestamp)|Stores date and time as a combined value|
+| [TINYINT](#tinyint)|Stores a very small integer. The signed range is -128 to 127.|
+| [VARBINARY](#varbinary)|Stores binary byte strings rather than non-binary character strings|
+| [VARCHAR](#varchar)|Stores character strings of varying length (up to 255 bytes); collation is in code-set order|
+
+
+## BIGINT
+
+Provides 8 bytes storage for long integer values. An attempt to put a BIGINT value into another exact numeric type with smaller size/precision (e.g. INT) fails if the value overflows the maximum allowable by the smaller type.
+
+For behavior with other types in expressions, see [Numeric type](#numeric) promotion in expressions, Storing values of one numeric data type in columns of another numeric data type.
+
+**Field Details**
+`public static final DataType BIGINT`
+
+
+## BINARY
+**Description**
+The BINARY type is similar to the CHAR type, but stores binary byte strings rather than non-binary character strings.
+It contains no character set, and comparison and sorting are based on the numeric value of the bytes.
+
+**Field Details**
+`public static final DataType BinaryType`
+
+
+## BLOB
+**Description**
+A binary large object represents an array of raw bytes of varying length.
+
+**Field Details**
+`public static final DataType BLOB`
+
+
+## BOOLEAN
+**Description**
+Booleans are used to represent true and false values returned by comparison operators and logical functions. The values are, true and false.
+The display values can be localized.
+
+**Field Details**
+`public static final DataType BOOLEAN`
+
+
+## BYTE
+**Description**
+The BYTE data type stores any kind of binary data in an undifferentiated byte stream. Binary data typically consists of digitized information, such as spreadsheets, program load modules, digitized voice patterns, and so on.
+The term simple large object refers to an instance of a TEXT or BYTE data type. No more than 195 columns of the same table can be declared as BYTE and TEXT data types.
+
+**Field Details**
+`public static final DataType ByteType`
+
+
+## CLOB
+**Description**
+A character large object represents an array of characters of varying length. It is used to store large character-based data such as documents.
+The length is expressed in number characters, unless you specify the suffix K, M, or G, which uses the multiples of 1024, 1024*1024, or 1024*1024*1024 respectively.
+
+**Field Details**
+`public static final DataType CLOB`
+
+
+## DATE
+**Description**
+Provides for storage of a date as year-month-day. Supported formats are: `yyyy-mm-dd`, `mm/dd/yyyy`, and `dd.mm.yyyy`
+The year (yyyy) must always be expressed with four digits, while months (mm) and days (dd) may have either one or two digits. DATEs, TIMEs, and TIMESTAMPs must not be mixed with one another in expressions except with an explicit CAST.
+
+**Field Details**
+`public static final DataType DATE`
+Gets the DateType object.
+
+
+## DECIMAL
+**Description**
+Provides an exact decimal value having a specified precision and scale. The precision is the total number of digits both to the left and the right of the decimal point, and the scale is the number of digits in the fraction to the right of the decimal point.
+A numeric value (e.g. INT, BIGINT, SMALLINT) can be put into a DECIMAL as long as non-fractional precision is not lost else a range exception is thrown (SQLState: "22003"). When truncating trailing digits from a DECIMAL, the value is rounded down.
+For behavior with other types in expressions, see Numeric type promotion in expressions, Scale for decimal arithmetic and Storing values of one numeric data type in columns of another numeric data type.
+
+**Field Details**
+`public static final DataType DECIMAL`
+
+
+## DOUBLE
+**Description**
+Provides 8-byte storage for numbers using IEEE floating-point notation.
+Arithmetic operations do not round their resulting values to zero. If the values are too small, you will receive an exception. Numeric floating point constants are limited to a length of 30 characters.
+For behavior with other types in expressions, see Numeric type promotion in expressions, and Storing values of one numeric data type in columns of another numeric data type.
+
+**Field Details**
+`public static final DataType DOUBLE`
+Gets the DoubleType object.
+
+
+## FLOAT
+**Description**
+Alias for a REAL or DOUBLE data type, depending on the specified precision. The default precision is 53 making it equivalent to DOUBLE. A precision of 23 or less makes FLOAT equivalent to REAL while greater than 23 makes it equivalent to DOUBLE.
+
+**Field Details**
+`public static final DataType FLOAT`
+
+
+## INT
+**Description**
+The INT data type is a synonym for INTEGER.
+**Field Details**
+`public static final DataType INT`
+
+
+## INTEGER
+**Description**
+Integer values are written as a sequence of digits. It provides 4 bytes storage for integer values. INT can be used as a synonym for INTEGER in CREATE TABLE.
+For behavior with other types in expressions, see Numeric type promotion in expressions, and Storing values of one numeric data type in columns of another numeric data type.
+
+**Field Details**
+`public static final DataType IntegerType`
+
+
+## LONG
+**Description**
+ The long data type is a 64-bit two's complement integer. The signed long has a minimum value of -263 and a maximum value of 263-1. Use this data type when you need a range of values wider than those provided by int.
+**Field Details**
+`public static final DataType LongType`
+
+
+## NUMERIC
+**Description**
+Synonym for DECIMAL data type.
+The meta-data differences from DECIMAL are listed below. Otherwise, NUMERIC behaves identically to DECIMAL.
+
+**Field Details**
+`public static final DataType NUMERIC`
+
+
+## REAL
+**Description**
+Provides 4-byte storage for numbers using IEEE floating-point notation.
+Arithmetic operations do not round their resulting values to zero. If the values are too small, you will receive an exception. Constants always map to DOUBLE – use an explicit CAST to convert a constant to REAL.
+For behavior with other types in expressions, see Numeric type promotion in expressions, Storing values of one numeric data type in columns of another numeric data type.
+
+**Field Details**
+`public static final DataType REAL`
+
+
+## SHORT
+**Description**
+The short data type is a 16-bit signed two's complement integer. It has a minimum value of -32,768 and a maximum value of 32,767 (inclusive). As with byte, the same guidelines apply: you can use a short to save memory in large arrays, in situations where the memory savings actually matters.
+
+**Field Details**
+`public static final DataType ShortType`
+
+
+## SMALLINT
+**Description**
+Provides 2 bytes storage for short integer values.
+For behavior with other types in expressions, see Numeric type promotion in expressions, and Storing values of one numeric data type in columns of another numeric data type.
+
+**Field Details**
+`public static final DataType SMALLINT`
+
+
+## STRING
+The string type is used for storing text strings. A text string is a sequence of characters in the Unicode format with the final zero at the end of it. A string constant can be assigned to a string variable. A string constant is a sequence of Unicode characters enclosed in quotes or double quotes: "This is a string constant".
+To include a double quote (") into a string, the backslash character (\) must be put before it. The \ (backslash character) is used to escape characters. Any special character constants can be written in a string, if the backslash character (\) is typed before them.
+
+**Field Details**
+`public static final DataType StringType`
+Gets the StringType object.
+
+
+## TIMESTAMP
+**Description**
+Provides for storage of both DATE and TIME as a combined value. In addition it allows for fractional seconds having up to six digits. Supported formats are:
+
+- yyyy-MM-dd hh:mm:ss[.nnnnnn]
+- yyyy-MM-dd-hh.mm.ss[.nnnnnn]
+
+The year (yyyy) must always be expressed with four digits. Months (MM), days (dd), and hours (hh) may have one or two digits while minutes (mm) and seconds (ss) must have two digits. Microseconds, if present, may have between one and six digits. DATEs, TIMEs, and TIMESTAMPs must not be mixed with one another in expressions except with an explicit CAST.
+
+**Field Details**
+`public static final DataType TimestampType`
+
+
+## TINYINT
+**Description**
+A very small integer. The signed range is -128 to 127. The unsigned range is 0 to 255.
+
+**Field Details**
+`public static final TINYINT`
+
+
+## VARBINARY
+**Description**
+The VARBINARY type is similar to the VARCHAR type, but stores binary byte strings rather than non-binary character strings.
+It contains no character set, and comparison and sorting are based on the numeric value of the bytes.
+
+**Field Details**
+`public static final DataType VARBINARY`
+
+
+## VARCHAR
+**Description**
+Provides for variable-length strings with a maximum limit for length. If a string value is longer than the maximum length, then any trailing blanks are trimmed to make the length same as the maximum length, while an exception is raised if characters other than spaces are required to be truncated. When mixing CHARs and VARCHARs in expressions, the shorter value is padded with spaces to the length of longer string.
+The type of a string constant is CHAR, not VARCHAR. To represent a single quotation mark within a string, use two quotation marks:
+
+`VALUES 'visiting John's place' `
+
+The length of VARCHAR is an unsigned integer constant.
+
+**Field Details**
+`public static final DataType VARCHAR`
\ No newline at end of file
diff --git a/docs/reference/misc/supported_datatypes.md b/docs/reference/misc/supported_datatypes.md
old mode 100644
new mode 100755
index 6df91e9021..8f846f2f91
--- a/docs/reference/misc/supported_datatypes.md
+++ b/docs/reference/misc/supported_datatypes.md
@@ -1,61 +1,401 @@
# Data Types
+
The SQL type system determines the compile-time and runtime type of an expression. Each type has a certain range of permissible values that can be assigned to a column or value of that type.
-The special value NULL, denotes an unassigned or missing value of any of the types (columns that have been assigned as non-nullable using NOT NULL clause or the primary key columns cannot have a NULL value). The supported types are given below:
-
-## Data Types Supported for Column Tables
-
-| column-data-type |
-|--------|
-|STRING |
-|INTEGER |
-|INT |
-|BIGINT |
-|LONG |
-|DOUBLE |
-|DECIMAL |
-|NUMERIC |
-|DATE |
-|TIMESTAMP |
-|FLOAT |
-|REAL |
-|BOOLEAN |
-|CLOB |
-|BLOB |
-|BINARY |
-|VARBINARY |
-|SMALLINT |
-|SHORT |
-|TINYINT |
-|BYTE |
-|CHAR |
-|VARCHAR
-
-## Data Types Supported for Row Tables
-
-| row-data-type |
-|--------|
-|BIGINT|
-|BLOB|
-|CHAR|
-|CHAR FOR BIT DATA|
-|CLOB|
-|DATE|
-|DECIMAL|
-|DOUBLE|
-|DOUBLE PRECISION|
-|FLOAT|
-|INTEGER|
-|JSON|
-|LONG VARCHAR|
-|LONG VARCHAR FOR BIT DATA|
-|NUMERIC|
-|REAL|
-|SMALLINT|
-|TIME|
-|TIMESTAMP|
-|User-Defined Types|
-|VARCHAR|
-|VARCHAR FOR BIT DATA|
-|XML|
+The special value NULL, denotes an unassigned or missing value of any of the types (columns that have been assigned as non-nullable using NOT NULL clause or the primary key columns cannot have a NULL value). The supported types are given below.
+
+- [BIGINT](#bigint)
+- [BINARY](#binary)
+- [BLOB](#blob)
+- [BOOLEAN](#boolean)
+- [BYTE](#byte)
+- [CLOB](#clob)
+- [CHAR](#char)
+- [DATE](#date)
+- [DECIMAL](#decimal)
+- [DOUBLE](#double)
+- [FLOAT](#float)
+- [INT](#int)
+- [INTEGER](#integer)
+- [LONG](#long)
+- [NUMERIC](#numeric)
+- [REAL](#real)
+- [SHORT](#short)
+- [SMALLINT](#smallint)
+- [STRING](#string)
+- [TIMESTAMP](#timestamp)
+- [TINYINT](#tinyint)
+- [VARBINARY](#varbinary)
+- [VARCHAR](#varchar)
+
+
+
+## BIGINT
+
+Provides 8 bytes storage for long integer values. An attempt to put a BIGINT value into another exact numeric type with smaller size/precision (e.g. INT) fails if the value overflows the maximum allowable by the smaller type.
+
+For behavior with other types in expressions, see Numeric type promotion in expressions, Storing values of one numeric data type in columns of another numeric data type.
+
+| | |
+|----------------------|---------------------------------------------------|
+| Equivalent Java type | java.lang.Long |
+| Minimum value | java.lang.Long.MIN\_VALUE (-9223372036854775808 ) |
+| Maximum value | java.lang.Long.MAX\_VALUE (9223372036854775807 ) |
+| JDBC metadata type | java.sql.Types.BIGINT |
+| JDBC methods | ResultSet.getLong, PreparedStatement.setLong |
+
+
+## BINARY
+
+
+## BLOB
+
+A binary large object represents an array of raw bytes of varying length.
+
+| | |
+|--------------------------------------|----------------------------------------------|
+| Equivalent Java type | java.lang.Blob |
+| Maximum length (also default length) | 2 GB - 1 (or 2,147,483,647) |
+| JDBC metadata type | java.sql.Types.BLOB |
+| JDBC methods | ResultSet.getBlob, PreparedStatement.setBlob |
+
+``` pre
+{ BLOB | BINARY LARGE OBJECT } [ ( length [{ K | M | G }] ) ]
+```
+
+The length of the BLOB is expressed in number of bytes by default. The suffixes K, M, and G stand for kilobyte, megabyte and gigabyte, and use the multiples of 1024, 1024\*1024, or 1024\*1024\*1024 respectively.
+
+``` pre
+CREATE TABLE blob_data(id INT primary key, data BLOB(10M));
+–- search for a blob
+select length(data) from blob_data where id = 100;
+```
+
+
+## BOOLEAN
+
+
+## BYTE
+
+
+## CHAR
+
+Provides for fixed-length strings. If a string value is shorter than the expected length, then spaces are inserted to pad the string to the expected length. If a string value is longer than the expected length, then any trailing blanks are trimmed to make the length same as the expected length, while an exception is raised if characters other than spaces are required to be truncated. For comparision operations, the shorter CHAR string is padded with spaces to the longer value. Similarly when mixing CHARs and VARCHARs in expressions , the shorter value is padded with spaces to the length of longer string.
+
+To represent a single quotation mark within a string, use two quotation marks:
+
+``` pre
+VALUES 'going to Chandra''s place'
+```
+
+The length of CHAR is an unsigned integer constant.
+
+| | |
+|----------------------|--------------------------------------------------|
+| Equivalent Java type | java.lang.String |
+| Maximum length | java.lang.Integer.MAX\_VALUE (2147483647 ) |
+| Default length | 1 |
+| JDBC metadata type | java.sql.Types.CHAR |
+| JDBC methods | ResultSet.getString, PreparedStatement.setString |
+
+``` pre
+CHAR[ACTER] [(length)]
+```
+
+
+
+## CLOB
+
+A character large object represents an array of characters of varying length. It is used to store large character-based data such as documents.
+
+The length is expressed in number characters, unless you specify the suffix K, M, or G, which uses the multiples of 1024, 1024\*1024, or 1024\*1024\*1024 respectively.
+
+| | |
+|--------------------------------------|----------------------------------------------|
+| Equivalent Java type | java.sql.Clob |
+| Maximum length (also default length) | 2 GB - 1 (or 2,147,483,647) |
+| JDBC metadata type | java.sql.Types.CLOB |
+| JDBC methods | ResultSet.getClob, PreparedStatement.setClob |
+
+``` pre
+{ CLOB | CHARACTER LARGE OBJECT } [ ( length [{ K | M | G }] ) ]
+```
+
+``` pre
+CREATE TABLE clob_data(id INT primary key, text CLOB(10M));
+–- search for a clob
+select text from clob_data where id = 100;
+```
+
+
+
+## DATE
+
+Provides for storage of a date as year-month-day. Supported formats are:
+
+``` pre
+yyyy-mm-dd
+```
+
+``` pre
+mm/dd/yyyy
+```
+
+``` pre
+dd.mm.yyyy
+```
+
+The year (yyyy) must always be expressed with four digits, while months (mm) and days (dd) may have either one or two digits. DATEs, TIMEs, and TIMESTAMPs must not be mixed with one another in expressions except with an explicit CAST.
+
+| | |
+|----------------------|----------------------------------------------|
+| Equivalent Java type | java.sql.Date |
+| JDBC metadata type | java.sql.Types.DATE |
+| JDBC methods | ResultSet.getDate, PreparedStatement.setDate |
+
+``` pre
+VALUES '2010-05-04'
+```
+
+``` pre
+VALUES DATE('2001-10-12')
+```
+
+The latter example uses the DATE() function described in the section Built-in functions and procedures.
+
+
+
+## DECIMAL
+
+Provides an exact decimal value having a specified precision and scale. The precision is the total number of digits both to the left and the right of the decimal point, and the scale is the number of digits in the fraction to the right of the decimal point.
+
+A numeric value (e.g. INT, BIGINT, SMALLINT) can be put into a DECIMAL as long as non-fractional precision is not lost else a range exception is thrown (SQLState: "22003"). When truncating trailing digits from a DECIMAL, the value is rounded down.
+
+For behavior with other types in expressions, see Numeric type promotion in expressions, Scale for decimal arithmetic and Storing values of one numeric data type in columns of another numeric data type.
+
+| | |
+|----------------------|----------------------------------------------------------|
+| Equivalent Java type | java.math.BigDecimal |
+| Precision min/max | 1 to 31 |
+| Scale min/max | less than or equal to precision |
+| Default precision | 5 |
+| Default scale | 0 |
+| JDBC metadata type | java.sql.Types.DECIMAL |
+| JDBC methods | ResultSet.getBigDecimal, PreparedStatement.setBigDecimal |
+
+``` pre
+{ DECIMAL | DEC } [(precision [, scale ])]
+```
+
+``` pre
+-- this cast loses fractional precision
+values cast (23.8372 AS decimal(4,1));
+-–- results in:
+23.8
+-- this cast is outside the range
+values cast (97824 AS decimal(4,1));
+–-- throws exception:
+ERROR 22003: The resulting value is outside the range for the data type DECIMAL/NUMERIC(4,1).
+```
+
+
+
+## DOUBLE
+
+Provides 8-byte storage for numbers using IEEE floating-point notation.
+
+Arithmetic operations do not round their resulting values to zero. If the values are too small, you will receive an exception. Numeric floating point constants are limited to a length of 30 characters.
+
+For behavior with other types in expressions, see Numeric type promotion in expressions, and Storing values of one numeric data type in columns of another numeric data type.
+
+| | |
+|-------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| Equivalent Java type | java.lang.Double Note: The maximum/minimum limits are different from those of java.lang.Double as noted below.
|
+| Minimum value | -1.79769E+30 |
+| Maximum value | 1.79769E+308 |
+| Smallest positive value | 2.225E-307 |
+| Largest negative value | -2.225E-307 |
+| Default precision | 5 |
+| Default scale | 0 |
+| JDBC metadata type | java.sql.Types.DOUBLE |
+| JDBC methods | ResultSet.getDouble, PreparedStatement.setDouble |
+
+``` pre
+–- examples of valid values
+values 233.31E3;
+values 8928E+06;
+-- this example will throw a range exception (SQLState: "42820")
+values 123456789012345678901234567890123456789e0;
+```
+
+
+
+## FLOAT
+
+Alias for a REAL or DOUBLE data type, depending on the specified precision. The default precision is 53 making it equivalent to DOUBLE. A precision of 23 or less makes FLOAT equivalent to REAL while greater than 23 makes it equivalent to DOUBLE.
+
+| | |
+|------------------------|-----------------------------------------------------------------------------------------------------------------------|
+| Equivalent Java type | java.lang.Double or java.lang.Float depending on precision |
+| Minumum/Maximum limits | Same as those for FLOAT if the precision is less than 23. Otherwise, same minimum/maximum limits as those for DOUBLE. |
+| Default precision | 53 |
+| JDBC metadata type | java.sql.Types.FLOAT |
+| JDBC methods | ResultSet.getFloat/getDouble, PreparedStatement.setFloat/setDouble |
+
+``` pre
+FLOAT [(precision)]
+```
+
+
+
+## INT
+
+
+
+## INTEGER
+
+Provides 4 bytes storage for integer values. INT can be used as a synonym for INTEGER in CREATE TABLE.
+
+For behavior with other types in expressions, see Numeric type promotion in expressions, and Storing values of one numeric data type in columns of another numeric data type.
+
+| | |
+|----------------------|--------------------------------------------|
+| Equivalent Java type | java.lang.Integer |
+| Minimum value | java.lang.Integer.MIN\_VALUE (-2147483648) |
+| Maximum value | java.lang.Integer.MAX\_VALUE (21474836487) |
+| JDBC metadata type | java.sql.Types.INTEGER |
+| JDBC methods | ResultSet.getInt, PreparedStatement.setInt |
+
+
+
+
+## LONG
+
+
+
+## NUMERIC
+
+Synonym for DECIMAL data type.
+
+The meta-data differences from DECIMAL are listed below. Otherwise, NUMERIC behaves identically to DECIMAL.
+
+| | |
+|--------------------|------------------------|
+| JDBC metadata type | java.sql.Types.NUMERIC |
+
+``` pre
+NUMERIC [(precision [, scale ])]
+```
+
+
+## REAL
+
+
+
+## SHORT
+
+
+
+## SMALLINT
+
+Provides 2 bytes storage for short integer values.
+
+For behavior with other types in expressions, see Numeric type promotion in expressions, and Storing values of one numeric data type in columns of another numeric data type.
+
+| | |
+|----------------------|------------------------------------------------|
+| Equivalent Java type | java.lang.Short |
+| Minimum value | java.lang.Short.MIN\_VALUE (-32768 ) |
+| Maximum value | java.lang.Short.MAX\_VALUE (32767) |
+| JDBC metadata type | java.sql.Types.SMALLINT |
+| JDBC methods | ResultSet.getShort, PreparedStatement.setShort |
+
+
+## STRING
+
+
+## TIMESTAMP
+
+Provides for storage of both DATE and TIME as a combined value. In addition it allows for fractional seconds having up to six digits. Supported formats are:
+
+``` pre
+yyyy-MM-dd hh:mm:ss[.nnnnnn]
+```
+
+``` pre
+yyyy-MM-dd-hh.mm.ss[.nnnnnn]
+```
+
+The year (yyyy) must always be expressed with four digits. Months (MM), days (dd), and hours (hh) may have one or two digits while minutes (mm) and seconds (ss) must have two digits. Microseconds, if present, may have between one and six digits. DATEs, TIMEs, and TIMESTAMPs must not be mixed with one another in expressions except with an explicit CAST.
+
+| | |
+|----------------------|--------------------------------------------------------|
+| Equivalent Java type | java.sql.Timestamp |
+| JDBC metadata type | java.sql.Types.TIMESTAMP |
+| JDBC methods | ResultSet.getTimestamp, PreparedStatement.setTimestamp |
+
+``` pre
+VALUES '2000-02-03 12:23:04'
+VALUES TIMESTAMP(' 2000-02-03 12:23:04.827')
+VALUES TIMESTAMP('2000-02-03 12:23:04')
+```
+
+The latter examples use the TIMESTAMP() function described in the section Built-in functions and procedures.
+
+
+## TINYINT
+
+
+
+## VARBINARY
+
+
+## VARCHAR
+
+Provides for variable-length strings with a maximum limit for length. If a string value is longer than the maximum length, then any trailing blanks are trimmed to make the length same as the maximum length, while an exception is raised if characters other than spaces are required to be truncated. When mixing CHARs and VARCHARs in expressions, the shorter value is padded with spaces to the length of longer string.
+
+The type of a string constant is CHAR, not VARCHAR. To represent a single quotation mark within a string, use two quotation marks:
+
+``` pre
+VALUES 'going to Chandra''s place'
+```
+
+The length of VARCHAR is an unsigned integer constant.
+
+| | |
+|----------------------|--------------------------------------------------|
+| Equivalent Java type | java.lang.String |
+| Maximum length | 32672 |
+| JDBC metadata type | java.sql.Types.VARCHAR |
+| JDBC methods | ResultSet.getString, PreparedStatement.setString |
+
+``` pre
+{ VARCHAR | CHAR VARYING | CHARACTER VARYING }(length)
+```
diff --git a/docs/reference/sql_reference/create-table.md b/docs/reference/sql_reference/create-table.md
index cd09e8c16a..b643f37813 100644
--- a/docs/reference/sql_reference/create-table.md
+++ b/docs/reference/sql_reference/create-table.md
@@ -53,31 +53,32 @@ Refer to the [constraint](#constraint) section for more information on table-con
`column-data-type`
```
column-data-type:
- STRING |
- INTEGER |
- INT |
BIGINT |
- LONG |
- DOUBLE |
- DECIMAL |
- NUMERIC |
+ BINARY |
+ BLOB |
+ BOOLEAN |
+ BYTE |
+ CLOB |
DATE |
- TIMESTAMP |
+ DECIMAL |
+ DOUBLE |
FLOAT |
+ INT |
+ INTEGER |
+ LONG |
+ NUMERIC |
REAL |
- BOOLEAN |
- CLOB |
- BLOB |
- BINARY |
- VARBINARY |
- SMALLINT |
SHORT |
+ SMALLINT |
+ STRING |
+ TIMESTAMP |
TINYINT |
- BYTE |
+ VARBINARY |
+ VARCHAR |
```
Column tables can also use ARRAY, MAP and STRUCT types.
Decimal and numeric has default precision of 38 and scale of 18.
-In this release, LONG is supported only for column tables. It is recommended to use BEGIN fo row tables instead.
+In this release, LONG is supported only for column tables. It is recommended to use BEGINT for row tables instead.
@@ -85,7 +86,9 @@ In this release, LONG is supported only for column tables. It is recommended to
The COLOCATE_WITH clause specifies a partitioned table to colocate with. The referenced table must already exist.
`PARTITION_BY`
-Use the PARTITION_BY {COLUMN} clause to provide a set of column names that determines the partitioning. If not specified, it is a replicated table. Column and row tables support hash partitioning on one or more columns. These are specified as comma-separated column names in the PARTITION_BY option of the CREATE TABLE DDL or createTable API. The hashing scheme follows the Spark Catalyst Hash Partitioning to minimize shuffles in joins. If no PARTITION_BY option is specified for a column table, then, the table is still partitioned internally on a generated scheme. The default number of storage partitions (BUCKETS) is 128 in cluster mode for column and row tables, and 11 in local mode for column and partitioned row tables. This can be changed using the BUCKETS option in CREATE TABLE DDL or createTable API.
+Use the PARTITION_BY {COLUMN} clause to provide a set of column names that determines the partitioning.
+If not specified, for row table (mentioned further for case of column table) it is a 'replicated row table'.
+Column and row tables support hash partitioning on one or more columns. These are specified as comma-separated column names in the PARTITION_BY option of the CREATE TABLE DDL or createTable API. The hashing scheme follows the Spark Catalyst Hash Partitioning to minimize shuffles in joins. If no PARTITION_BY option is specified for a column table, then, the table is still partitioned internally on a generated scheme. The default number of storage partitions (BUCKETS) is 128 in cluster mode for column and row tables, and 11 in local mode for column and partitioned row tables. This can be changed using the BUCKETS option in CREATE TABLE DDL or createTable API.
`BUCKETS`
@@ -221,6 +224,36 @@ Note that only the column names and datatypes from the queried table are used wh
For information on using the Apache Spark API, refer to [Using the Spark DataFrame API](../../sde/running_queries.md#using-the-spark-dataframe-api).
+### Example: Create Table with Eviction Settings
+
+Use eviction settings to keep your table within a specified limit, either by removing evicted data completely or by creating an overflow table that persists the evicted data to a disk store.
+
+1. Decide whether to evict based on:
+ - Entry count (useful if table row sizes are relatively uniform).
+
+ - Total bytes used.
+
+ - Percentage of JVM heap used. This uses the SnappyData resource manager. When the manager determines that eviction is required, the manager orders the eviction controller to start evicting from all tables where the eviction criterion is set to LRUHEAPPERCENT.
+
+2. Decide what action to take when the limit is reached:
+ - Locally destroy the row (partitioned tables only).
+ - Overflow the row data to disk.
+
+3. If you want to overflow data to disk (or persist the entire table to disk), configure a named disk store to use for the overflow data. If you do not specify a disk store when creating an overflow table, SnappyData stores the overflow data in the default disk store.
+
+4. Create the table with the required eviction configuration.
+
+ For example, to evict using LRU entry count and overflow evicted rows to a disk store:
+
+ CREATE TABLE Orders(OrderId INT NOT NULL,ItemId INT ) USING row
+ OPTIONS (EVICTION_BY 'LRUCOUNT 2', OVERFLOW 'true', DISKSTORE 'OverflowDiskStore', PERSISTENCE 'async')
+
+ To create a table that simply removes evicted data from memory without persisting the evicted data, use the DESTROY eviction action. For example:
+ Default in SnappyData is synchronous persistence, overflow is true and eviction_by is LRUHEAPPERCENT
+
+ CREATE TABLE Orders(OrderId INT NOT NULL,ItemId INT) USING row
+ OPTIONS (PARTITION_BY 'OrderId', EVICTION_BY 'LRUMEMSIZE 1000')
+
### Constraint (only for Row Tables)
A CONSTRAINT clause is an optional part of a CREATE TABLE statement that defines a rule to which table data must conform.
@@ -252,12 +285,8 @@ If you do not specify a name for a column or table constraint, then SnappyData g
### Identity Columns (only for Row Tables)
-SnappyData supports both GENERATED ALWAYS and GENERATED BY DEFAULT identity columns only for BIGINT and INTEGER data types. The START WITH and INCREMENT BY clauses are supported only for GENERATED BY DEFAULT identity columns.
+SnappyData supports both GENERATED ALWAYS and GENERATED BY DEFAULT identity columns only for BIGINT and INTEGER data types. The START WITH and INCREMENT BY clauses are supported only for GENERATED BY DEFAULT identity columns.
-
For a GENERATED ALWAYS identity column, SnappyData increments the default value on every insertion, and stores the incremented value in the column. You cannot insert a value directly into a GENERATED ALWAYS identity column, and you cannot update a value in a GENERATED ALWAYS identity column. Instead, you must either specify the DEFAULT keyword when inserting data into the table or you must leave the identity column out of the insertion column list.
Consider a table with the following column definition:
diff --git a/docs/reference/system_tables/system_tables.md b/docs/reference/system_tables/system_tables.md
index 6ce8a699dd..95dded6061 100644
--- a/docs/reference/system_tables/system_tables.md
+++ b/docs/reference/system_tables/system_tables.md
@@ -6,8 +6,6 @@ The following system tables are available:
* [MEMBERS](members.md)
-* [MEMORYANALYTICS](memoryanalytics.md)
-
* [SYSDISKSTORES](sysdiskstores.md)
* [SYSTABLES](systables.md)
\ No newline at end of file
diff --git a/docs/release_notes/known_issues.md b/docs/release_notes/known_issues.md
index bfb7ddfc3f..525983c005 100644
--- a/docs/release_notes/known_issues.md
+++ b/docs/release_notes/known_issues.md
@@ -18,6 +18,12 @@ The following key issues have been registered as bugs in the SnappyData bug trac
+SNAP-1375 |
+JVM crash reported |
+This was reported on: - RHEL kernel version: 3.10.0-327.13.1.el7.x86_64 - Java version: 1.8.0_121 |
+To resolve this, use: - RHEL kernel version: 3.10.0-693.2.2.el7.x86_64 - Java version: 1.8.0_144 |
+
+
SNAP-1422 |
Catalog in smart connector inconsistent with servers |
Catalog in smart connector inconsistent with servers|When a table is queried from spark-shell (or from an application that uses smart connector mode) the table metadata is cached on the smart connector side. If this table is dropped from SnappyData embedded cluster (by using snappy-shell, or JDBC application, or a Snappy job), the metadata on the smart connector side stays cached even though catalog has changed (table is dropped). In such cases, the user may see unexpected errors like "org.apache.spark.sql.AnalysisException: Table `SNAPPYTABLE` already exists" in the smart connector app side for example for `DataFrameWriter.saveAsTable()` API if the same table name that was dropped is used in `saveAsTable()` |
@@ -55,9 +61,9 @@ select
and l_shipinstruct = 'DELIVER IN PERSON'
)
-
+
select
- sum(l_extendedprice)
+ sum(l_extendedprice)
from
LINEITEM,
PART
@@ -67,9 +73,24 @@ select
-
-
-
+
+SNAP-1911 |
+JVM crash reported |
+This was reported on: - RHEL kernel version: 3.10.0-327.13.1.el7.x86_64 - Java version: 1.8.0_131 |
+To resolve this, use: - RHEL kernel version: 3.10.0-693.2.2.el7.x86_64 - Java version: 1.8.0_144 |
+
+
+SNAP-1999 |
+JVM crash reported |
+This was reported on: - RHEL kernel version: 3.10.0-327.13.1.el7.x86_64 - Java version: 1.8.0_131 |
+To resolve this, use: - RHEL kernel version: 3.10.0-693.2.2.el7.x86_64 - Java version: 1.8.0_144 |
+
+
+SNAP-2017 |
+JVM crash reported |
+This was reported on: - RHEL kernel version: 3.10.0-514.10.2.el7.x86_64 - Java version: 1.8.0_144 |
+To resolve this, use: - RHEL kernel version:Â 3.10.0-693.2.2.el7.x86_64 - Java version: 1.8.0_144 |
+