You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by al...@apache.org on 2020/06/19 06:03:32 UTC

[zeppelin] branch branch-0.9 updated: [ZEPPELIN-4796] Better representation of string-like data in Cassandra interpreter

This is an automated email from the ASF dual-hosted git repository.

alexott pushed a commit to branch branch-0.9
in repository https://gitbox.apache.org/repos/asf/zeppelin.git


The following commit(s) were added to refs/heads/branch-0.9 by this push:
     new dbfea93  [ZEPPELIN-4796] Better representation of string-like data in Cassandra interpreter
dbfea93 is described below

commit dbfea93fbecb54acc37e945bf881ff0287d178c4
Author: Alex Ott <al...@gmail.com>
AuthorDate: Sun Jun 7 11:49:18 2020 +0200

    [ZEPPELIN-4796] Better representation of string-like data in Cassandra interpreter
    
    ### What is this PR for?
    
    Refactoring of the Cassandra interpreter to a new driver changes the results representation to be strict CQL format, that is not always desirable.
    
    This PR improves the situation by allow to control the output format (CQL vs human-readable), and also control such things as locale, timezone, format of timestamp/date/time columns, and precision for float & double types.
    
    All settings could be set either on interpreter, or cell level.
    
    ### What type of PR is it?
    
    Improvement
    
    ### What is the Jira issue?
    
    * https://issues.apache.org/jira/browse/ZEPPELIN-4796
    
    ### How should this be tested?
    * https://travis-ci.org/github/alexott/zeppelin/builds/698160753
    * Tested manually (see screenshot)
    
    ### Screenshots (if appropriate)
    
    ![Screenshot from 2020-06-14 11-36-09](https://user-images.githubusercontent.com/30342/84590026-4ad10980-ae33-11ea-9fd6-b07ce779e766.png)
    
    Author: Alex Ott <al...@gmail.com>
    
    Closes #3790 from alexott/ZEPPELIN-4796 and squashes the following commits:
    
    6e4fcc9eb [Alex Ott] [ZEPPELIN-4796] Added cell-level formatting, and documentation
    e008982a6 [Alex Ott] [ZEPPELIN-4796] initial implementation of new formatting
    
    (cherry picked from commit 46ac4376c1e357e7defe845aa3e930f54c7ec140)
    Signed-off-by: Alex Ott <al...@apache.org>
---
 .../zeppelin/cassandra/CassandraInterpreter.java   |  18 +-
 .../src/main/resources/interpreter-setting.json    |  56 ++++++
 cassandra/src/main/resources/scalate/helpMenu.ssp  | 126 ++++++++++++-
 .../apache/zeppelin/cassandra/CqlFormatter.scala   | 197 +++++++++++++++++++++
 .../zeppelin/cassandra/InterpreterLogic.scala      |  58 ++++--
 .../zeppelin/cassandra/ParagraphParser.scala       |   8 +-
 .../cassandra/CassandraInterpreterTest.java        | 155 ++++++++++------
 .../zeppelin/cassandra/InterpreterLogicTest.java   |   3 +-
 cassandra/src/test/resources/prepare_all.cql       |  66 ++++++-
 .../src/test/resources/scalate/DescribeTables.html |   3 +
 .../src/test/resources/scalate/DescribeTypes.html  |   3 +
 .../zeppelin/cassandra/CqlFormatterTest.scala      | 130 ++++++++++++++
 docs/interpreter/cassandra.md                      | 148 +++++++++++++++-
 13 files changed, 876 insertions(+), 95 deletions(-)

diff --git a/cassandra/src/main/java/org/apache/zeppelin/cassandra/CassandraInterpreter.java b/cassandra/src/main/java/org/apache/zeppelin/cassandra/CassandraInterpreter.java
index 682bf2d..0f8197b 100644
--- a/cassandra/src/main/java/org/apache/zeppelin/cassandra/CassandraInterpreter.java
+++ b/cassandra/src/main/java/org/apache/zeppelin/cassandra/CassandraInterpreter.java
@@ -112,6 +112,22 @@ public class CassandraInterpreter extends Interpreter {
   public static final String CASSANDRA_TRUSTSTORE_PASSWORD =
           "cassandra.ssl.truststore.password";
 
+  public static final String CASSANDRA_FORMAT_FLOAT_PRECISION =
+          "cassandra.format.float_precision";
+  public static final String CASSANDRA_FORMAT_DOUBLE_PRECISION =
+          "cassandra.format.double_precision";
+  public static final String CASSANDRA_FORMAT_TIMESTAMP =
+          "cassandra.format.timestamp";
+  public static final String CASSANDRA_FORMAT_TIME =
+          "cassandra.format.time";
+  public static final String CASSANDRA_FORMAT_DATE =
+          "cassandra.format.date";
+  public static final String CASSANDRA_FORMAT_TYPE =
+          "cassandra.format.output";
+  public static final String CASSANDRA_FORMAT_TIMEZONE =
+          "cassandra.format.timezone";
+  public static final String CASSANDRA_FORMAT_LOCALE =
+          "cassandra.format.locale";
 
   public static final String DEFAULT_HOST = "127.0.0.1";
   public static final String DEFAULT_PORT = "9042";
@@ -226,7 +242,7 @@ public class CassandraInterpreter extends Interpreter {
     }
 
     session = clusterBuilder.withConfigLoader(loader).build();
-    helper = new InterpreterLogic(session);
+    helper = new InterpreterLogic(session, properties);
   }
 
   @Override
diff --git a/cassandra/src/main/resources/interpreter-setting.json b/cassandra/src/main/resources/interpreter-setting.json
index f8d2951..34de7a1 100644
--- a/cassandra/src/main/resources/interpreter-setting.json
+++ b/cassandra/src/main/resources/interpreter-setting.json
@@ -199,6 +199,62 @@
         "defaultValue": "none",
         "description": "Cassandra truststore password. Default = none",
         "type": "password"
+      },
+      "cassandra.format.output": {
+        "envName": null,
+        "propertyName": "cassandra.format.output",
+        "defaultValue": "human",
+        "description": "Output format: human-readable, or strict CQL. Default = human",
+        "type": "string"
+      },
+      "cassandra.format.locale": {
+        "envName": null,
+        "propertyName": "cassandra.format.locale",
+        "defaultValue": "en_US",
+        "description": "Locale for formatting of output data. Default = en_US",
+        "type": "string"
+      },
+      "cassandra.format.timezone": {
+        "envName": null,
+        "propertyName": "cassandra.format.timezone",
+        "defaultValue": "UTC",
+        "description": "Timezone for output of time/date-related values. Default = UTC",
+        "type": "string"
+      },
+      "cassandra.format.timestamp": {
+        "envName": null,
+        "propertyName": "cassandra.format.timestamp",
+        "defaultValue": "yyyy-MM-dd'T'HH:mm:ss.SSSXXX",
+        "description": "Format string for timestamp columns",
+        "type": "string"
+      },
+      "cassandra.format.date": {
+        "envName": null,
+        "propertyName": "cassandra.format.date",
+        "defaultValue": "yyyy-MM-dd",
+        "description": "Format string for date columns",
+        "type": "string"
+      },
+      "cassandra.format.time": {
+        "envName": null,
+        "propertyName": "cassandra.format.time",
+        "defaultValue": "HH:mm:ss.SSS",
+        "description": "Format string for time columns",
+        "type": "string"
+      },
+      "cassandra.format.float_precision": {
+        "envName": null,
+        "propertyName": "cassandra.format.float_precision",
+        "defaultValue": "5",
+        "description": "Precision for formatting of float columns",
+        "type": "number"
+      },
+      "cassandra.format.double_precision": {
+        "envName": null,
+        "propertyName": "cassandra.format.double_precision",
+        "defaultValue": "12",
+        "description": "Precision for formatting of double columns",
+        "type": "number"
       }
     },
     "editor": {
diff --git a/cassandra/src/main/resources/scalate/helpMenu.ssp b/cassandra/src/main/resources/scalate/helpMenu.ssp
index f75eef6..2883b86 100644
--- a/cassandra/src/main/resources/scalate/helpMenu.ssp
+++ b/cassandra/src/main/resources/scalate/helpMenu.ssp
@@ -24,6 +24,7 @@
 <%@ val basicCommandsId: UUID = Uuids.random() %>
 <%@ val schemaDiscoveryId: UUID = Uuids.random() %>
 <%@ val queryParamsId: UUID = Uuids.random() %>
+<%@ val formatParamsId: UUID = Uuids.random() %>
 <%@ val preparedStatementsId: UUID = Uuids.random() %>
 <%@ val dynamicFormsId: UUID = Uuids.random() %>
 <%@ val configurationId: UUID = Uuids.random() %>
@@ -58,6 +59,11 @@
                         </a>
                     </li>
                     <li>
+                        <a role="button" data-toggle="collapse" data-target="#${formatParamsId}">
+                            <span class="text-info"><i class="glyphicon glyphicon-bookmark"/>&nbsp;&nbsp;Formatting Parameters</span>
+                        </a>
+                    </li>
+                    <li>
                         <a role="button" data-toggle="collapse" data-target="#${preparedStatementsId}">
                             <span class="text-info"><i class="glyphicon glyphicon-bookmark"/>&nbsp;&nbsp;Prepared Statements</span>
                         </a>
@@ -519,7 +525,7 @@
                             Please note that you <strong>should not add semi-colon (;)</strong> at the end of each parameter statement
                         </div>
 
-                        Some example:
+                        Some examples:
                         <br/><br/>
                         <div class="row">
                             <div class="col-md-8 col-md-offset-2">
@@ -576,6 +582,119 @@
     <div class="panel panel-default">
         <div class="panel-heading" role="tab">
             <h4 class="panel-title">
+                <a role="button" data-toggle="collapse" data-target="#${formatParamsId}" aria-expanded="false">
+                    <span class="text-info"><strong>Formatting Parameters</strong></span>
+                </a>
+            </h4>
+        </div>
+        <div id="${formatParamsId}" class="panel-collapse collapse" role="tabpanel">
+            <div class="panel-body">
+
+                <div class="panel panel-default">
+                  <div class="panel-body">
+                    <p>
+                        Sometimes you want to be able to adjust formatting of results just
+                        for single paragraph. You can do this by passing one or more
+                        following parameters in the <tt>%cassandra</tt> interpreter local
+                        properties:
+
+                        <br/><br/>
+                        <table class="table table-bordered">
+                            <caption>
+                                <h4>Formatting Parameters</h4>
+                            </caption>
+                            <thead>
+                                <tr>
+                                    <th>Parameter</th>
+                                    <th>Syntax</th>
+                                    <th>Description</th>
+                                </tr>
+                            </thead>
+                            <tbody>
+                                <tr>
+                                    <td>Output Format</td>
+                                    <td><strong>outputFormat=<em>cql|human</em></strong></td>
+                                    <td>Selects formatting of the results - as CQL
+                                    literals, or human-readable (default)</td>
+                                </tr>
+                                <tr>
+                                    <td>Locale</td>
+                                    <td><strong>locale=<em>value</em></strong></td>
+                                    <td>Name of the locale for formatting of results.
+                                    Should be a locale name supported by JVM.</td>
+                                </tr>
+                                <tr>
+                                    <td>Timezone</td>
+                                    <td><strong>timezone=<em>value</em></strong></td>
+                                    <td>Timezone string for formatting of values of
+                                    time/date/timestamp type.  Should be a timezone
+                                    supported by JVM.</td>
+                                </tr>
+                                <tr>
+                                    <td>Timestamp Format</td>
+                                    <td><strong>timestampFormat=<em>string</em></strong></td>
+                                    <td>Format string
+                                    for <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</>
+                                    that will be used for formatting of the values
+                                    of <tt>timestamp</tt> type</td>
+                                </tr>
+                                <tr>
+                                    <td>Time Format</td>
+                                    <td><strong>timeFormat=<em>string</em></strong></td>
+                                    <td>Format string
+                                    for <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</>
+                                    that will be used for formatting of the values
+                                    of <tt>time</tt> type</td>
+                                </tr>
+                                <tr>
+                                    <td>Date Format</td>
+                                    <td><strong>dateFormat=<em>string</em></strong></td>
+                                    <td>Format string
+                                    for <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</>
+                                    that will be used for formatting of the values
+                                    of <tt>date</tt> type</td>
+                                </tr>
+                                <tr>
+                                  <td>Float Precision</td>
+                                  <td><strong>floatPrecision=<em>int value</em></strong></td>
+                                  <td>Precision when output values of <tt>float</tt> type</td>
+                                </tr>
+                                <tr>
+                                  <td>Double Precision</td>
+                                  <td><strong>doublePrecision=<em>int value</em></strong></td>
+                                  <td>Precision when output values of <tt>double</tt> type</td>
+                                </tr>
+                            </tbody>
+                        </table>
+<br/><br/><b>Note</b>: if you need to apply the same output for all paragraphs, you can set
+corresponding settings on the interpreter level!
+<br/><br/>
+
+                        Some examples:
+                        <br/><br/>
+                        <div class="row">
+                            <div class="col-md-8 col-md-offset-2">
+                              <pre>
+%cassandra(locale=de_DE, floatPrecision=2, doublePrecision=4, timeFormat=hh:mma, timestampFormat=MM/dd/yy HH:mm, dateFormat="E, d MMM yy", timezone=Etc/GMT+2)
+select date, time, timestamp, double, float, tuple, udt from zep.test_format;
+                                
+%cassandra(outputFormat=cql)
+select id, double, float, text, date, time, timestamp from zep.test_format;                                
+                              </pre>
+                            </div>
+                        </div>
+                    </p>
+                    </div>
+                </div>
+
+
+            </div>
+        </div>
+    </div>        
+    
+    <div class="panel panel-default">
+        <div class="panel-heading" role="tab">
+            <h4 class="panel-title">
                 <a role="button" data-toggle="collapse" data-target="#${preparedStatementsId}" aria-expanded="false">
                     <span class="text-info"><strong>Prepared Statements</strong></span>
                 </a>
@@ -1003,9 +1122,10 @@
                 <strong>4.0</strong>&nbsp;:
                 <br/>
                 <ul>
-                    <li>Migrate to DataStax Java driver 4.x that supports both Cassandra and DSE.  This contains breaking changes, like, removal of <code>@retryPolicy</code></li>
+                    <li>Migrate to DataStax Java driver 4.x that supports both Cassandra and DSE.  This contains breaking changes, like, deprecation of <code>@retryPolicy</code></li>
                     <li>Fix schema disagreement that could arise when executing multiple DDL statements</li>
                     <li>Added support for <code>--</code> comment style</li>
+                    <li>Added options for formatting of output (interpreter and cell level)</li>
                 </ul>
                 <strong>3.0</strong>&nbsp;:
                 <br/>
@@ -1050,4 +1170,4 @@
             </div>
         </div>            
     </div>    
-</div>
\ No newline at end of file
+</div>
diff --git a/cassandra/src/main/scala/org/apache/zeppelin/cassandra/CqlFormatter.scala b/cassandra/src/main/scala/org/apache/zeppelin/cassandra/CqlFormatter.scala
new file mode 100644
index 0000000..ff15e82
--- /dev/null
+++ b/cassandra/src/main/scala/org/apache/zeppelin/cassandra/CqlFormatter.scala
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zeppelin.cassandra
+
+import java.net.InetAddress
+import java.nio.ByteBuffer
+import java.text.DecimalFormat
+import java.text.NumberFormat
+import java.time.{Instant, LocalDate, LocalTime, ZoneId}
+import java.time.format.DateTimeFormatter
+import java.util.concurrent.ConcurrentHashMap
+import java.util.{Locale, Properties, TimeZone}
+
+import com.datastax.oss.driver.api.core.`type`.codec.TypeCodec
+import com.datastax.oss.driver.api.core.`type`.{DataType, DataTypes}
+import com.datastax.oss.driver.api.core.cql.Row
+import com.datastax.oss.driver.api.core.data.{TupleValue, UdtValue}
+import io.netty.buffer.ByteBufUtil
+import org.apache.commons.lang3.LocaleUtils
+import org.apache.zeppelin.interpreter.InterpreterException
+
+import scala.collection.JavaConverters._
+
+object CqlFormatter {
+  val DEFAULT_TIMEZONE = "UTC"
+  val DEFAULT_FORMAT = "human"
+  val DEFAULT_FLOAT_PRECISION = 5
+  val DEFAULT_DOUBLE_PRECISION = 12
+  val DEFAULT_TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"
+  val DEFAULT_TIME_FORMAT = "HH:mm:ss.SSS"
+  val DEFAULT_DATE_FORMAT = "yyyy-MM-dd"
+  val DEFAULT_LOCALE = "en_US"
+
+  val allAvailableTimezones: Set[String] = TimeZone.getAvailableIDs.toSet
+
+  def getNumberFormatter(locale: Locale, precision: Int): DecimalFormat = {
+    val df = NumberFormat.getNumberInstance(locale).asInstanceOf[DecimalFormat]
+    df.applyPattern("#." + "#" * precision)
+    df
+  }
+
+  def getDateTimeFormatter(locale: Locale, timeZone: ZoneId, format: String): DateTimeFormatter = {
+    try {
+      DateTimeFormatter.ofPattern(format).withLocale(locale).withZone(timeZone)
+    } catch {
+      case ex: IllegalArgumentException =>
+        throw new InterpreterException(
+          s"Invalid time/date format: '$format'. error message: ${ex.getMessage}")
+    }
+  }
+
+  def getLocale(localeStr: String): Locale = {
+    try {
+      LocaleUtils.toLocale(localeStr)
+    } catch {
+      case _: IllegalArgumentException =>
+        throw new InterpreterException(s"Invalid locale: '$localeStr'")
+    }
+  }
+
+  def getTimezone(tzStr: String): ZoneId = {
+    if (!allAvailableTimezones.contains(tzStr)) {
+      throw new InterpreterException(s"Invalid timezone: '$tzStr'")
+    }
+    TimeZone.getTimeZone(tzStr).toZoneId
+  }
+
+}
+
+class CqlFormatter(val outputFormat: String = CqlFormatter.DEFAULT_FORMAT,
+                   val floatPrecision: Int = CqlFormatter.DEFAULT_FLOAT_PRECISION,
+                   val doublePrecision: Int = CqlFormatter.DEFAULT_DOUBLE_PRECISION,
+                   val timestampFormat: String = CqlFormatter.DEFAULT_TIMESTAMP_FORMAT,
+                   val timeFormat: String = CqlFormatter.DEFAULT_TIME_FORMAT,
+                   val dateFormat: String = CqlFormatter.DEFAULT_DATE_FORMAT,
+                   val timeZoneId: String = CqlFormatter.DEFAULT_TIMEZONE,
+                   val localeStr: String = CqlFormatter.DEFAULT_LOCALE) {
+
+  val isCqlFormat: Boolean = "cql".equalsIgnoreCase(outputFormat)
+  val locale: Locale = CqlFormatter.getLocale(localeStr)
+  val timeZone: ZoneId = CqlFormatter.getTimezone(timeZoneId)
+
+  val floatFormatter: DecimalFormat = CqlFormatter.getNumberFormatter(locale, floatPrecision)
+  val doubleFormatter: DecimalFormat = CqlFormatter.getNumberFormatter(locale, doublePrecision)
+
+  val timestampFormatter: DateTimeFormatter = CqlFormatter.getDateTimeFormatter(
+    locale, timeZone, timestampFormat)
+  val timeFormatter: DateTimeFormatter = CqlFormatter.getDateTimeFormatter(
+    locale, timeZone, timeFormat)
+  val dateFormatter: DateTimeFormatter = CqlFormatter.getDateTimeFormatter(
+    locale, timeZone, dateFormat)
+
+  def this(properties: Properties) {
+    this(
+      properties.getProperty(CassandraInterpreter.CASSANDRA_FORMAT_TYPE,
+        CqlFormatter.DEFAULT_FORMAT),
+      properties.getProperty(CassandraInterpreter.CASSANDRA_FORMAT_FLOAT_PRECISION,
+        CqlFormatter.DEFAULT_FLOAT_PRECISION.toString).toInt,
+      properties.getProperty(
+        CassandraInterpreter.CASSANDRA_FORMAT_DOUBLE_PRECISION,
+        CqlFormatter.DEFAULT_DOUBLE_PRECISION.toString).toInt,
+      properties.getProperty(CassandraInterpreter.CASSANDRA_FORMAT_TIMESTAMP,
+        CqlFormatter.DEFAULT_TIMESTAMP_FORMAT),
+      properties.getProperty(CassandraInterpreter.CASSANDRA_FORMAT_TIME,
+        CqlFormatter.DEFAULT_TIME_FORMAT),
+      properties.getProperty(CassandraInterpreter.CASSANDRA_FORMAT_DATE,
+        CqlFormatter.DEFAULT_DATE_FORMAT),
+      properties.getProperty(CassandraInterpreter.CASSANDRA_FORMAT_TIMEZONE,
+        CqlFormatter.DEFAULT_TIMEZONE),
+      properties.getProperty(CassandraInterpreter.CASSANDRA_FORMAT_LOCALE,
+        CqlFormatter.DEFAULT_LOCALE)
+    )
+  }
+
+  def copy(outputFormat: String = this.outputFormat,
+           floatPrecision: Int = this.floatPrecision,
+           doublePrecision: Int = this.doublePrecision,
+           timestampFormat: String = this.timestampFormat,
+           timeFormat: String = this.timeFormat,
+           dateFormat: String = this.dateFormat,
+           timeZoneId: String = this.timeZoneId,
+           localeStr: String = this.localeStr) =
+    new CqlFormatter(outputFormat, floatPrecision, doublePrecision, timestampFormat,
+      timeFormat, dateFormat, timeZoneId, localeStr)
+
+  def formatHuman(obj: Object): String = {
+    if (obj == null) {
+      "null"
+    } else {
+      obj match {
+        case f: java.lang.Float =>
+          floatFormatter.format(f)
+        case d: java.lang.Double =>
+          doubleFormatter.format(d)
+        case m: java.util.Map[Object, Object] =>
+          m.asScala.map{case(k,v) => formatHuman(k) + ": " + formatHuman(v)}.mkString("{", ", ", "}")
+        case l: java.util.List[Object] =>
+          l.asScala.map(x => formatHuman(x)).mkString("[", ", ", "]")
+        case s: java.util.Set[Object] =>
+          s.asScala.map(x => formatHuman(x)).mkString("{", ", ", "}")
+        case t: Instant =>
+            timestampFormatter.format(t.atZone(timeZone))
+        case d: LocalDate =>
+          dateFormatter.format(d)
+        case t: LocalTime =>
+          timeFormatter.format(t)
+        case b: ByteBuffer =>
+          "0x" + ByteBufUtil.hexDump(b.array())
+        case i: InetAddress =>
+          i.getHostAddress
+        case t: TupleValue =>
+          (0 until t.size()).map(i => formatHuman(t.getObject(i))).mkString("(", ", ", ")")
+        case u: UdtValue =>
+          val names = u.getType.getFieldNames
+          (0 until u.size()).map(i => names.get(i).asInternal + ": " + formatHuman(u.getObject(i)))
+            .mkString("{", ", ", "}")
+
+        case _ => obj.toString()
+      }
+    }
+  }
+
+  def format(obj: Object, codec: TypeCodec[AnyRef]): String = {
+    if (isCqlFormat) {
+      codec.format(obj)
+    } else {
+      formatHuman(obj)
+    }
+  }
+
+  def getValueAsString(row: Row, name: String, dataType: DataType): String = {
+    val value = row.getObject(name)
+    if (isCqlFormat) {
+      format(value, row.codecRegistry().codecFor(dataType, value))
+    } else {
+      formatHuman(value)
+    }
+  }
+
+  override def toString = s"CqlFormatter(format=$outputFormat, fp=$floatPrecision, dp=$doublePrecision, " +
+    s"tsFormat=$timestampFormat, tmFormat=$timeFormat, dtFormat=$dateFormat, " +
+    s"timeozone=$timeZoneId, locale=$localeStr)"
+}
diff --git a/cassandra/src/main/scala/org/apache/zeppelin/cassandra/InterpreterLogic.scala b/cassandra/src/main/scala/org/apache/zeppelin/cassandra/InterpreterLogic.scala
index bd37c45..2e45eb0 100644
--- a/cassandra/src/main/scala/org/apache/zeppelin/cassandra/InterpreterLogic.scala
+++ b/cassandra/src/main/scala/org/apache/zeppelin/cassandra/InterpreterLogic.scala
@@ -19,10 +19,10 @@ package org.apache.zeppelin.cassandra
 import java.io.{ByteArrayOutputStream, PrintStream}
 import java.net.InetAddress
 import java.nio.ByteBuffer
-import java.text.SimpleDateFormat
 import java.time.format.DateTimeFormatter
 import java.time.{Duration, Instant, LocalDateTime, ZoneOffset}
 import java.util
+import java.util.Properties
 import java.util.concurrent.ConcurrentHashMap
 
 import com.datastax.oss.driver.api.core.`type`.{DataType, ListType, MapType, SetType, TupleType, UserDefinedType}
@@ -30,7 +30,7 @@ import com.datastax.oss.driver.api.core.`type`.DataTypes._
 import com.datastax.oss.driver.api.core.`type`.codec.TypeCodec
 import com.datastax.oss.driver.api.core.`type`.codec.registry.CodecRegistry
 import com.datastax.oss.driver.api.core.cql.{BatchStatement, BatchType, BatchableStatement, BoundStatement, ExecutionInfo, PreparedStatement, ResultSet, Row, SimpleStatement, Statement}
-import com.datastax.oss.driver.api.core.{ConsistencyLevel, CqlSession, DriverException, ProtocolVersion}
+import com.datastax.oss.driver.api.core.{ConsistencyLevel, CqlSession, DriverException}
 import org.apache.zeppelin.cassandra.TextBlockHierarchy._
 import org.apache.zeppelin.display.ui.OptionInput.ParamOption
 import org.apache.zeppelin.interpreter.InterpreterResult.Code
@@ -91,10 +91,12 @@ object InterpreterLogic {
  *
  * @param session java driver session
  */
-class InterpreterLogic(val session: CqlSession)  {
+class InterpreterLogic(val session: CqlSession, val properties: Properties)  {
 
   val enhancedSession: EnhancedSession = new EnhancedSession(session)
 
+  val formatter: CqlFormatter = new CqlFormatter(properties)
+
   import InterpreterLogic._
 
   def interpret[StatementT <: Statement[StatementT]](session:CqlSession, stringStatements : String,
@@ -103,13 +105,12 @@ class InterpreterLogic(val session: CqlSession)  {
     logger.info(s"Executing CQL statements : \n\n$stringStatements\n")
 
     try {
-      val protocolVersion = session.getContext.getProtocolVersion
-
-      val queries:List[AnyBlock] = parseInput(stringStatements)
+      val queries: List[AnyBlock] = parseInput(stringStatements)
 
       val queryOptions = extractQueryOptions(queries
         .filter(_.blockType == ParameterBlock)
         .map(_.get[QueryParameters]))
+      val executionFormatter = extractFormatter(context)
 
       logger.info(s"Current Cassandra query options = $queryOptions")
 
@@ -160,7 +161,7 @@ class InterpreterLogic(val session: CqlSession)  {
       if (results.nonEmpty) {
         results.last match {
           case(res: ResultSet, st: StatementT) =>
-            buildResponseMessage((res, st), protocolVersion)
+            buildResponseMessage((res, st), executionFormatter)
           case(output: String, _) => new InterpreterResult(Code.SUCCESS, output)
           case _ => throw new InterpreterException(s"Cannot parse result type : ${results.last}")
         }
@@ -187,7 +188,7 @@ class InterpreterLogic(val session: CqlSession)  {
   }
 
   def buildResponseMessage[StatementT <: Statement[StatementT]](lastResultSet: (ResultSet, StatementT),
-                                                                protocolVersion: ProtocolVersion): InterpreterResult = {
+                                                                fmt: CqlFormatter): InterpreterResult = {
     val output = new StringBuilder()
     val rows: collection.mutable.ArrayBuffer[Row] = ArrayBuffer()
 
@@ -216,8 +217,7 @@ class InterpreterLogic(val session: CqlSession)  {
               if (row.isNull(name)) {
                 null
               } else {
-                val value = row.getObject(name)
-                row.codecRegistry().codecFor(dataType, value).format(value)
+                fmt.getValueAsString(row, name, dataType)
               }
           }
           output.append(data.mkString("\t")).append("\n")
@@ -251,10 +251,42 @@ class InterpreterLogic(val session: CqlSession)  {
     }
   }
 
-  def extractQueryOptions(parameters: List[QueryParameters]): CassandraQueryOptions = {
-
-    logger.debug(s"Extracting query options from $parameters")
+  def extractFormatter(context: InterpreterContext): CqlFormatter = {
+    if (context == null) {
+      formatter
+    } else {
+      val props = context.getLocalProperties
+      logger.debug("Extracting query options from {}", props)
+      if (props == null || props.isEmpty) {
+        formatter
+      } else {
+        logger.debug("extracting properties into formatter. default: {}", formatter)
+        val locale = props.getOrDefault("locale", formatter.localeStr)
+        val timezone = props.getOrDefault("timezone", formatter.timeZoneId)
+        val outputFormat = props.getOrDefault("outputFormat", formatter.outputFormat)
+        val floatPrecision: Int = props.getOrDefault("floatPrecision",
+          formatter.floatPrecision.toString).toInt
+        val doublePrecision: Int = props.getOrDefault("doublePrecision",
+          formatter.doublePrecision.toString).toInt
+        val timestampFormat = props.getOrDefault("timestampFormat", formatter.timestampFormat)
+        val timeFormat = props.getOrDefault("timeFormat", formatter.timeFormat)
+        val dateFormat = props.getOrDefault("dateFormat", formatter.dateFormat)
+
+        new CqlFormatter(
+          outputFormat = outputFormat,
+          floatPrecision = floatPrecision,
+          doublePrecision = doublePrecision,
+          timestampFormat = timestampFormat,
+          timeFormat = timeFormat,
+          dateFormat = dateFormat,
+          timeZoneId = timezone,
+          localeStr = locale
+        )
+      }
+    }
+  }
 
+  def extractQueryOptions(parameters: List[QueryParameters]): CassandraQueryOptions = {
     val consistency: Option[ConsistencyLevel] = parameters
       .filter(_.paramType == ConsistencyParam)
       .map(_.getParam[Consistency])
diff --git a/cassandra/src/main/scala/org/apache/zeppelin/cassandra/ParagraphParser.scala b/cassandra/src/main/scala/org/apache/zeppelin/cassandra/ParagraphParser.scala
index 514161e..b0314d5 100644
--- a/cassandra/src/main/scala/org/apache/zeppelin/cassandra/ParagraphParser.scala
+++ b/cassandra/src/main/scala/org/apache/zeppelin/cassandra/ParagraphParser.scala
@@ -18,7 +18,6 @@ package org.apache.zeppelin.cassandra
 
 import com.datastax.oss.driver.api.core.{ConsistencyLevel, DefaultConsistencyLevel}
 import com.datastax.oss.driver.api.core.cql.{BatchType, DefaultBatchType}
-import org.apache.zeppelin.cassandra.CassandraInterpreter._
 import org.apache.zeppelin.interpreter.InterpreterException
 
 import scala.util.matching.Regex
@@ -144,7 +143,7 @@ class ParagraphParser extends RegexParsers{
   //Query parameters
   def consistency: Parser[Consistency] = """\s*@consistency.+""".r ^^ {x => extractConsistency(x.trim)}
   def serialConsistency: Parser[SerialConsistency] = """\s*@serialConsistency.+""".r ^^ {x => extractSerialConsistency(x.trim)}
-  def timestamp: Parser[Timestamp] = """\s*@timestamp.+""".r ^^ {x => extractTimestamp(x.trim)}
+  def timestamp: Parser[Timestamp] = """\s*@timestamp[^F].+""".r ^^ {x => extractTimestamp(x.trim)}
   def fetchSize: Parser[FetchSize] = """\s*@fetchSize.+""".r ^^ {x => extractFetchSize(x.trim)}
   def requestTimeOut: Parser[RequestTimeOut] = """\s*@requestTimeOut.+""".r ^^ {x => extractRequestTimeOut(x.trim)}
 
@@ -157,7 +156,6 @@ class ParagraphParser extends RegexParsers{
   def removePrepare(): Parser[RemovePrepareStm] = """\s*@remove_prepare.+""".r ^^ {x => extractRemovePreparedStatement(x.trim)}
   def bind(): Parser[BoundStm] = """\s*@bind.+""".r ^^ {x => extractBoundStatement(x.trim)}
 
-
   //Meta data
   private def describeCluster: Parser[DescribeClusterCmd] = """(?i)\s*(?:DESCRIBE|DESC)\s+CLUSTER.*""".r ^^ extractDescribeClusterCmd
   private def describeKeyspaces: Parser[DescribeKeyspacesCmd] = """(?i)\s*(?:DESCRIBE|DESC)\s+KEYSPACES.*""".r ^^ extractDescribeKeyspacesCmd
@@ -190,14 +188,14 @@ class ParagraphParser extends RegexParsers{
   }
 
   def queries:Parser[List[AnyBlock]] = rep(singleLineComment | multiLineComment | consistency | serialConsistency |
-    timestamp | fetchSize | requestTimeOut | removePrepare | prepare | bind | batch | describeCluster |
+    timestamp | fetchSize | requestTimeOut | removePrepare | prepare | bind | batch |
     describeKeyspace | describeKeyspaces |
     describeTable | describeTables |
     describeType | describeTypes |
     describeFunction | describeFunctions |
     describeAggregate | describeAggregates |
     describeMaterializedView | describeMaterializedViews |
-    helpCommand | createFunctionStatement | genericStatement)
+    describeCluster | helpCommand | createFunctionStatement | genericStatement)
 
   def extractConsistency(text: String): Consistency = {
     text match {
diff --git a/cassandra/src/test/java/org/apache/zeppelin/cassandra/CassandraInterpreterTest.java b/cassandra/src/test/java/org/apache/zeppelin/cassandra/CassandraInterpreterTest.java
index 8dac12f..ffdb7d9 100644
--- a/cassandra/src/test/java/org/apache/zeppelin/cassandra/CassandraInterpreterTest.java
+++ b/cassandra/src/test/java/org/apache/zeppelin/cassandra/CassandraInterpreterTest.java
@@ -17,8 +17,8 @@
 package org.apache.zeppelin.cassandra;
 
 import com.datastax.oss.driver.api.core.CqlSession;
-import com.datastax.oss.driver.internal.core.type.codec.TimestampCodec;
 import org.apache.zeppelin.display.AngularObjectRegistry;
+import org.apache.zeppelin.display.GUI;
 import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterResult;
@@ -27,20 +27,15 @@ import org.cassandraunit.CQLDataLoader;
 import org.cassandraunit.dataset.cql.ClassPathCQLDataSet;
 import org.cassandraunit.utils.EmbeddedCassandraServerHelper;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Answers;
-import org.mockito.Mock;
-import org.mockito.runners.MockitoJUnitRunner;
 
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.time.Instant;
+import java.util.Map;
 import java.util.Properties;
 
 import static org.apache.zeppelin.cassandra.CassandraInterpreter.CASSANDRA_CLUSTER_NAME;
@@ -65,17 +60,16 @@ import static org.apache.zeppelin.cassandra.CassandraInterpreter.CASSANDRA_SOCKE
 import static org.apache.zeppelin.cassandra.CassandraInterpreter.CASSANDRA_SOCKET_TCP_NO_DELAY;
 import static org.apache.zeppelin.cassandra.CassandraInterpreter.CASSANDRA_SPECULATIVE_EXECUTION_POLICY;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.Mockito.when;
 
-@RunWith(MockitoJUnitRunner.class)
 public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestCase {
   private static final String ARTISTS_TABLE = "zeppelin.artists";
   private static final int DEFAULT_UNIT_TEST_PORT = 9142;
 
   private static volatile CassandraInterpreter interpreter;
 
-  @Mock(answer = Answers.RETURNS_DEEP_STUBS)
-  private InterpreterContext intrContext;
+  private InterpreterContext intrContext = InterpreterContext.builder()
+          .setParagraphTitle("Paragraph1")
+          .build();
 
   @BeforeClass
   public static synchronized void setUp() throws IOException, InterruptedException {
@@ -121,11 +115,6 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     interpreter.close();
   }
 
-  @Before
-  public void prepareContext() {
-    when(intrContext.getParagraphTitle()).thenReturn("Paragraph1");
-  }
-
   @Test
   public void should_create_cluster_and_session_upon_call_to_open() throws Exception {
     assertThat(interpreter.session).isNotNull();
@@ -145,20 +134,20 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData()).isEqualTo("name\tborn\tcountry\tdied\tgender\t" +
         "styles\ttype\n" +
-        "'Bogdan Raczynski'\t'1977-01-01'\t'Poland'\tnull\t'Male'\t" +
-        "['Dance','Electro']\t'Person'\n" +
-        "'Krishna Das'\t'1947-05-31'\t'USA'\tnull\t'Male'\t['Unknown']\t'Person'\n" +
-        "'Sheryl Crow'\t'1962-02-11'\t'USA'\tnull\t'Female'\t" +
-        "['Classic','Rock','Country','Blues','Pop','Folk']\t'Person'\n" +
-        "'Doof'\t'1968-08-31'\t'United Kingdom'\tnull\tnull\t['Unknown']\t'Person'\n" +
-        "'House of Large Sizes'\t'1986-01-01'\t'USA'\t'2003'\tnull\t['Unknown']\t'Group'\n" +
-        "'Fanfarlo'\t'2006-01-01'\t'United Kingdom'\tnull\tnull\t" +
-        "['Rock','Indie','Pop','Classic']\t'Group'\n" +
-        "'Jeff Beck'\t'1944-06-24'\t'United Kingdom'\tnull\t'Male'\t" +
-        "['Rock','Pop','Classic']\t'Person'\n" +
-        "'Los Paranoias'\tnull\t'Unknown'\tnull\tnull\t['Unknown']\tnull\n" +
-        "'…And You Will Know Us by the Trail of Dead'\t'1994-01-01'\t'USA'\tnull\tnull\t" +
-        "['Rock','Pop','Classic']\t'Group'\n");
+        "Bogdan Raczynski\t1977-01-01\tPoland\tnull\tMale\t" +
+        "[Dance, Electro]\tPerson\n" +
+        "Krishna Das\t1947-05-31\tUSA\tnull\tMale\t[Unknown]\tPerson\n" +
+        "Sheryl Crow\t1962-02-11\tUSA\tnull\tFemale\t" +
+        "[Classic, Rock, Country, Blues, Pop, Folk]\tPerson\n" +
+        "Doof\t1968-08-31\tUnited Kingdom\tnull\tnull\t[Unknown]\tPerson\n" +
+        "House of Large Sizes\t1986-01-01\tUSA\t2003\tnull\t[Unknown]\tGroup\n" +
+        "Fanfarlo\t2006-01-01\tUnited Kingdom\tnull\tnull\t" +
+        "[Rock, Indie, Pop, Classic]\tGroup\n" +
+        "Jeff Beck\t1944-06-24\tUnited Kingdom\tnull\tMale\t" +
+        "[Rock, Pop, Classic]\tPerson\n" +
+        "Los Paranoias\tnull\tUnknown\tnull\tnull\t[Unknown]\tnull\n" +
+        "…And You Will Know Us by the Trail of Dead\t1994-01-01\tUSA\tnull\tnull\t" +
+        "[Rock, Pop, Classic]\tGroup\n");
   }
 
   @Test
@@ -174,9 +163,60 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData())
         .isEqualTo("name\tborn\tcountry\tdied\tgender\tstyles\ttype\n" +
-        "'Bogdan Raczynski'\t'1977-01-01'\t'Poland'\tnull\t'Male'\t" +
-        "['Dance','Electro']\t'Person'\n" +
-        "'Krishna Das'\t'1947-05-31'\t'USA'\tnull\t'Male'\t['Unknown']\t'Person'\n");
+        "Bogdan Raczynski\t1977-01-01\tPoland\tnull\tMale\t" +
+        "[Dance, Electro]\tPerson\n" +
+        "Krishna Das\t1947-05-31\tUSA\tnull\tMale\t[Unknown]\tPerson\n");
+  }
+
+  @Test
+  public void should_interpret_select_statement_with_cql_format() throws Exception {
+    //When
+    intrContext.getLocalProperties().put("outputFormat", "cql");
+    final InterpreterResult actual = interpreter.interpret(
+            "SELECT * FROM " + ARTISTS_TABLE + " LIMIT 2;", intrContext);
+    intrContext.getLocalProperties().remove("outputFormat");
+
+    //Then
+    assertThat(actual).isNotNull();
+    assertThat(actual.code()).isEqualTo(Code.SUCCESS);
+    assertThat(actual.message().get(0).getData())
+            .isEqualTo("name\tborn\tcountry\tdied\tgender\tstyles\ttype\n" +
+                    "'Bogdan Raczynski'\t'1977-01-01'\t'Poland'\tnull\t'Male'\t" +
+                    "['Dance','Electro']\t'Person'\n" +
+                    "'Krishna Das'\t'1947-05-31'\t'USA'\tnull\t'Male'\t['Unknown']\t'Person'\n");
+  }
+
+  @Test
+  public void should_interpret_select_statement_with_formatting_options() throws Exception {
+    //When
+    Map<String, String> props = intrContext.getLocalProperties();
+    props.put("outputFormat", "human");
+    props.put("locale", "de_DE");
+    props.put("floatPrecision", "2");
+    props.put("doublePrecision", "4");
+    props.put("timeFormat", "hh:mma");
+    props.put("timestampFormat", "MM/dd/yy HH:mm");
+    props.put("dateFormat", "E, d MMM yy");
+    props.put("timezone", "Etc/GMT+2");
+    String query =
+            "select date, time, timestamp, double, float, tuple, udt from zeppelin.test_format;";
+    final InterpreterResult actual = interpreter.interpret(query, intrContext);
+    props.remove("outputFormat");
+    props.remove("locale");
+    props.remove("floatPrecision");
+    props.remove("doublePrecision");
+    props.remove("timeFormat");
+    props.remove("timestampFormat");
+    props.remove("dateFormat");
+    props.remove("timezone");
+
+    //Then
+    assertThat(actual).isNotNull();
+    assertThat(actual.code()).isEqualTo(Code.SUCCESS);
+    String expected = "date\ttime\ttimestamp\tdouble\tfloat\ttuple\tudt\n" +
+            "Di, 29 Jan 19\t04:05AM\t06/16/20 21:59\t10,0153\t20,03\t(1, text, 10)\t" +
+            "{id: 1, t: text, lst: [1, 2, 3]}\n";
+    assertThat(actual.message().get(0).getData()).isEqualTo(expected);
   }
 
   @Test
@@ -202,9 +242,9 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     //Then
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData()).isEqualTo("title\tartist\tyear\n" +
-            "'The Impossible Dream EP'\t'Carter the Unstoppable Sex Machine'\t1992\n" +
-            "'The Way You Are'\t'Tears for Fears'\t1983\n" +
-            "'Primitive'\t'Soulfly'\t2003\n");
+            "The Impossible Dream EP\tCarter the Unstoppable Sex Machine\t1992\n" +
+            "The Way You Are\tTears for Fears\t1983\n" +
+            "Primitive\tSoulfly\t2003\n");
   }
     
   @Test
@@ -323,7 +363,7 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     //Then
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData()).isEqualTo("key\tval\n" +
-            "'myKey'\t'myValue'\n");
+            "myKey\tmyValue\n");
   }
 
   @Test
@@ -345,15 +385,15 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData()).isEqualTo(
             "login\taddresses\tage\tdeceased\tfirstname\tlast_update\tlastname\tlocation\n" +
-                    "'jdoe'\t" +
-                    "{street_number:3,street_name:'Beverly Hills Bld',zip_code:90209," +
-                    "country:'USA',extra_info:['Right on the hills','Next to the post box']," +
-                    "phone_numbers:{'home':2016778524,'office':2015790847}}\tnull\t" +
+                    "jdoe\t" +
+                    "{street_number: 3, street_name: Beverly Hills Bld, zip_code: 90209, " +
+                    "country: USA, extra_info: [Right on the hills, Next to the post box], " +
+                    "phone_numbers: {home: 2016778524, office: 2015790847}}\tnull\t" +
                     "null\t" +
-                    "'John'\t" +
+                    "John\t" +
                     "null\t" +
-                    "'DOE'\t" +
-                    "('USA',90209,'Beverly Hills')\n");
+                    "DOE\t" +
+                    "(USA, 90209, Beverly Hills)\n");
   }
 
   @Test
@@ -375,9 +415,14 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
   public void should_extract_variable_from_statement() throws Exception {
     //Given
     AngularObjectRegistry angularObjectRegistry = new AngularObjectRegistry("cassandra", null);
-    when(intrContext.getAngularObjectRegistry()).thenReturn(angularObjectRegistry);
-    when(intrContext.getGui().input("login", "hsue")).thenReturn("hsue");
-    when(intrContext.getGui().input("age", "27")).thenReturn("27");
+    GUI gui = new GUI();
+    gui.textbox("login", "hsue");
+    gui.textbox("age", "27");
+    InterpreterContext intrContext = InterpreterContext.builder()
+            .setParagraphTitle("Paragraph1")
+            .setAngularObjectRegistry(angularObjectRegistry)
+            .setGUI(gui)
+            .build();
 
     String queries = "@prepare[test_insert_with_variable]=" +
             "INSERT INTO zeppelin.users(login,firstname,lastname,age) VALUES(?,?,?,?)\n" +
@@ -389,7 +434,7 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     //Then
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData()).isEqualTo("firstname\tlastname\tage\n" +
-            "'Helen'\t'SUE'\t27\n");
+            "Helen\tSUE\t27\n");
   }
 
   @Test
@@ -421,9 +466,9 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     //Then
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData()).isEqualTo("name\tcountry\tstyles\n" +
-            "'Bogdan Raczynski'\t'Poland'\t['Dance','Electro']\n" +
-            "'Krishna Das'\t'USA'\t['Unknown']\n" +
-            "'Sheryl Crow'\t'USA'\t['Classic','Rock','Country','Blues','Pop','Folk']\n");
+            "Bogdan Raczynski\tPoland\t[Dance, Electro]\n" +
+            "Krishna Das\tUSA\t[Unknown]\n" +
+            "Sheryl Crow\tUSA\t[Classic, Rock, Country, Blues, Pop, Folk]\n");
   }
 
   @Test
@@ -438,9 +483,7 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
 
     //Then
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
-    Instant tm = Instant.parse("2015-07-30T12:00:01Z");
-    assertThat(actual.message().get(0).getData()).contains("last_update\n" +
-            new TimestampCodec().format(tm));
+    assertThat(actual.message().get(0).getData()).contains("last_update\n2015-07-30T12:00:01.000Z");
   }
 
   @Test
@@ -456,7 +499,7 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     //Then
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData()).isEqualTo("firstname\tlastname\n" +
-            "null\t'NULL'\n");
+            "null\tNULL\n");
   }
 
   @Test
@@ -472,7 +515,7 @@ public class CassandraInterpreterTest { //extends AbstractCassandraUnit4CQLTestC
     //Then
     assertThat(actual.code()).isEqualTo(Code.SUCCESS);
     assertThat(actual.message().get(0).getData()).isEqualTo("login\tdeceased\n" +
-            "'bind_bool'\tfalse\n");
+            "bind_bool\tfalse\n");
   }
 
   @Test
diff --git a/cassandra/src/test/java/org/apache/zeppelin/cassandra/InterpreterLogicTest.java b/cassandra/src/test/java/org/apache/zeppelin/cassandra/InterpreterLogicTest.java
index 0ae802e..4617b54 100644
--- a/cassandra/src/test/java/org/apache/zeppelin/cassandra/InterpreterLogicTest.java
+++ b/cassandra/src/test/java/org/apache/zeppelin/cassandra/InterpreterLogicTest.java
@@ -55,6 +55,7 @@ import java.time.temporal.ChronoField;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Properties;
 
 import scala.Option;
 
@@ -82,7 +83,7 @@ public class InterpreterLogicTest {
   @Mock
   private CqlSession session;
 
-  final InterpreterLogic helper = new InterpreterLogic(session);
+  final InterpreterLogic helper = new InterpreterLogic(session, new Properties());
 
   @Captor
   ArgumentCaptor<ParamOption[]> optionsCaptor;
diff --git a/cassandra/src/test/resources/prepare_all.cql b/cassandra/src/test/resources/prepare_all.cql
index ec40293..3e2d25d 100644
--- a/cassandra/src/test/resources/prepare_all.cql
+++ b/cassandra/src/test/resources/prepare_all.cql
@@ -29,9 +29,12 @@ CREATE TYPE IF NOT EXISTS zeppelin.address(
  country text,
  extra_info list<text>,
  phone_numbers map<text,bigint>
-
 );
 
+TRUNCATE zeppelin.ts;
+
+TRUNCATE zeppelin.prepared;
+
 
 CREATE TABLE IF NOT EXISTS zeppelin.users(
  login text,
@@ -45,6 +48,60 @@ CREATE TABLE IF NOT EXISTS zeppelin.users(
  PRIMARY KEY(login)
 );
 
+TRUNCATE zeppelin.users;
+
+-- for formatting test
+create type if not exists zeppelin.tudt (
+  id int,
+  t text,
+  lst frozen<list<int>>
+);
+
+create table if not exists zeppelin.test_format (
+  id int primary key,
+  text text,
+  date date,
+  timestamp timestamp,
+  time time,
+  uuid uuid,
+  tuuid timeuuid,
+  m1 map<int, text>,
+  m2 frozen<map<text,int>>,
+  l1 list<int>,
+  l2 frozen<list<int>>,
+  s1 set<int>,
+  s2 frozen<set<text>>,
+  udt zeppelin.tudt,
+  ludt1 list<frozen<zeppelin.tudt>>,
+  ludt2 frozen<list<zeppelin.tudt>>,
+  blob blob,
+  bool boolean,
+  dec decimal,
+  double double,
+  float float,
+  bigint bigint,
+  smallint smallint,
+  tinyint tinyint,
+  varint varint,
+  ascii ascii,
+  tuple tuple<int, text, float>,
+  varchar varchar,
+  nullval text
+);
+
+TRUNCATE zeppelin.test_format;
+
+insert into zeppelin.test_format(id, text, date, timestamp, time, uuid, tuuid, m1, m2, l1, l2, s1, s2,
+  udt, ludt1, ludt2, blob, bool, dec, double, float, bigint, smallint, tinyint, varint, ascii, tuple, varchar)
+  values (1, 'text', '2019-01-29', '2020-06-16T23:59:59.123Z', '04:05:00.234', 123e4567-e89b-12d3-a456-426655440000,
+  now(), {1:'m1', 2:'m2'}, {'m1':1, 'm2':2}, [1,2,3], [1,2,3], {1,2,3}, {'1','2','3'},
+  {id: 1, t: 'text', lst: [1,2,3]}, [{id: 1, t: 'text', lst: [1,2,3]}, {id: 2, t: 'text2'}],
+  [{id: 1, t: 'text', lst: [1,2,3]}, {id: 2, t: 'text2'}], bigintAsBlob(1024), true, 123562352352.0,
+  10.0153423453425634653463466346543, 20.0303443, 123562352352, 10000, 10,
+  124325345634643900999999, 'ascii', (1, 'text', 10.0), 'varchar русский'); 
+
+-- other tests
+
 CREATE KEYSPACE IF NOT EXISTS samples
     WITH REPLICATION = {'class':'SimpleStrategy', 'replication_factor':1}
     AND DURABLE_WRITES=false;
@@ -88,8 +145,6 @@ CREATE INDEX IF NOT EXISTS idx2 ON live_data.complex_table(indexed2);
 CREATE INDEX IF NOT EXISTS keys_map_idx ON live_data.complex_table(KEYS(key_indexed_map));
 
 
-
-
 CREATE TABLE IF NOT EXISTS live_data.sensor_data (
     sensor_id uuid,
     month int,
@@ -126,8 +181,3 @@ INSERT INTO zeppelin.artists(name,born,country,died,gender,styles,type) VALUES('
 INSERT INTO zeppelin.artists(name,born,country,died,gender,styles,type) VALUES('Los Paranoias',null,'Unknown',null,null,['Unknown'],null);
 INSERT INTO zeppelin.artists(name,born,country,died,gender,styles,type) VALUES('…And You Will Know Us by the Trail of Dead','1994-01-01','USA',null,null,['Rock', 'Pop', 'Classic'],'Group');
 
-TRUNCATE zeppelin.ts;
-
-TRUNCATE zeppelin.prepared;
-
-TRUNCATE zeppelin.users;
diff --git a/cassandra/src/test/resources/scalate/DescribeTables.html b/cassandra/src/test/resources/scalate/DescribeTables.html
index b6bfcba..a017fda 100644
--- a/cassandra/src/test/resources/scalate/DescribeTables.html
+++ b/cassandra/src/test/resources/scalate/DescribeTables.html
@@ -459,6 +459,9 @@
                       <td>prepared</td>
                     </tr>
                     <tr class="text-primary">
+                      <td>test_format</td>
+                    </tr>
+                    <tr class="text-primary">
                       <td>ts</td>
                     </tr>
                     <tr class="text-primary">
diff --git a/cassandra/src/test/resources/scalate/DescribeTypes.html b/cassandra/src/test/resources/scalate/DescribeTypes.html
index d13af85..2e161ae 100644
--- a/cassandra/src/test/resources/scalate/DescribeTypes.html
+++ b/cassandra/src/test/resources/scalate/DescribeTypes.html
@@ -166,6 +166,9 @@
                     <tr class="text-warning">
                       <td>address</td>
                     </tr>
+                    <tr class="text-warning">
+                      <td>tudt</td>
+                    </tr>
                   </tbody>
                 </table>
               </div>
diff --git a/cassandra/src/test/scala/org/apache/zeppelin/cassandra/CqlFormatterTest.scala b/cassandra/src/test/scala/org/apache/zeppelin/cassandra/CqlFormatterTest.scala
new file mode 100644
index 0000000..9bb3866
--- /dev/null
+++ b/cassandra/src/test/scala/org/apache/zeppelin/cassandra/CqlFormatterTest.scala
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zeppelin.cassandra
+
+import java.net.InetAddress
+import java.nio.ByteBuffer
+import java.time.{Instant, LocalDate, LocalTime, ZoneId}
+import java.util.Properties
+
+import com.datastax.oss.driver.api.core.`type`.{DataTypes, UserDefinedType}
+import com.datastax.oss.driver.api.core.`type`.codec.registry.CodecRegistry
+import com.datastax.oss.driver.api.core.detach.AttachmentPoint
+import io.netty.buffer.ByteBufUtil
+import org.scalatest.{BeforeAndAfterEach, FlatSpec, Matchers}
+
+import scala.collection.JavaConverters._
+
+class CqlFormatterTest extends FlatSpec
+  with BeforeAndAfterEach
+  with Matchers {
+
+  val longVal: java.lang.Long = java.lang.Long.valueOf(12345678901L)
+  val floatVal: java.lang.Float = java.lang.Float.valueOf(123.456789f)
+  val intVal: java.lang.Integer = Integer.valueOf(123456)
+  val doubleVal: java.lang.Double = java.lang.Double.valueOf(123.4567890123456789)
+  val dateVal: LocalDate = LocalDate.of(2020, 6, 16)
+  val timeVal: LocalTime = LocalTime.of(23, 59, 59, 123400000)
+  val timestampVal:Instant = Instant.parse("2020-06-16T23:59:59.123456Z")
+  val byteBufVal: Array[Byte] = Array[Byte](192.toByte, 168.toByte, 0, 10)
+
+  "CqlFormatter" should "format objects with default settings" in {
+    val defaultFormatter = new CqlFormatter()
+
+    // please note that it doesn't format UdtValue & TupleType, as it's quite hard to emulate
+    // they are checked in the integration tests
+    defaultFormatter.formatHuman(intVal) should be("123456")
+    defaultFormatter.formatHuman(longVal) should be("12345678901")
+    defaultFormatter.formatHuman(floatVal) should be("123.45679")
+    defaultFormatter.formatHuman(doubleVal) should be("123.456789012346")
+    defaultFormatter.formatHuman("just text") should be("just text")
+    defaultFormatter.formatHuman(java.lang.Boolean.TRUE) should be("true")
+    defaultFormatter.formatHuman(List(1,2,3).asJava) should be("[1, 2, 3]")
+    defaultFormatter.formatHuman(List("1","2","3").asJava) should be("[1, 2, 3]")
+    defaultFormatter.formatHuman(Set(1, 2, 3).asJava) should be("{3, 1, 2}")
+    defaultFormatter.formatHuman(Set("1", "2", "3").asJava) should be("{3, 1, 2}")
+    defaultFormatter.formatHuman(Map(1 -> 1, 2 -> 2, 3 -> 3).asJava) should be("{1: 1, 2: 2, 3: 3}")
+    defaultFormatter.formatHuman(Map(1 -> "1", 2 -> "2", 3 -> "3").asJava) should be("{1: 1, 2: 2, 3: 3}")
+    defaultFormatter.formatHuman(dateVal) should be("2020-06-16")
+    defaultFormatter.formatHuman(timeVal) should be("23:59:59.123")
+    defaultFormatter.formatHuman(timestampVal) should be("2020-06-16T23:59:59.123Z")
+    defaultFormatter.formatHuman(Map(1 -> timestampVal).asJava) should be("{1: 2020-06-16T23:59:59.123Z}")
+    defaultFormatter.formatHuman(InetAddress.getLoopbackAddress) should be("127.0.0.1")
+    defaultFormatter.formatHuman(InetAddress.getByAddress(byteBufVal)) should be("192.168.0.10")
+    defaultFormatter.formatHuman(ByteBuffer.wrap(byteBufVal)) should be("0xc0a8000a")
+  }
+
+  "CqlFormatter" should "format objects with copied settings" in {
+    val copiedFormatter = new CqlFormatter()
+      .copy(floatPrecision = 2, doublePrecision = 4, timeZoneId = "Etc/GMT+2",
+        timeFormat = "hh:mma", dateFormat = "E, d MMM yy", localeStr = "en_US")
+    copiedFormatter.formatHuman(floatVal) should be("123.46")
+    copiedFormatter.formatHuman(doubleVal) should be("123.4568")
+    copiedFormatter.formatHuman(timestampVal) should be("2020-06-16T21:59:59.123-02:00")
+    copiedFormatter.formatHuman(timeVal) should be("11:59PM")
+    copiedFormatter.formatHuman(dateVal) should be("Tue, 16 Jun 20")
+  }
+
+  "CqlFormatter" should "format objects with settings from property object" in {
+    val properties = new Properties()
+    properties.setProperty(CassandraInterpreter.CASSANDRA_FORMAT_FLOAT_PRECISION, "2")
+    properties.setProperty(CassandraInterpreter.CASSANDRA_FORMAT_DOUBLE_PRECISION, "4")
+    properties.setProperty(CassandraInterpreter.CASSANDRA_FORMAT_TIME, "hh:mma")
+    properties.setProperty(CassandraInterpreter.CASSANDRA_FORMAT_DATE, "E, d MMM yy")
+    properties.setProperty(CassandraInterpreter.CASSANDRA_FORMAT_TIMEZONE, "Etc/GMT+2")
+
+    val copiedFormatter = new CqlFormatter(properties)
+    copiedFormatter.formatHuman(floatVal) should be("123.46")
+    copiedFormatter.formatHuman(doubleVal) should be("123.4568")
+    copiedFormatter.formatHuman(timestampVal) should be("2020-06-16T21:59:59.123-02:00")
+    copiedFormatter.formatHuman(timeVal) should be("11:59PM")
+    copiedFormatter.formatHuman(dateVal) should be("Tue, 16 Jun 20")
+  }
+
+  "CqlFormatter" should "format objects with locale" in {
+    val copiedFormatter = new CqlFormatter()
+      .copy(floatPrecision = 2, doublePrecision = 4, localeStr = "de_DE")
+    copiedFormatter.formatHuman(floatVal) should be("123,46")
+    copiedFormatter.formatHuman(doubleVal) should be("123,4568")
+    copiedFormatter.formatHuman(timestampVal) should be("2020-06-16T23:59:59.123Z")
+  }
+
+  "CqlFormatter" should "format objects using CQL syntax" in {
+    val cqlFormatter = new CqlFormatter().copy(outputFormat = "cql")
+    val codecRegistry = CodecRegistry.DEFAULT
+
+    // please note that it doesn't format UdtValue & TupleType, as it's quite hard to emulate
+    // they are checked in the integration tests
+    cqlFormatter.format(intVal, codecRegistry.codecFor(DataTypes.INT)) should be("123456")
+    cqlFormatter.format(longVal, codecRegistry.codecFor(DataTypes.BIGINT)) should be("12345678901")
+    cqlFormatter.format(floatVal, codecRegistry.codecFor(DataTypes.FLOAT)) should be("123.45679")
+    cqlFormatter.format(doubleVal, codecRegistry.codecFor(DataTypes.DOUBLE)) should be("123.45678901234568")
+    cqlFormatter.format("just text", codecRegistry.codecFor(DataTypes.TEXT)) should be("'just text'")
+    cqlFormatter.format(java.lang.Boolean.TRUE, codecRegistry.codecFor(DataTypes.BOOLEAN)) should be("true")
+    cqlFormatter.format(dateVal,
+      codecRegistry.codecFor(DataTypes.DATE)) should be("'2020-06-16'")
+    cqlFormatter.format(timeVal,
+      codecRegistry.codecFor(DataTypes.TIME)) should be("'23:59:59.123400000'")
+    cqlFormatter.format(InetAddress.getLoopbackAddress,
+      codecRegistry.codecFor(DataTypes.INET)) should be("'127.0.0.1'")
+    cqlFormatter.format(InetAddress.getByAddress(byteBufVal),
+      codecRegistry.codecFor(DataTypes.INET)) should be("'192.168.0.10'")
+    cqlFormatter.format(ByteBuffer.wrap(byteBufVal),
+      codecRegistry.codecFor(DataTypes.BLOB)) should be("0xc0a8000a")
+  }
+
+}
diff --git a/docs/interpreter/cassandra.md b/docs/interpreter/cassandra.md
index 1837c53..b4613e2 100644
--- a/docs/interpreter/cassandra.md
+++ b/docs/interpreter/cassandra.md
@@ -316,7 +316,7 @@ There is a drop-down menu on the top left corner to expand objects details. On t
   ![Describe Schema]({{BASE_PATH}}/assets/themes/zeppelin/img/docs-img/cassandra-DescribeSchema.png)
 </center>
 
-## Runtime Parameters
+## Runtime Execution Parameters
 
 Sometimes you want to be able to pass runtime query parameters to your statements.
 
@@ -424,6 +424,99 @@ Some remarks about query parameters:
 > 3. each query parameter applies to **all CQL statements** in the same paragraph, unless you override the option using plain CQL text (like forcing timestamp with the `USING` clause)
 > 4. the order of each query parameter with regard to CQL statement does not matter
 
+## Runtime Formatting Parameters
+
+Sometimes you want to be able to format output of your statement. Cassandra interpreter allows to specify different parameters as local properties of the paragraph. Below is the list of all formatting parameters:
+
+<center>
+ <table class="table-configuration">
+   <tr>
+     <th>Parameter</th>
+     <th>Syntax</th>
+     <th>Description</th>
+   </tr>
+   <tr>
+     <td nowrap>Output Format</td>
+     <td><strong>outputFormat=<em>value</em></strong></td>
+     <td>Controls, should we output data as CQL literals, or in human-readable form. Possible values: <strong>cql, human</strong> (default: <strong>human</strong></td>
+   </tr>
+   <tr>
+     <td nowrap>Locale</td>
+     <td><strong>locale=<em>value</em></strong></td>
+     <td>Locale for formatting of numbers & time-related values. Could be any locale supported by JVM (default: <strong>en_US</strong>)</td>
+   </tr>
+   <tr>
+     <td nowrap>Timezone</td>
+     <td><strong>timezone=<em>value</em></strong></td>
+     <td>Timezone for formatting of time-related values. Could be any timezone supported by JVM (default: <strong>UTC</strong>)</td>
+   </tr>
+   <tr>
+     <td nowrap>Float precision</td>
+     <td><strong>floatPrecision=<em>value</em></strong></td>
+     <td>Precision when formatting <tt>float</tt> values. Any positive integer value</td>
+   </tr>
+   <tr>
+     <td nowrap>Double precision</td>
+     <td><strong>doublePrecision=<em>value</em></strong></td>
+     <td>Precision when formatting <tt>double</tt> values. Any positive integer value</td>
+   </tr>
+   <tr>
+     <td nowrap>Timestamp Format</td>
+     <td><strong>timestampFormat=<em>value</em></strong></td>
+     <td>Format string for <tt>timestamp</tt> values. Should be valid <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</a> pattern</td>
+   </tr>
+   <tr>
+     <td nowrap>Time Format</td>
+     <td><strong>timeFormat=<em>value</em></strong></td>
+     <td>Format string for <tt>time</tt> values. Should be valid <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</a> pattern</td>
+   </tr>
+   <tr>
+     <td nowrap>Date Format</td>
+     <td><strong>dateFormat=<em>value</em></strong></td>
+     <td>Format string for <tt>date</tt> values. Should be valid <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</a> pattern</td>
+   </tr>
+ </table>
+</center>
+
+
+Some examples:
+
+```sql
+create table if not exists zep.test_format (
+  id int primary key,
+  text text,
+  date date,
+  timestamp timestamp,
+  time time,
+  double double,
+  float float
+);
+
+insert into zep.test_format(id, text, date, timestamp, time, double, float)
+  values (1, 'text', '2019-01-29', '2020-06-16T23:59:59.123Z', '04:05:00.234', 
+  10.0153423453425634653463466346543, 20.0303443); 
+```
+  
+```
+%cassandra(outputFormat=human, locale=de_DE, floatPrecision=2, doublePrecision=4, timeFormat=hh:mma, timestampFormat=MM/dd/yy HH:mm, dateFormat="E, d MMM yy", timezone=Etc/GMT+2)
+select id, double, float, text, date, time, timestamp from zep.test_format;
+```
+
+will output data formatted according to settings, including German locale:
+
+```
+id  double   float  text  date           time     timestamp
+1   10,0153	 20,03  text  Di, 29 Jan 19  04:05AM  06/16/20 21:59
+```
+
+while with `outputFormat=cql`, data is formatted as CQL literals:
+
+```
+id double              float       text    date        time                  timestamp
+1  10.015342345342564  20.030344  'text'  '2019-01-29' '04:05:00.234000000'  '2020-06-17T01:59:59.123+02:00'
+```
+
+
 ## Support for Prepared Statements
 
 For performance reason, it is better to prepare statements before-hand and reuse them later by providing bound values.
@@ -767,18 +860,57 @@ Below are the configuration parameters and their default values.
      </td>
      <td></td>
    </tr>
+   <tr>
+     <td>`cassandra.format.output`</td>
+     <td>Output format for data - strict CQL (`cql`), or human-readable (`human`)</td>
+     <td>`human`</td>
+   </tr>
+   <tr>
+     <td>`cassandra.format.locale`</td>
+     <td>Which locale to use for output (any locale supported by JVM could be specified)</td>
+     <td>`en_US`</td>
+   </tr>
+   <tr>
+     <td>`cassandra.format.timezone`</td>
+     <td>For which timezone format time/date-related types (any timezone supported by JVM could be specified)</td>
+     <td>`UTC`</td>
+   </tr>
+   <tr>
+     <td>`cassandra.format.timestamp`</td>
+     <td>Format string for `timestamp` columns (any valid <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</a> pattern could be used)</td>
+     <td>`yyyy-MM-dd'T'HH:mm:ss.SSSXXX`</td>
+   </tr>
+   <tr>
+     <td>`cassandra.format.time`</td>
+     <td>Format string for `time` columns (any valid <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</a> pattern could be used)</td>
+     <td>`HH:mm:ss.SSS`</td>
+   </tr>
+   <tr>
+     <td>`cassandra.format.date`</td>
+     <td>Format string for `date` columns (any valid <a href="https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html">DateTimeFormatter</a> pattern could be used)</td>
+     <td>`yyyy-MM-dd`</td>
+   </tr>
+   <tr>
+     <td>`cassandra.format.float_precision`</td>
+     <td>Precision when formatting values of `float` type</td>
+     <td>`5`</td>
+   </tr>
+   <tr>
+     <td>`cassandra.format.double_precision`</td>
+     <td>Precision when formatting values of `double` type</td>
+     <td>`12`</td>
+   </tr>
  </table>
 
 ## Change Log
 
-**3.2** _(Zeppelin {{ site.ZEPPELIN_VERSION }})_ :
+**4.0** _(Zeppelin {{ site.ZEPPELIN_VERSION }})_ :
 
-* Refactor to use unified Java driver 4.5
-  ([ZEPPELIN-4378](https://issues.apache.org/jira/browse/ZEPPELIN-4378):
-  * changes in configuration were necessary, as new driver has different architecture, and
-  configuration options;
-  * interpreter got support for DSE-specific data types, and other extensions;
-  * support for `@retryPolicy` is removed, as only single retry policy is shipped with driver.
+* Refactor to use unified Java driver 4.5 ([ZEPPELIN-4378](https://issues.apache.org/jira/browse/ZEPPELIN-4378):
+  * changes in configuration were necessary, as new driver has different architecture, and configuration options
+  * interpreter got support for DSE-specific data types, and other extensions
+  * support for `@retryPolicy` is removed, as only single retry policy is shipped with driver
+  * added support for formatting options, both interpreter & cell level
 
 **3.1** _(Zeppelin {{ site.ZEPPELIN_VERSION }})_ :