You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by bo...@apache.org on 2019/11/08 09:33:26 UTC

[impala] 03/06: IMPALA-8704: ISO:SQL:2016 datetime patterns - Milestone 2

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

boroknagyz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 443ca2e8b7f77f70011e351a87207037871b9a74
Author: Gabor Kaszab <ga...@cloudera.com>
AuthorDate: Wed Sep 18 13:01:47 2019 +0200

    IMPALA-8704: ISO:SQL:2016 datetime patterns - Milestone 2
    
    This patch adds additional datetime format tokens on top of what was
    introduced with Milestone 1 (IMPALA-8703).
    
    The tokens introduced:
    - Free text token: Surrounded by double quotes, a free text section
      can be given in the format where the same text is expected in the
      input without the surrounding double quotes.
    - FX modifier: This modifier has to be given at the beginning of the
      format and is valid for the whole format.
      In a string to datetime conversion this forces strict separator
      matching and expects all the tokens in the input to have the same
      length as their maximum length. E.g. A month has to be of length 2
      prefixed by zero if needed.
      This is the default in a datetime to string conversion.
    - FM modifier: This modifier affects only the following token and
      overrides the FX modifier.
      In a string to datetime conversion when using this modifier the
      length of the token in the input can be shorter than the max length
      of that token type if followed by a separator. E.g. 1-digit month,
      less than 4-digit year, etc. This is the default behaviour in a
      string to datetime conversion.
      In a datetime to string conversion tokens with this modifier aren't
      padded to the maximum length. Example output: "2010-1-9".
    
    Change-Id: I30d2f6656054371476aaa8bd0d51f572b9369855
    Reviewed-on: http://gerrit.cloudera.org:8080/14291
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Attila Jeges <at...@cloudera.com>
---
 be/src/runtime/date-parse-util.cc                  |  19 +-
 be/src/runtime/datetime-iso-sql-format-parser.cc   |  77 +++-
 be/src/runtime/datetime-iso-sql-format-parser.h    |  13 +-
 .../runtime/datetime-iso-sql-format-tokenizer.cc   | 101 +++++-
 be/src/runtime/datetime-iso-sql-format-tokenizer.h |  36 +-
 be/src/runtime/datetime-parser-common.cc           |  46 +++
 be/src/runtime/datetime-parser-common.h            |  32 +-
 be/src/runtime/timestamp-parse-util.cc             |  23 +-
 tests/query_test/test_cast_with_format.py          | 400 +++++++++++++++++++++
 9 files changed, 699 insertions(+), 48 deletions(-)

diff --git a/be/src/runtime/date-parse-util.cc b/be/src/runtime/date-parse-util.cc
index 008c1b9..ef3b041 100644
--- a/be/src/runtime/date-parse-util.cc
+++ b/be/src/runtime/date-parse-util.cc
@@ -142,8 +142,6 @@ string DateParser::Format(const DateTimeFormatContext& dt_ctx, const DateValue&
   result.reserve(dt_ctx.fmt_out_len);
   for (const DateTimeFormatToken& tok: dt_ctx.toks) {
     int32_t num_val = -1;
-    const char* str_val = nullptr;
-    int str_val_len = 0;
     switch (tok.type) {
       case YEAR:
       case ROUND_YEAR: {
@@ -156,8 +154,7 @@ string DateParser::Format(const DateTimeFormatContext& dt_ctx, const DateValue&
       }
       case MONTH_IN_YEAR: num_val = month; break;
       case MONTH_IN_YEAR_SLT: {
-        str_val = TimestampFunctions::MONTH_ARRAY[month - 1].c_str();
-        str_val_len = 3;
+        result.append(TimestampFunctions::MONTH_ARRAY[month - 1]);
         break;
       }
       case DAY_IN_MONTH: num_val = day; break;
@@ -166,19 +163,21 @@ string DateParser::Format(const DateTimeFormatContext& dt_ctx, const DateValue&
         break;
       }
       case SEPARATOR: {
-        str_val = tok.val;
-        str_val_len = tok.len;
+        result.append(tok.val, tok.len);
+        break;
+      }
+      case TEXT: {
+        result.append(FormatTextToken(tok));
         break;
       }
       default: DCHECK(false) << "Unknown date format token";
     }
     if (num_val > -1) {
       string tmp_str = std::to_string(num_val);
-      if (tmp_str.length() < tok.len) tmp_str.insert(0, tok.len - tmp_str.length(), '0');
+      if (!tok.fm_modifier && tmp_str.length() < tok.len) {
+        tmp_str.insert(0, tok.len - tmp_str.length(), '0');
+      }
       result.append(tmp_str);
-    } else {
-      DCHECK(str_val != nullptr && str_val_len > 0);
-      result.append(str_val, str_val_len);
     }
   }
   return result;
diff --git a/be/src/runtime/datetime-iso-sql-format-parser.cc b/be/src/runtime/datetime-iso-sql-format-parser.cc
index d0c0547..5a39573 100644
--- a/be/src/runtime/datetime-iso-sql-format-parser.cc
+++ b/be/src/runtime/datetime-iso-sql-format-parser.cc
@@ -17,7 +17,6 @@
 
 #include "runtime/datetime-iso-sql-format-parser.h"
 
-#include <boost/algorithm/string.hpp>
 #include <cmath>
 
 #include "common/names.h"
@@ -40,20 +39,47 @@ bool IsoSqlFormatParser::ParseDateTime(const char* input_str, int input_len,
   const char* current_pos = input_str;
   const char* end_pos = input_str + input_len;
   for (int i = 0; i < dt_ctx.toks.size(); ++i) {
-    if (current_pos >= end_pos) return false;
     const DateTimeFormatToken* tok = &dt_ctx.toks[i];
+    if (current_pos >= end_pos) {
+      // Accept empty text tokens at the end of the format.
+      if (tok->type == TEXT && tok->len == 0) continue;
+      return false;
+    }
+
     if (tok->type == SEPARATOR) {
-      bool res = ProcessSeparators(&current_pos, end_pos, dt_ctx, &i);
-      if (!res || current_pos == end_pos) return res;
-      DCHECK(i < dt_ctx.toks.size());
-      // Next token, following the separator sequence.
-      tok = &dt_ctx.toks[i];
+      if (dt_ctx.fx_modifier) {
+        DCHECK(tok->len == 1);
+        if (*current_pos != *tok->val) return false;
+        ++current_pos;
+        continue;
+      } else {
+        bool res = ProcessSeparatorSequence(&current_pos, end_pos, dt_ctx, &i);
+        if (!res || current_pos >= end_pos) return res;
+        DCHECK(i < dt_ctx.toks.size());
+        // Next token, following the separator sequence.
+        tok = &dt_ctx.toks[i];
+      }
+    }
+
+    if (tok->type == TEXT) {
+      const char* format_it = tok->val;
+      const char* format_end = tok->val + tok->len;
+      while (format_it < format_end && current_pos < end_pos) {
+        char format_char_to_compare = GetNextCharFromTextToken(&format_it, tok);
+        if (format_char_to_compare != *current_pos) return false;
+        ++format_it;
+        ++current_pos;
+      }
+      if (format_it < format_end) return false;
+      continue;
     }
 
     const char* token_end_pos = FindEndOfToken(current_pos, end_pos - current_pos, *tok);
     if (token_end_pos == nullptr) return false;
     int token_len = token_end_pos - current_pos;
 
+    if (dt_ctx.fx_modifier && !tok->fm_modifier && token_len != tok->len) return false;
+
     switch(tok->type) {
       case YEAR: {
         if (!ParseAndValidate(current_pos, token_len, 0, 9999, &result->year)) {
@@ -181,11 +207,36 @@ bool IsoSqlFormatParser::ParseDateTime(const char* input_str, int input_len,
   return true;
 }
 
-bool IsoSqlFormatParser::ProcessSeparators(const char** current_pos, const char* end_pos,
-    const DateTimeFormatContext& dt_ctx, int* current_tok_idx) {
+char IsoSqlFormatParser::GetNextCharFromTextToken(const char** format,
+    const DateTimeFormatToken* tok) {
+  DCHECK(format != nullptr && *format != nullptr);
+  DCHECK(tok != nullptr);
+  DCHECK(tok->val <= *format && *format < tok->val + tok->len);
+  if (**format != '\\') return **format;
+  const char* format_end = tok->val + tok->len;
+  // Take care of the double escaped quotes.
+  if (tok->is_double_escaped && format_end - *format >= 4 &&
+      (strncmp(*format, "\\\\\\\"", 4) == 0 || strncmp(*format, "\\\\\\'", 4) == 0)) {
+    *format += 3;
+    return **format;
+  }
+  // Skip the escaping backslash.
+  ++(*format);
+  switch (**format) {
+    case 'b': return '\b';
+    case 'n': return '\n';
+    case 'r': return '\r';
+    case 't': return '\t';
+  }
+  return **format;
+}
+
+bool IsoSqlFormatParser::ProcessSeparatorSequence(const char** current_pos,
+    const char* end_pos, const DateTimeFormatContext& dt_ctx, int* current_tok_idx) {
   DCHECK(current_pos != nullptr && *current_pos != nullptr);
   DCHECK(end_pos != nullptr);
   DCHECK(current_tok_idx != nullptr && *current_tok_idx < dt_ctx.toks.size());
+  DCHECK(dt_ctx.toks[*current_tok_idx].type == SEPARATOR);
   if (!IsoSqlFormatTokenizer::IsSeparator(**current_pos)) return false;
   // Advance to the end of the separator sequence.
   ++(*current_pos);
@@ -201,6 +252,14 @@ bool IsoSqlFormatParser::ProcessSeparators(const char** current_pos, const char*
 
   // If we reached the end of input or the end of token sequence, we can return.
   if (*current_pos >= end_pos || *current_tok_idx >= dt_ctx.toks.size()) {
+    // Skip trailing empty text tokens in format.
+    if (*current_pos >= end_pos && *current_tok_idx < dt_ctx.toks.size()) {
+      while (*current_tok_idx < dt_ctx.toks.size() &&
+          dt_ctx.toks[*current_tok_idx].type == TEXT &&
+          dt_ctx.toks[*current_tok_idx].len == 0) {
+        ++(*current_tok_idx);
+      }
+    }
     return (*current_pos >= end_pos && *current_tok_idx >= dt_ctx.toks.size());
   }
 
diff --git a/be/src/runtime/datetime-iso-sql-format-parser.h b/be/src/runtime/datetime-iso-sql-format-parser.h
index b0454d1..aed93d7 100644
--- a/be/src/runtime/datetime-iso-sql-format-parser.h
+++ b/be/src/runtime/datetime-iso-sql-format-parser.h
@@ -34,6 +34,7 @@ class IsoSqlFormatParser {
 public:
   /// Given a list of format tokens in 'dt_ctx' runs through 'input_str' and parses it
   /// into 'result'. Return value indicates if the parsing was successful.
+  /// The caller has to make sure that 'dt_ctx.fmt' is a null-terminated string.
   static bool ParseDateTime(const char* input_str, int input_len,
       const DateTimeFormatContext& dt_ctx, DateTimeParseResult* result)
       WARN_UNUSED_RESULT;
@@ -71,8 +72,18 @@ private:
   /// that '**tok' is of type SEPARATOR. Returns false if '**current_pos' is not a
   /// separator or if either the input ends while having remaining items in 'dt_ctx->toks'
   /// or the other way around.
-  static bool ProcessSeparators(const char** current_pos, const char* end_pos,
+  static bool ProcessSeparatorSequence(const char** current_pos, const char* end_pos,
       const DateTimeFormatContext& dt_ctx, int* dt_ctx_it);
+
+  // Gets the next character starting from '*format' that can be used for input
+  // matching. Takes care of the escaping backslashes regardless if the text token inside
+  // the format is itself double escped or not. Returns the next character in a form
+  // expected in the input. If '*format' points at the beginning of an escape sequence,
+  // '*format' is moved to the last character of the escape sequence. Otherwise,
+  // '*format' is not changed. E.g. If the text token is "\"abc" then this returns '"'
+  // after skipping the backslash and moves '*format' to '"'.
+  static char GetNextCharFromTextToken(const char** format,
+      const DateTimeFormatToken* tok);
 };
 
 }
diff --git a/be/src/runtime/datetime-iso-sql-format-tokenizer.cc b/be/src/runtime/datetime-iso-sql-format-tokenizer.cc
index 07c8cc8..1f477b0 100644
--- a/be/src/runtime/datetime-iso-sql-format-tokenizer.cc
+++ b/be/src/runtime/datetime-iso-sql-format-tokenizer.cc
@@ -60,7 +60,9 @@ const unordered_map<string, IsoSqlFormatTokenizer::TokenItem>
   {"TZH", IsoSqlFormatTokenizer::TokenItem(TIMEZONE_HOUR, false, true)},
   {"TZM", IsoSqlFormatTokenizer::TokenItem(TIMEZONE_MIN, false, true)},
   {"T", IsoSqlFormatTokenizer::TokenItem(ISO8601_TIME_INDICATOR, false, true)},
-  {"Z", IsoSqlFormatTokenizer::TokenItem(ISO8601_ZULU_INDICATOR, false, true)}
+  {"Z", IsoSqlFormatTokenizer::TokenItem(ISO8601_ZULU_INDICATOR, false, true)},
+  {"FM", IsoSqlFormatTokenizer::TokenItem(FM_MODIFIER, false, false)},
+  {"FX", IsoSqlFormatTokenizer::TokenItem(FX_MODIFIER, false, false)}
 });
 
 const unordered_map<string, int> IsoSqlFormatTokenizer::SPECIAL_LENGTHS({
@@ -73,14 +75,15 @@ const unsigned IsoSqlFormatTokenizer::MAX_TOKEN_SIZE = 5;
 const int IsoSqlFormatTokenizer::MAX_FORMAT_LENGTH = 100;
 
 FormatTokenizationResult IsoSqlFormatTokenizer::Tokenize() {
-  DCHECK(dt_ctx_ != NULL);
-  DCHECK(dt_ctx_->fmt != NULL);
+  DCHECK(dt_ctx_ != nullptr);
+  DCHECK(dt_ctx_->fmt != nullptr);
   DCHECK(dt_ctx_->fmt_len > 0);
   DCHECK(dt_ctx_->toks.size() == 0);
   DCHECK(used_tokens_.empty());
   if (dt_ctx_->fmt_len > MAX_FORMAT_LENGTH) return TOO_LONG_FORMAT_ERROR;
   const char* str_end = dt_ctx_->fmt + dt_ctx_->fmt_len;
   const char* current_pos = dt_ctx_->fmt;
+  ProcessFXModifier(&current_pos);
   while (current_pos < str_end) {
     ProcessSeparators(&current_pos);
     if (current_pos == str_end) break;
@@ -104,10 +107,14 @@ void IsoSqlFormatTokenizer::ProcessSeparators(const char** current_pos) {
 
 FormatTokenizationResult IsoSqlFormatTokenizer::ProcessNextToken(
     const char** current_pos) {
-  DCHECK(*current_pos != nullptr);
+  DCHECK(current_pos != nullptr && *current_pos != nullptr);
   const char* str_begin = dt_ctx_->fmt;
   const char* str_end = dt_ctx_->fmt + dt_ctx_->fmt_len;
-  DCHECK(*current_pos < str_end);
+  DCHECK(str_begin <= *current_pos && *current_pos < str_end);
+  if (IsStartOfTextToken(*current_pos)) {
+    fm_modifier_active_ = false;
+    return ProcessTextToken(current_pos, str_begin, str_end);
+  }
   unsigned curr_token_size =
       min(MAX_TOKEN_SIZE, static_cast<unsigned>(str_end - *current_pos));
   string token_to_probe(*current_pos, curr_token_size);
@@ -116,10 +123,21 @@ FormatTokenizationResult IsoSqlFormatTokenizer::ProcessNextToken(
     token_to_probe.resize(curr_token_size);
     const auto token = VALID_TOKENS.find(token_to_probe);
     if (token != VALID_TOKENS.end()) {
+      if (token->second.type == FX_MODIFIER) return MISPLACED_FX_MODIFIER_ERROR;
+      if (token->second.type == FM_MODIFIER) {
+        fm_modifier_active_ = true;
+        *current_pos += curr_token_size;
+        return SUCCESS;
+      }
       if (cast_mode_ == PARSE && IsUsedToken(token_to_probe)) return DUPLICATE_FORMAT;
       if (!accept_time_toks_ && token->second.time_token) return DATE_WITH_TIME_ERROR;
-      dt_ctx_->toks.push_back(DateTimeFormatToken(token->second.type,
+      DateTimeFormatToken format_token(DateTimeFormatToken(token->second.type,
           *current_pos - str_begin, GetMaxTokenLength(token->first), *current_pos));
+      if (fm_modifier_active_) {
+        fm_modifier_active_ = false;
+        format_token.fm_modifier = true;
+      }
+      dt_ctx_->toks.push_back(format_token);
       dt_ctx_->has_date_toks |= token->second.date_token;
       dt_ctx_->has_time_toks |= token->second.time_token;
       used_tokens_.insert(token_to_probe);
@@ -147,6 +165,9 @@ bool IsoSqlFormatTokenizer::IsMeridiemIndicatorProvided() const {
 }
 
 FormatTokenizationResult IsoSqlFormatTokenizer::CheckIncompatibilities() const {
+  DCHECK(dt_ctx_ != nullptr);
+  if (!dt_ctx_->has_date_toks && !dt_ctx_->has_time_toks) return NO_DATETIME_TOKENS_ERROR;
+
   if (cast_mode_ == FORMAT) {
     if (IsUsedToken("TZH") || IsUsedToken("TZM")) {
       return TIMEZONE_OFFSET_NOT_ALLOWED_ERROR;
@@ -211,5 +232,73 @@ bool IsoSqlFormatTokenizer::IsSeparator(char c) {
       c == '\'' || c == ';' ;
 }
 
+void IsoSqlFormatTokenizer::ProcessFXModifier(const char** current_pos) {
+  DCHECK(current_pos != nullptr && *current_pos != nullptr);
+  DCHECK(*current_pos == dt_ctx_->fmt);
+  if (strncmp(*current_pos, "FX", 2) == 0) {
+    dt_ctx_->fx_modifier = true;
+    *current_pos += 2;
+  }
+}
+
+bool IsoSqlFormatTokenizer::IsStartOfTextToken(const char* current_pos) {
+  DCHECK(current_pos != nullptr);
+  if (*current_pos == '"' || strncmp(current_pos, "\\\"", 2) == 0) return true;
+  return false;
+}
+
+FormatTokenizationResult IsoSqlFormatTokenizer::ProcessTextToken(
+    const char** current_pos, const char* str_begin, const char* str_end) {
+  DCHECK(dt_ctx_ != nullptr);
+  DCHECK(current_pos != nullptr);
+  DCHECK(*current_pos != nullptr);
+  DCHECK(str_begin != nullptr);
+  DCHECK(str_end != nullptr);
+  DCHECK(str_begin <= *current_pos && *current_pos < str_end);
+  bool is_escaped = (**current_pos == '\\');
+  // Exclude opening quotation marks from the stored token.
+  *current_pos += is_escaped ? 2 : 1;
+  const char* start_of_content = *current_pos;
+  *current_pos = FindEndOfTextToken(*current_pos, str_end, is_escaped);
+  if (*current_pos == nullptr) return TEXT_TOKEN_NOT_CLOSED;
+  DateTimeFormatToken token(TEXT, start_of_content - str_begin,
+      *current_pos - start_of_content - 1 - is_escaped, start_of_content);
+  token.is_double_escaped = is_escaped;
+  dt_ctx_->toks.push_back(token);
+  return SUCCESS;
+}
+
+const char* IsoSqlFormatTokenizer::FindEndOfTextToken(const char* str_start,
+    const char* str_end, bool is_escaped) {
+  DCHECK(str_start != nullptr);
+  const char* current_pos = str_start;
+  while (current_pos < str_end) {
+    // Skip escaped double quotes.
+    if (!is_escaped && strncmp(current_pos, "\\\"", 2) == 0) {
+      current_pos += 2;
+      continue;
+    }
+    if (is_escaped && strncmp(current_pos, "\\\\\\\"", 4) == 0) {
+      current_pos += 4;
+      continue;
+    }
+    // Skip escaped backslash.
+    if (strncmp(current_pos, "\\\\", 2) == 0) {
+      current_pos += 2;
+      continue;
+    }
+    if (!is_escaped && *current_pos == '"') {
+      ++current_pos;
+      return current_pos;
+    }
+    if (is_escaped && strncmp(current_pos, "\\\"", 2) == 0) {
+      current_pos += 2;
+      return current_pos;
+    }
+    ++current_pos;
+  }
+  return nullptr;
+}
+
 }
 }
diff --git a/be/src/runtime/datetime-iso-sql-format-tokenizer.h b/be/src/runtime/datetime-iso-sql-format-tokenizer.h
index 25e2783..2c3b838 100644
--- a/be/src/runtime/datetime-iso-sql-format-tokenizer.h
+++ b/be/src/runtime/datetime-iso-sql-format-tokenizer.h
@@ -37,7 +37,7 @@ class IsoSqlFormatTokenizer {
 public:
   IsoSqlFormatTokenizer(DateTimeFormatContext* dt_ctx, CastDirection cast_mode,
       bool time_toks) : dt_ctx_(dt_ctx), cast_mode_(cast_mode),
-      accept_time_toks_(time_toks) {}
+      accept_time_toks_(time_toks), fm_modifier_active_(false) {}
 
   void Reset(DateTimeFormatContext* dt_ctx, CastDirection cast_mode, bool time_toks) {
     dt_ctx_ = dt_ctx;
@@ -88,6 +88,8 @@ private:
 
   /// The context that is used for the input of the parsing. It is also populated during
   /// the parsing process. Not owned by this class.
+  /// Note, that 'dt_ctx_->fmt' is a null-terminated string so it's safe to use string
+  /// functions that make this assumption.
   DateTimeFormatContext* dt_ctx_;
 
   /// Decides whether this is a 'datetime to string' or a 'string to datetime' cast.
@@ -95,6 +97,10 @@ private:
 
   bool accept_time_toks_;
 
+  /// True when the FM modifier has to be applied to the following non-separator token.
+  /// It is set back to false once applied on a token.
+  bool fm_modifier_active_;
+
   /// Iterates through all the consecutive separator characters from a given pointer
   /// 'current' and saves them to 'dt_ctx_'.
   void ProcessSeparators(const char** current);
@@ -114,10 +120,32 @@ private:
   bool IsMeridiemIndicatorProvided() const;
 
   /// Checks if the end product of the parsing contains format tokens that collide with
-  /// each other like YYYY and RR. Note, this is mostly relevant for a 'string to
-  /// datetime' conversion. The only restriction for 'datetime to string' path is that
-  /// timezone tokens are not allowed since Impala doesn't store timezone information.
+  /// each other like YYYY and RR.
   FormatTokenizationResult CheckIncompatibilities() const;
+
+  /// Checks if '*current_pos' points to an FX modifier and advances '*current_pos' after
+  /// the FX modifier. Sets 'dt_ctx_->fx_modifier' to true if '*current_pos' points to an
+  /// FX modifier. Call this when '*current_pos' points to the first character of the
+  /// format string.
+  void ProcessFXModifier(const char** current_pos);
+
+  /// Returns true if 'current_pos' points to either a double quote or to a backslash
+  /// that is followed by a double quote.
+  static bool IsStartOfTextToken(const char* current_pos);
+
+  /// Finds the end of the text token and saves metadata about the token into 'dt_ctx_'.
+  /// This has to be called when '*current_pos' points to the beginning of a text token
+  /// or in other words if IsStartOfTextToken(*current_pos) is true. As a side effect
+  /// 'current_pos' is advanced right after the closing double qoute of the text token.
+  FormatTokenizationResult ProcessTextToken(const char** current_pos,
+      const char* str_begin, const char* str_end);
+
+  // Starting from 'str_start' finds the closing quotation mark of the text token even
+  // if it's escaped. 'str_start' has to point to the first character of the text token
+  // right after the opening quote. Returns a pointer pointing right after the closing
+  // double quote of the text token or nullptr if the text token is unclosed.
+  static const char* FindEndOfTextToken(const char* str_start, const char* str_end,
+      bool is_escaped);
 };
 
 }
diff --git a/be/src/runtime/datetime-parser-common.cc b/be/src/runtime/datetime-parser-common.cc
index 514e0d5..bf68425 100644
--- a/be/src/runtime/datetime-parser-common.cc
+++ b/be/src/runtime/datetime-parser-common.cc
@@ -52,6 +52,7 @@ void DateTimeFormatContext::Reset(const char* fmt, int fmt_len) {
   this->fmt_out_len = fmt_len;
   this->has_date_toks = false;
   this->has_time_toks = false;
+  this->fx_modifier = false;
   this->toks.clear();
   this->century_break_ptime = boost::posix_time::not_a_date_time;
   this->current_time = nullptr;
@@ -108,6 +109,15 @@ void ReportBadFormat(FunctionContext* context, FormatTokenizationResult error_ty
       case CONFLICTING_FRACTIONAL_SECOND_TOKENS_ERROR:
         ss << "PARSE ERROR: Multiple fractional second token provided.";
         break;
+      case TEXT_TOKEN_NOT_CLOSED:
+        ss << "PARSE ERROR: Missing closing quotation mark.";
+        break;
+      case NO_DATETIME_TOKENS_ERROR:
+        ss << "PARSE ERROR: No datetime tokens provided.";
+        break;
+      case MISPLACED_FX_MODIFIER_ERROR:
+        ss << "PARSE ERROR: FX modifier should be at the beginning of the format string.";
+        break;
       default:
         const StringValue& fmt = StringValue::FromStringVal(format);
         ss << "Bad date/time conversion format: " << fmt.DebugString();
@@ -176,5 +186,41 @@ bool GetMonthAndDayFromDaysSinceJan1(int year, int days_since_jan1, int* month,
   return (*day >= 1 && *day <= 31);
 }
 
+string FormatTextToken(const DateTimeFormatToken& tok) {
+  DCHECK(tok.type == TEXT);
+  string result;
+  result.reserve(tok.len);
+  for (const char* text_it = tok.val; text_it < tok.val + tok.len; ++text_it) {
+    if (*text_it != '\\') {
+      result.append(text_it, 1);
+      continue;
+    }
+    if (tok.is_double_escaped && strncmp(text_it, "\\\\\\\"", 4) == 0) {
+      result.append("\"");
+      text_it += 3;
+    } else if (!tok.is_double_escaped && strncmp(text_it, "\\\"", 2) == 0) {
+      result.append("\"");
+      ++text_it;
+    } else if (strncmp(text_it, "\\\\", 2) == 0) {
+      result.append("\\");
+      ++text_it;
+    } else if (strncmp(text_it, "\\b", 2) == 0) {
+      result.append("\b");
+      ++text_it;
+    } else if (strncmp(text_it, "\\n", 2) == 0) {
+      result.append("\n");
+      ++text_it;
+    } else if (strncmp(text_it, "\\r", 2) == 0) {
+      result.append("\r");
+      ++text_it;
+    } else if (strncmp(text_it, "\\t", 2) == 0) {
+      result.append("\t");
+      ++text_it;
+    }
+  }
+  return result;
+}
+
+
 }
 }
diff --git a/be/src/runtime/datetime-parser-common.h b/be/src/runtime/datetime-parser-common.h
index 3093c76..70c0eb4 100644
--- a/be/src/runtime/datetime-parser-common.h
+++ b/be/src/runtime/datetime-parser-common.h
@@ -69,7 +69,10 @@ enum FormatTokenizationResult {
   TIMEZONE_OFFSET_NOT_ALLOWED_ERROR,
   MISSING_TZH_TOKEN_ERROR,
   DATE_WITH_TIME_ERROR,
-  CONFLICTING_FRACTIONAL_SECOND_TOKENS_ERROR
+  CONFLICTING_FRACTIONAL_SECOND_TOKENS_ERROR,
+  TEXT_TOKEN_NOT_CLOSED,
+  NO_DATETIME_TOKENS_ERROR,
+  MISPLACED_FX_MODIFIER_ERROR
 };
 
 /// Holds all the token types that serve as building blocks for datetime format patterns.
@@ -93,7 +96,10 @@ enum DateTimeFormatTokenType {
   TIMEZONE_MIN,
   MERIDIEM_INDICATOR,
   ISO8601_TIME_INDICATOR,
-  ISO8601_ZULU_INDICATOR
+  ISO8601_ZULU_INDICATOR,
+  TEXT,
+  FM_MODIFIER,
+  FX_MODIFIER
 };
 
 /// Indicates whether the cast is a 'datetime to string' or a 'string to datetime' cast.
@@ -121,12 +127,17 @@ struct DateTimeFormatToken {
   int len;
   /// A pointer to the beginning of this token in the format string.
   const char* val;
+  /// True if FM modifier is active for this token. This overrides the FX modifier active
+  /// for the whole format.
+  bool fm_modifier;
+
+  /// True if this is a text token that is surrounded by escaped double quotes making the
+  /// content of the token double-escaped.
+  bool is_double_escaped;
 
   DateTimeFormatToken(DateTimeFormatTokenType type, int pos, int len, const char* val)
-    : type(type),
-      pos(pos),
-      len(len),
-      val(val) {
+    : type(type), pos(pos), len(len), val(val), fm_modifier(false),
+      is_double_escaped(false) {
   }
 };
 
@@ -151,6 +162,10 @@ struct DateTimeFormatContext {
   std::vector<DateTimeFormatToken> toks;
   bool has_date_toks;
   bool has_time_toks;
+
+  /// True if the format contains an FX modifier effective for all the tokens.
+  bool fx_modifier;
+
   /// Used for casting with SimpleDateFormat to handle rounded year. Make sure you call
   /// SetCenturyBreakAndCurrentTime() before using this member.
   boost::posix_time::ptime century_break_ptime;
@@ -226,5 +241,10 @@ int GetDayInYear(int year, int month, int day_in_month);
 /// 365. Returns true on success.
 bool GetMonthAndDayFromDaysSinceJan1(int year, int days_since_jan1, int* month, int* day)
     WARN_UNUSED_RESULT;
+
+// Receives a text token and gives its string formatted representation. This is used in
+// a string to datetime conversion path.
+std::string FormatTextToken(const DateTimeFormatToken& tok);
+
 }
 }
diff --git a/be/src/runtime/timestamp-parse-util.cc b/be/src/runtime/timestamp-parse-util.cc
index d46b586..b8d80e1 100644
--- a/be/src/runtime/timestamp-parse-util.cc
+++ b/be/src/runtime/timestamp-parse-util.cc
@@ -224,8 +224,6 @@ string TimestampParser::Format(const DateTimeFormatContext& dt_ctx, const date&
   result.reserve(dt_ctx.fmt_out_len);
   for (const DateTimeFormatToken& tok: dt_ctx.toks) {
     int32_t num_val = -1;
-    const char* str_val = NULL;
-    int str_val_len = 0;
     switch (tok.type) {
       case YEAR:
       case ROUND_YEAR: {
@@ -238,8 +236,7 @@ string TimestampParser::Format(const DateTimeFormatContext& dt_ctx, const date&
       }
       case MONTH_IN_YEAR: num_val = d.month().as_number(); break;
       case MONTH_IN_YEAR_SLT: {
-        str_val = d.month().as_short_string();
-        str_val_len = 3;
+        result.append(d.month().as_short_string(), 3);
         break;
       }
       case DAY_IN_MONTH: num_val = d.day(); break;
@@ -259,8 +256,8 @@ string TimestampParser::Format(const DateTimeFormatContext& dt_ctx, const date&
         if (t.hours() >= 12) {
           indicator_txt = (tok.len == 2) ? &PM : &PM_LONG;
         }
-        str_val_len = tok.len;
-        str_val = (isupper(*tok.val)) ? indicator_txt->first : indicator_txt->second;
+        result.append((isupper(*tok.val)) ? indicator_txt->first : indicator_txt->second,
+            tok.len);
         break;
       }
       case MINUTE_IN_HOUR: num_val = t.minutes(); break;
@@ -277,22 +274,24 @@ string TimestampParser::Format(const DateTimeFormatContext& dt_ctx, const date&
       case SEPARATOR:
       case ISO8601_TIME_INDICATOR:
       case ISO8601_ZULU_INDICATOR: {
-        str_val = tok.val;
-        str_val_len = tok.len;
+        result.append(tok.val, tok.len);
         break;
       }
       case TZ_OFFSET: {
         break;
       }
+      case TEXT: {
+        result.append(FormatTextToken(tok));
+        break;
+      }
       default: DCHECK(false) << "Unknown date/time format token";
     }
     if (num_val > -1) {
       string tmp_str = std::to_string(num_val);
-      if (tmp_str.length() < tok.len) tmp_str.insert(0, tok.len - tmp_str.length(), '0');
+      if (!tok.fm_modifier && tmp_str.length() < tok.len) {
+        tmp_str.insert(0, tok.len - tmp_str.length(), '0');
+      }
       result.append(tmp_str);
-    } else {
-      DCHECK(str_val != nullptr && str_val_len > 0);
-      result.append(str_val, str_val_len);
     }
   }
   return result;
diff --git a/tests/query_test/test_cast_with_format.py b/tests/query_test/test_cast_with_format.py
index 8287f09..3dee880 100644
--- a/tests/query_test/test_cast_with_format.py
+++ b/tests/query_test/test_cast_with_format.py
@@ -575,6 +575,345 @@ class TestCastWithFormat(ImpalaTestSuite):
         "FORMAT 'YYYY-MM-DD HH24:MI:SS TZH:TZM')")
     assert result.data == ["NULL"]
 
+  def test_text_token(self):
+    # Parse ISO:8601 tokens using the text token.
+    result = self.client.execute(r'''select cast('1985-11-19T01:02:03Z' as timestamp '''
+        r'''format 'YYYY-MM-DD"T"HH24:MI:SS"Z"')''')
+    assert result.data == ["1985-11-19 01:02:03"]
+
+    # Free text at the end of the input
+    result = self.client.execute(r'''select cast('1985-11-19text' as timestamp '''
+        r'''format 'YYYY-MM-DD"text"')''')
+    assert result.data == ["1985-11-19 00:00:00"]
+
+    # Free text at the beginning of the input
+    result = self.client.execute(r'''select cast('19801985-11-20' as timestamp '''
+        r'''format '"1980"YYYY-MM-DD')''')
+    assert result.data == ["1985-11-20 00:00:00"]
+
+    # Empty text in format
+    result = self.client.execute(r'''select cast('1985-11-21' as timestamp '''
+        r'''format '""YYYY""-""MM""-""DD""')''')
+    assert result.data == ["1985-11-21 00:00:00"]
+
+    result = self.client.execute(r'''select cast('1985-11-22' as timestamp '''
+        r'''format 'YYYY-MM-DD""""""')''')
+    assert result.data == ["1985-11-22 00:00:00"]
+
+    result = self.client.execute(r'''select cast('1985-12-09-' as timestamp '''
+        r'''format 'YYYY-MM-DD-""')''')
+    assert result.data == ["1985-12-09 00:00:00"]
+
+    result = self.client.execute(r'''select cast('1985-12-10-' as date '''
+        r'''format 'FXYYYY-MM-DD-""')''')
+    assert result.data == ["1985-12-10"]
+
+    result = self.client.execute(r'''select cast('1985-11-23' as timestamp '''
+        r'''format 'YYYY-MM-DD""""""HH24')''')
+    assert result.data == ["NULL"]
+
+    # Text in input doesn't match with the text in format
+    result = self.client.execute(r'''select cast('1985-11-24Z01:02:03Z' as timestamp '''
+        r'''format 'YYYY-MM-DD"T"HH24:MI:SS"Z"')''')
+    assert result.data == ["NULL"]
+
+    result = self.client.execute(r'''select cast('1985-11-24T01:02:04T' as timestamp '''
+        r'''format 'YYYY-MM-DD"T"HH24:MI:SS"Z"')''')
+    assert result.data == ["NULL"]
+
+    result = self.client.execute(r'''select cast('1985-11-2401:02:05Z' as timestamp '''
+        r'''format 'YYYY-MM-DD"T"HH24:MI:SS"Z"')''')
+    assert result.data == ["NULL"]
+
+    result = self.client.execute(r'''select cast('1985-11-24T01:02:06' as timestamp '''
+        r'''format 'YYYY-MM-DD"T"HH24:MI:SS"Z"')''')
+    assert result.data == ["NULL"]
+
+    result = self.client.execute(r'''select cast('1985-11-24 01:02:07te' as timestamp '''
+        r'''format 'YYYY-MM-DD HH24:MI:SS"text"')''')
+    assert result.data == ["NULL"]
+
+    result = self.client.execute(r'''select cast('1985-11-24 01:02:08text' as '''
+        r'''timestamp format 'YYYY-MM-DD HH24:MI:SS"te"')''')
+    assert result.data == ["NULL"]
+
+    # Consecutive text tokens
+    result = self.client.execute(r'''select cast('1985-11text1text2-25' as timestamp '''
+        r'''format 'YYYY-MM"text1""text2"-DD')''')
+    assert result.data == ["1985-11-25 00:00:00"]
+
+    # Separators in text token
+    result = self.client.execute(r'''select cast("1985-11 -'./,:-25" as date '''
+        r'''format "YYYY-MM\" -'./,:\"-DD")''')
+    assert result.data == ["1985-11-25"]
+
+    # Known limitation: If a text token containing separator characters at the beginning
+    # is right after a separator token sequence then parsing can't find where to stop when
+    # parsing the consecutive separators. Use FX modifier in this case for strict
+    # matching.
+    result = self.client.execute(r'''select cast("1986-11'25" as date '''
+        r'''format "YYYY-MM\"'\"DD")''')
+    assert result.data == ["1986-11-25"]
+
+    result = self.client.execute(r'''select cast("1986-11-'25" as timestamp '''
+        r'''format "YYYY-MM-\"'\"DD")''')
+    assert result.data == ["NULL"]
+
+    result = self.client.execute(r'''select cast("1986-10-'25" as timestamp '''
+        r'''format "FXYYYY-MM-\"'\"DD")''')
+    assert result.data == ["1986-10-25 00:00:00"]
+
+    # Escaped quotation mark is in the text token.
+    result = self.client.execute(r'''select cast('1985-11a"b26' as timestamp '''
+        r'''format 'YYYY-MM"a\"b"DD')''')
+    assert result.data == ["1985-11-26 00:00:00"]
+
+    # Format part is surrounded by double quotes so the quotes indicating the start and
+    # end of the text token has to be escaped.
+    result = self.client.execute('select cast("year: 1985, month: 11, day: 27" as date'
+        r''' format "\"year: \"YYYY\", month: \"MM\", day: \"DD")''')
+    assert result.data == ["1985-11-27"]
+
+    # Scenario when there is an escaped double quote inside a text token that is itself
+    # surrounded by escaped double quotes.
+    result = self.client.execute(r'''select cast("1985 some \"text 11-28" as date'''
+        r''' format "YYYY\" some \\\"text \"MM-DD")''')
+    assert result.data == ["1985-11-28"]
+
+    # When format is surrounded by single quotes and there is a single quote inside the
+    # text token that has to be escaped.
+    result = self.client.execute(r'''select cast("1985 some 'text 11-29" as date'''
+        r''' format 'YYYY" some \'text "MM-DD')''')
+    assert result.data == ["1985-11-29"]
+    result = self.client.execute(r'''select cast("1985 some 'text 11-29" as timestamp'''
+        r''' format 'YYYY" some \'text "MM-DD')''')
+    assert result.data == ["1985-11-29 00:00:00"]
+
+    # Datetime to string path: Simple text token.
+    result = self.client.execute(r'''select cast(cast("1985-11-30" as date) as string '''
+        r'''format "YYYY-\"text\"MM-DD")''')
+    assert result.data == ["1985-text11-30"]
+
+    # Datetime to string path: Consecutive text tokens.
+    result = self.client.execute(r'''select cast(cast("1985-12-01" as date) as string '''
+        r'''format "YYYY-\"text1\"\"text2\"MM-DD")''')
+    assert result.data == ["1985-text1text212-01"]
+    result = self.client.execute(r'''select cast(cast("1985-12-01" as timestamp) as '''
+        r'''string format "YYYY-\"text1\"\"text2\"MM-DD")''')
+    assert result.data == ["1985-text1text212-01"]
+
+    # Datetime to string path: Text token containing separators.
+    result = self.client.execute(r'''select cast(cast("1985-12-02" as date) as '''
+        r'''string format "YYYY-\" -'./,:\"MM-DD")''')
+    assert result.data == ["1985- -'./,:12-02"]
+    result = self.client.execute(r'''select cast(cast("1985-12-02" as timestamp) as '''
+        r'''string format "YYYY-\" -'./,:\"MM-DD")''')
+    assert result.data == ["1985- -'./,:12-02"]
+
+    # Datetime to string path: Text token containing a double quote.
+    result = self.client.execute(r'''select cast(cast('1985-12-03' as date) as string '''
+        r'''format 'YYYY-"some \"text"MM-DD')''')
+    assert result.data == ['1985-some "text12-03']
+    result = self.client.execute(r'''select cast(cast('1985-12-03' as timestamp) as '''
+        r'''string format 'YYYY-"some \"text"MM-DD')''')
+    assert result.data == ['1985-some "text12-03']
+
+    # Datetime to string path: Text token containing a double quote where the text token
+    # itself is covered by escaped double quotes.
+    result = self.client.execute(r'''select cast(cast("1985-12-04" as date) as string '''
+        r'''format "YYYY-\"some \\\"text\"MM-DD")''')
+    assert result.data == ['1985-some "text12-04']
+    result = self.client.execute(r'''select cast(cast("1985-12-04" as timestamp) as '''
+        r'''string format "YYYY-\"some \\\"text\"MM-DD")''')
+    assert result.data == ['1985-some "text12-04']
+
+    # Backslash in format that escapes non-special chars.
+    result = self.client.execute(r'''select cast("1985- some \ text12-05" as date '''
+        r'''format 'YYYY-"some \ text"MM-DD')''')
+    assert result.data == ['1985-12-05']
+    result = self.client.execute(r'''select cast(cast("1985-12-06" as date) as string '''
+        r'''format 'YYYY-"some \ text"MM-DD')''')
+    assert result.data == ['1985-some  text12-06']
+
+    result = self.client.execute(r'''select cast("1985-some text12-07" as date '''
+        r'''format 'YYYY-"\some text"MM-DD')''')
+    assert result.data == ['1985-12-07']
+    result = self.client.execute(r'''select cast(cast("1985-12-08" as date) as string '''
+        r'''format 'YYYY-"\some text"MM-DD')''')
+    assert result.data == ['1985-some text12-08']
+
+    # Backslash in format that escapes special chars.
+    result = self.client.execute(r'''select cast("1985-\b\n\r\t12-09" as '''
+        r'''date format 'YYYY-"\b\n\r\t"MM-DD')''')
+    assert result.data == ['1985-12-09']
+    result = self.client.execute(r'''select cast(cast("1985-12-10" as date) as string '''
+        r'''format 'YYYY"\ttext\n"MM-DD')''')
+    assert result.data == [r'''1985	text
+12-10''']
+    result = self.client.execute(r'''select cast(cast("1985-12-11" as date) as string '''
+        r'''format "YYYY\"\ttext\n\"MM-DD")''')
+    assert result.data == [r'''1985	text
+12-11''']
+    result = self.client.execute(r'''select cast(cast("1985-12-12" as timestamp) as '''
+        r'''string format 'YYYY"\ttext\n"MM-DD')''')
+    assert result.data == [r'''1985	text
+12-12''']
+    result = self.client.execute(r'''select cast(cast("1985-12-13" as timestamp) as '''
+        r'''string format "YYYY\"\ttext\n\"MM-DD")''')
+    assert result.data == [r'''1985	text
+12-13''']
+
+    # Escaped backslash in text token.
+    result = self.client.execute(r'''select cast(cast("1985-12-14" as date) as string '''
+        r'''format 'YYYY"some\\text"MM-DD')''')
+    assert result.data == [r'''1985some\text12-14''']
+    result = self.client.execute(r'''select cast(cast("1985-12-15" as timestamp) as '''
+        r'''string format 'YYYY"\\"MM"\\"DD')''')
+    assert result.data == [r'''1985\12\15''']
+    result = self.client.execute(r'''select cast("1985\\12\\14 01:12:10" as timestamp '''
+        r'''format 'YYYY"\\"MM"\\"DD HH12:MI:SS')''')
+    assert result.data == [r'''1985-12-14 01:12:10''']
+    # Known limitation: When the format token is surrounded by escaped quotes then an
+    # escaped backslash at the end of the token together with the closing double quote is
+    # taken as a double escaped quote.
+    err = self.execute_query_expect_failure(self.client,
+        r'''select cast(cast("1985-12-16" as timestamp) as string format '''
+        r'''"YYYY\"\\\"MM\"\\\"DD")''')
+    assert "Bad date/time conversion format" in str(err)
+
+  def test_fm_fx_modifiers(self):
+    # Exact mathcing for the whole format.
+    result = self.client.execute("select cast('2001-03-01 03:10:15.123456 -01:30' as "
+        "timestamp format 'FXYYYY-MM-DD HH12:MI:SS.FF6 TZH:TZM')")
+    assert result.data == ["2001-03-01 03:10:15.123456000"]
+
+    # Strict separator matching.
+    result = self.client.execute("select cast('2001-03-02 03:10:15' as timestamp format"
+        "'FXYYYY MM-DD HH12:MI:SS')")
+    assert result.data == ["NULL"]
+
+    result = self.client.execute("select cast('2001-03-03 03:10:15' as timestamp format"
+        "'FXYYYY-MM-DD HH12::MI:SS')")
+    assert result.data == ["NULL"]
+
+    result = self.client.execute("select cast('2001-03-04    ' as timestamp format"
+        "'FXYYYY-MM-DD ')")
+    assert result.data == ["NULL"]
+
+    # Strict token length matching.
+    result = self.client.execute("select cast('2001-3-05' as timestamp format "
+        "'FXYYYY-MM-DD')")
+    assert result.data == ["NULL"]
+
+    result = self.client.execute("select cast('15-03-06' as timestamp format "
+        "'FXYYYY-MM-DD')")
+    assert result.data == ["NULL"]
+
+    result = self.client.execute("select cast('15-03-07' as date format 'FXYY-MM-DD')")
+    assert result.data == ["2015-03-07"]
+
+    result = self.client.execute("select cast('2001-03-08 03:15:00 AM' as timestamp "
+        "format 'FXYYYY-MM-DD HH12:MI:SS PM')")
+    assert result.data == ["2001-03-08 03:15:00"]
+
+    result = self.client.execute("select cast('2001-03-08 03:15:00 AM' as timestamp "
+        "format 'FXYYYY-MM-DD HH12:MI:SS P.M.')")
+    assert result.data == ["NULL"]
+
+    result = self.client.execute("select cast('2001-03-09 03:15:00.1234' as timestamp "
+        "format 'FXYYYY-MM-DD HH12:MI:SS.FF4')")
+    assert result.data == ["2001-03-09 03:15:00.123400000"]
+
+    result = self.client.execute("select cast('2001-03-09 03:15:00.12345' as timestamp "
+        "format 'FXYYYY-MM-DD HH12:MI:SS.FF4')")
+    assert result.data == ["NULL"]
+
+    result = self.client.execute("select cast('2001-03-09 03:15:00.12345' as timestamp "
+        "format 'FXYYYY-MM-DD HH12:MI:SS.FF')")
+    assert result.data == ["NULL"]
+
+    # Strict token length matching with text token containing escaped double quote.
+    result = self.client.execute(r'''select cast('2001-03-09 some "text03:25:00' '''
+        r'''as timestamp format "FXYYYY-MM-DD \"some \\\"text\"HH12:MI:SS")''')
+    assert result.data == ["2001-03-09 03:25:00"]
+
+    # Use FM to ignore FX modifier for some of the tokens.
+    result = self.client.execute("select cast('2001-03-10 03:15:00.12345' as timestamp "
+        "format 'FXYYYY-MM-DD HH12:MI:SS.FMFF')")
+    assert result.data == ["2001-03-10 03:15:00.123450000"]
+
+    result = self.client.execute("select cast('019-03-10 04:15:00' as timestamp "
+        "format 'FXFMYYYY-MM-DD HH12:MI:SS')")
+    assert result.data == ["2019-03-10 04:15:00"]
+
+    # Multiple FM modifiers in a format.
+    result = self.client.execute("select cast('2001-3-11 3:15:00.12345' as timestamp "
+        "format 'FXYYYY-FMMM-DD FMHH12:MI:SS.FMFF')")
+    assert result.data == ["2001-03-11 03:15:00.123450000"]
+
+    result = self.client.execute("select cast('2001-3-11 3:15:30' as timestamp "
+        "format 'FXYYYY-FMMM-DD FMFMHH12:MI:SS')")
+    assert result.data == ["2001-03-11 03:15:30"]
+
+    # FM modifier effects only the next token.
+    result = self.client.execute("select cast('2001-3-12 3:1:00.12345' as timestamp "
+        "format 'FXYYYY-FMMM-DD FMHH12:MI:SS.FMFF')")
+    assert result.data == ["NULL"]
+
+    # FM modifier before text token is valid for the text token and not for the token
+    # right after the text token.
+    result = self.client.execute(r'''select cast('1999-10text1' as timestamp format '''
+        ''' 'FXYYYY-MMFM"text"DD')''')
+    assert result.data == ["NULL"]
+
+    # FM modifier skips the separators and effects the next non-separator token.
+    result = self.client.execute(r'''select cast('1999-10-2' as timestamp format '''
+        ''' 'FXYYYY-MMFM-DD')''')
+    assert result.data == ["1999-10-02 00:00:00"]
+
+    # FM modifier at the end has no effect.
+    result = self.client.execute("select cast('2001-03-13 03:01:00' as timestamp "
+        "format 'FXYYYY-MM-DD HH12:MI:SSFM')")
+    assert result.data == ["2001-03-13 03:01:00"]
+
+    result = self.client.execute("select cast('2001-03-13 03:01:0' as timestamp "
+        "format 'FXYYYY-MM-DD HH12:MI:SSFM')")
+    assert result.data == ["NULL"]
+
+    # In a datetime to string path FX is the default so it works with FX as it would
+    # without.
+    result = self.client.execute("select cast(cast('2001-03-05 03:10:15.123456' as "
+        "timestamp) as string format 'FXYYYY-MM-DD HH24:MI:SS.FF7')")
+    assert result.data == ["2001-03-05 03:10:15.1234560"]
+
+    # Datetime to string path: Tokens with FM modifier don't pad output to a given
+    # length.
+    result = self.client.execute("select cast(cast('2001-03-14 03:06:08' as timestamp) "
+        "as string format 'YYYY-MM-DD FMHH24:FMMI:FMSS')")
+    assert result.data == ["2001-03-14 3:6:8"]
+
+    result = self.client.execute("select cast(cast('0001-03-09' as date) "
+        "as string format 'FMYYYY-FMMM-FMDD')")
+    assert result.data == ["1-3-9"]
+
+    result = self.client.execute("select cast(date'0001-03-10' as string format "
+        "'FMYY-FMMM-FMDD')")
+    assert result.data == ["1-3-10"]
+
+    # Datetime to string path: FM modifier is effective even if FX modifier is also
+    # given.
+    result = self.client.execute("select cast(cast('2001-03-15 03:06:08' as "
+        "timestamp) as string format 'FXYYYY-MM-DD FMHH24:FMMI:FMSS')")
+    assert result.data == ["2001-03-15 3:6:8"]
+
+    result = self.client.execute("select cast(cast('0001-04-09' as date) "
+        "as string format 'FXYYYY-FMMM-FMDD')")
+    assert result.data == ["0001-4-9"]
+
+    result = self.client.execute("select cast(cast('0001-04-10' as date) "
+        "as string format 'FXFMYYYY-FMMM-FMDD')")
+    assert result.data == ["1-4-10"]
+
   def test_format_parse_errors(self):
     # Invalid format
     err = self.execute_query_expect_failure(self.client,
@@ -731,3 +1070,64 @@ class TestCastWithFormat(ImpalaTestSuite):
     err = self.execute_query_expect_failure(self.client,
         "select cast('2017-05-01' as timestamp format 'YYYY-MM-DD-RR--')")
     assert "Both year and round year are provided" in str(err)
+
+    # Unclosed quotation in text pattern
+    err = self.execute_query_expect_failure(self.client,
+        r'''select cast('1985-11-20text' as timestamp format 'YYYY-MM-DD"text')''')
+    assert "Missing closing quotation mark." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        r'''select cast('1985-11-21text' as timestamp format 'YYYY-MM-DD\"text"')''')
+    assert "Missing closing quotation mark." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        r'''select cast(date"1985-12-08" as string format 'YYYY-MM-DD \"X"');''')
+    assert "Missing closing quotation mark." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        r'''select cast(date"1985-12-09" as string format 'YYYY-MM-DD "X');''')
+    assert "Missing closing quotation mark." in str(err)
+
+    # Format containing text token only.
+    err = self.execute_query_expect_failure(self.client,
+        r'''select cast("1985-11-29" as date format '" some text "')''')
+    assert "No datetime tokens provided." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        r'''select cast(cast("1985-12-02" as date) as string format "\"free text\"")''')
+    assert "No datetime tokens provided." in str(err)
+
+    # FX modifier not at the begining of the format.
+    err = self.execute_query_expect_failure(self.client,
+        'select cast("2001-03-01 00:10:02" as timestamp format '
+        '"YYYY-MM-DD FXHH12:MI:SS")')
+    assert "FX modifier should be at the beginning of the format string." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        'select cast("2001-03-01 00:10:02" as timestamp format '
+        '"YYYY-MM-DD HH12:MI:SS FX")')
+    assert "FX modifier should be at the beginning of the format string." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        'select cast(date"2001-03-01" as string format "YYYYFX-MM-DD")')
+    assert "FX modifier should be at the beginning of the format string." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        'select cast(date"2001-03-02" as string format "FXFMFXYYYY-MM-DD")')
+    assert "FX modifier should be at the beginning of the format string." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        'select cast(date"2001-03-03" as string format "FXFXYYYY-MM-DD")')
+    assert "FX modifier should be at the beginning of the format string." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        'select cast(date"2001-03-04" as string format "FMFXYYYY-MM-DD")')
+    assert "FX modifier should be at the beginning of the format string." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        'select cast(date"2001-03-03" as string format "-FXYYYY-MM-DD")')
+    assert "FX modifier should be at the beginning of the format string." in str(err)
+
+    err = self.execute_query_expect_failure(self.client,
+        r'''select cast(date"2001-03-03" as string format '"text"FXYYYY-MM-DD')''')
+    assert "FX modifier should be at the beginning of the format string." in str(err)