2019-03-14 02:55:04 +00:00
# pragma once
2019-05-07 23:31:35 +00:00
# include <Functions/IFunction.h>
2019-05-16 19:39:42 +00:00
# include <Functions/DummyJSONParser.h>
# include <Functions/SimdJSONParser.h>
# include <Functions/RapidJSONParser.h>
2019-06-20 09:12:49 +00:00
# include "config_functions.h"
2019-05-16 19:39:42 +00:00
# include <Common/CpuId.h>
2019-05-13 23:44:55 +00:00
# include <Common/typeid_cast.h>
2019-08-13 15:26:31 +00:00
# include <Core/AccurateComparison.h>
2019-05-16 19:39:42 +00:00
# include <Core/Settings.h>
2019-03-14 02:55:04 +00:00
# include <Columns/ColumnConst.h>
2019-03-14 08:07:25 +00:00
# include <Columns/ColumnString.h>
2019-05-13 23:44:55 +00:00
# include <Columns/ColumnVector.h>
# include <Columns/ColumnFixedString.h>
# include <Columns/ColumnNullable.h>
# include <Columns/ColumnArray.h>
# include <Columns/ColumnTuple.h>
# include <DataTypes/DataTypesNumber.h>
# include <DataTypes/DataTypeString.h>
# include <DataTypes/DataTypeEnum.h>
2019-03-14 05:48:29 +00:00
# include <DataTypes/DataTypeFactory.h>
2019-05-13 23:44:55 +00:00
# include <DataTypes/DataTypeNullable.h>
# include <DataTypes/DataTypeArray.h>
# include <DataTypes/DataTypeTuple.h>
2019-05-16 19:39:42 +00:00
# include <Interpreters/Context.h>
2019-04-26 21:58:14 +00:00
# include <ext/range.h>
2019-03-14 02:55:04 +00:00
namespace DB
{
namespace ErrorCodes
{
2019-03-14 06:19:21 +00:00
extern const int ILLEGAL_COLUMN ;
2019-03-14 02:55:04 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT ;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH ;
}
2019-05-13 23:44:55 +00:00
/// Functions to parse JSONs and extract values from it.
/// The first argument of all these functions gets a JSON,
/// after that there are any number of arguments specifying path to a desired part from the JSON's root.
/// For example,
2019-05-15 22:00:05 +00:00
/// select JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100
2019-05-16 19:39:42 +00:00
template < typename Name , template < typename > typename Impl >
2019-05-13 23:44:55 +00:00
class FunctionJSON : public IFunction
2019-03-14 02:55:04 +00:00
{
2019-05-13 23:44:55 +00:00
public :
2019-05-16 19:39:42 +00:00
static FunctionPtr create ( const Context & context_ ) { return std : : make_shared < FunctionJSON > ( context_ ) ; }
FunctionJSON ( const Context & context_ ) : context ( context_ ) { }
2019-05-13 23:44:55 +00:00
static constexpr auto name = Name : : name ;
String getName ( ) const override { return Name : : name ; }
bool isVariadic ( ) const override { return true ; }
size_t getNumberOfArguments ( ) const override { return 0 ; }
bool useDefaultImplementationForConstants ( ) const override { return false ; }
2019-05-15 22:00:05 +00:00
DataTypePtr getReturnTypeImpl ( const ColumnsWithTypeAndName & arguments ) const override
{
2019-05-16 19:39:42 +00:00
return Impl < DummyJSONParser > : : getType ( Name : : name , arguments ) ;
2019-05-15 22:00:05 +00:00
}
2019-05-13 23:44:55 +00:00
void executeImpl ( Block & block , const ColumnNumbers & arguments , size_t result_pos , size_t input_rows_count ) override
2019-03-14 02:55:04 +00:00
{
2019-05-16 19:39:42 +00:00
/// Choose JSONParser.
# if USE_SIMDJSON
2019-08-04 13:46:15 +00:00
if ( context . getSettings ( ) . allow_simdjson & & Cpu : : CpuFlagsCache : : have_SSE42 & & Cpu : : CpuFlagsCache : : have_PCLMUL )
2019-03-14 02:55:04 +00:00
{
2019-05-16 19:39:42 +00:00
Executor < SimdJSONParser > : : run ( block , arguments , result_pos , input_rows_count ) ;
return ;
}
# endif
# if USE_RAPIDJSON
Executor < RapidJSONParser > : : run ( block , arguments , result_pos , input_rows_count ) ;
# else
Executor < DummyJSONParser > : : run ( block , arguments , result_pos , input_rows_count ) ;
# endif
}
private :
const Context & context ;
template < typename JSONParser >
class Executor
{
public :
static void run ( Block & block , const ColumnNumbers & arguments , size_t result_pos , size_t input_rows_count )
{
MutableColumnPtr to { block . getByPosition ( result_pos ) . type - > createColumn ( ) } ;
to - > reserve ( input_rows_count ) ;
if ( arguments . size ( ) < 1 )
throw Exception { " Function " + String ( Name : : name ) + " requires at least one argument " , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH } ;
const auto & first_column = block . getByPosition ( arguments [ 0 ] ) ;
if ( ! isString ( first_column . type ) )
throw Exception { " The first argument of function " + String ( Name : : name ) + " should be a string containing JSON, illegal type: " + first_column . type - > getName ( ) ,
2019-05-15 22:00:05 +00:00
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT } ;
2019-05-16 19:39:42 +00:00
const ColumnPtr & arg_json = first_column . column ;
auto col_json_const = typeid_cast < const ColumnConst * > ( arg_json . get ( ) ) ;
auto col_json_string
= typeid_cast < const ColumnString * > ( col_json_const ? col_json_const - > getDataColumnPtr ( ) . get ( ) : arg_json . get ( ) ) ;
if ( ! col_json_string )
throw Exception { " Illegal column " + arg_json - > getName ( ) , ErrorCodes : : ILLEGAL_COLUMN } ;
const ColumnString : : Chars & chars = col_json_string - > getChars ( ) ;
const ColumnString : : Offsets & offsets = col_json_string - > getOffsets ( ) ;
2019-05-17 14:21:37 +00:00
std : : vector < Move > moves = prepareListOfMoves ( block , arguments ) ;
2019-05-15 22:00:05 +00:00
2019-05-16 19:39:42 +00:00
/// Preallocate memory in parser if necessary.
JSONParser parser ;
if ( parser . need_preallocate )
2019-05-17 14:21:37 +00:00
parser . preallocate ( calculateMaxSize ( offsets ) ) ;
2019-05-15 22:00:05 +00:00
2019-05-16 19:39:42 +00:00
Impl < JSONParser > impl ;
2019-03-14 02:55:04 +00:00
2019-05-16 19:39:42 +00:00
/// prepare() does Impl-specific preparation before handling each row.
impl . prepare ( Name : : name , block , arguments , result_pos ) ;
2019-05-13 23:44:55 +00:00
2019-05-27 18:46:55 +00:00
bool json_parsed_ok = false ;
if ( col_json_const )
{
StringRef json { reinterpret_cast < const char * > ( & chars [ 0 ] ) , offsets [ 0 ] - 1 } ;
json_parsed_ok = parser . parse ( json ) ;
}
2019-05-16 19:39:42 +00:00
for ( const auto i : ext : : range ( 0 , input_rows_count ) )
2019-05-13 23:44:55 +00:00
{
2019-05-27 18:46:55 +00:00
if ( ! col_json_const )
{
StringRef json { reinterpret_cast < const char * > ( & chars [ offsets [ i - 1 ] ] ) , offsets [ i ] - offsets [ i - 1 ] - 1 } ;
json_parsed_ok = parser . parse ( json ) ;
}
2019-05-13 23:44:55 +00:00
2019-05-27 18:46:55 +00:00
bool ok = json_parsed_ok ;
2019-05-16 19:39:42 +00:00
if ( ok )
2019-03-14 02:55:04 +00:00
{
2019-05-16 19:39:42 +00:00
auto it = parser . getRoot ( ) ;
/// Perform moves.
for ( size_t j = 0 ; ( j ! = moves . size ( ) ) & & ok ; + + j )
2019-05-13 23:44:55 +00:00
{
2019-05-16 19:39:42 +00:00
switch ( moves [ j ] . type )
2019-05-16 12:16:21 +00:00
{
2019-05-16 19:39:42 +00:00
case MoveType : : ConstIndex :
ok = moveIteratorToElementByIndex ( it , moves [ j ] . index ) ;
break ;
case MoveType : : ConstKey :
ok = moveIteratorToElementByKey ( it , moves [ j ] . key ) ;
break ;
case MoveType : : Index :
{
const Field field = ( * block . getByPosition ( arguments [ j + 1 ] ) . column ) [ i ] ;
ok = moveIteratorToElementByIndex ( it , field . get < Int64 > ( ) ) ;
break ;
}
case MoveType : : Key :
{
const Field field = ( * block . getByPosition ( arguments [ j + 1 ] ) . column ) [ i ] ;
ok = moveIteratorToElementByKey ( it , field . get < String > ( ) . data ( ) ) ;
break ;
}
2019-05-16 12:16:21 +00:00
}
2019-05-13 23:44:55 +00:00
}
2019-05-16 19:39:42 +00:00
if ( ok )
ok = impl . addValueToColumn ( * to , it ) ;
2019-05-10 08:49:03 +00:00
}
2019-05-16 19:39:42 +00:00
/// We add default value (=null or zero) if something goes wrong, we don't throw exceptions in these JSON functions.
if ( ! ok )
to - > insertDefault ( ) ;
2019-05-16 12:16:21 +00:00
}
2019-05-16 19:39:42 +00:00
block . getByPosition ( result_pos ) . column = std : : move ( to ) ;
2019-05-13 23:44:55 +00:00
}
2019-03-14 02:55:04 +00:00
2019-05-16 19:39:42 +00:00
private :
/// Represents a move of a JSON iterator described by a single argument passed to a JSON function.
/// For example, the call JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1)
/// contains two moves: {MoveType::ConstKey, "b"} and {MoveType::ConstIndex, 1}.
/// Keys and indices can be nonconst, in this case they are calculated for each row.
enum class MoveType
{
Key ,
Index ,
ConstKey ,
ConstIndex ,
} ;
2019-05-17 14:21:37 +00:00
2019-05-16 19:39:42 +00:00
struct Move
{
Move ( MoveType type_ , size_t index_ = 0 ) : type ( type_ ) , index ( index_ ) { }
Move ( MoveType type_ , const String & key_ ) : type ( type_ ) , key ( key_ ) { }
MoveType type ;
size_t index = 0 ;
String key ;
} ;
2019-05-13 23:44:55 +00:00
2019-05-17 14:21:37 +00:00
static std : : vector < Move > prepareListOfMoves ( Block & block , const ColumnNumbers & arguments )
{
constexpr size_t num_extra_arguments = Impl < JSONParser > : : num_extra_arguments ;
const size_t num_moves = arguments . size ( ) - num_extra_arguments - 1 ;
std : : vector < Move > moves ;
moves . reserve ( num_moves ) ;
for ( const auto i : ext : : range ( 0 , num_moves ) )
{
const auto & column = block . getByPosition ( arguments [ i + 1 ] ) ;
if ( ! isString ( column . type ) & & ! isInteger ( column . type ) )
throw Exception { " The argument " + std : : to_string ( i + 2 ) + " of function " + String ( Name : : name )
+ " should be a string specifying key or an integer specifying index, illegal type: " + column . type - > getName ( ) ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT } ;
2019-06-27 19:28:52 +00:00
if ( isColumnConst ( * column . column ) )
2019-05-17 14:21:37 +00:00
{
const auto & column_const = static_cast < const ColumnConst & > ( * column . column ) ;
if ( isString ( column . type ) )
moves . emplace_back ( MoveType : : ConstKey , column_const . getField ( ) . get < String > ( ) ) ;
else
moves . emplace_back ( MoveType : : ConstIndex , column_const . getField ( ) . get < Int64 > ( ) ) ;
}
else
{
if ( isString ( column . type ) )
moves . emplace_back ( MoveType : : Key , " " ) ;
else
moves . emplace_back ( MoveType : : Index , 0 ) ;
}
}
return moves ;
}
2019-05-16 19:39:42 +00:00
using Iterator = typename JSONParser : : Iterator ;
2019-05-15 22:00:05 +00:00
2019-05-16 19:39:42 +00:00
/// Performs moves of types MoveType::Index and MoveType::ConstIndex.
static bool moveIteratorToElementByIndex ( Iterator & it , int index )
2019-05-13 23:44:55 +00:00
{
2019-05-16 19:39:42 +00:00
if ( JSONParser : : isArray ( it ) )
2019-05-13 23:44:55 +00:00
{
2019-05-16 19:39:42 +00:00
if ( index > 0 )
2019-05-17 14:21:37 +00:00
return JSONParser : : arrayElementByIndex ( it , index - 1 ) ;
2019-05-16 19:39:42 +00:00
else
2019-05-17 14:21:37 +00:00
return JSONParser : : arrayElementByIndex ( it , JSONParser : : sizeOfArray ( it ) + index ) ;
2019-05-13 23:44:55 +00:00
}
2019-05-16 19:39:42 +00:00
if ( JSONParser : : isObject ( it ) )
2019-05-13 23:44:55 +00:00
{
2019-05-16 19:39:42 +00:00
if ( index > 0 )
2019-05-17 14:21:37 +00:00
return JSONParser : : objectMemberByIndex ( it , index - 1 ) ;
2019-05-16 19:39:42 +00:00
else
2019-05-17 14:21:37 +00:00
return JSONParser : : objectMemberByIndex ( it , JSONParser : : sizeOfObject ( it ) + index ) ;
2019-05-13 23:44:55 +00:00
}
2019-05-16 19:39:42 +00:00
return false ;
2019-05-13 23:44:55 +00:00
}
2019-03-14 02:55:04 +00:00
2019-05-16 19:39:42 +00:00
/// Performs moves of types MoveType::Key and MoveType::ConstKey.
static bool moveIteratorToElementByKey ( Iterator & it , const String & key )
2019-05-13 23:44:55 +00:00
{
2019-05-16 19:39:42 +00:00
if ( JSONParser : : isObject ( it ) )
2019-05-17 14:21:37 +00:00
return JSONParser : : objectMemberByName ( it , key ) ;
2019-05-16 19:39:42 +00:00
return false ;
2019-03-14 02:55:04 +00:00
}
2019-05-17 14:21:37 +00:00
static size_t calculateMaxSize ( const ColumnString : : Offsets & offsets )
{
size_t max_size = 0 ;
for ( const auto i : ext : : range ( 0 , offsets . size ( ) ) )
if ( max_size < offsets [ i ] - offsets [ i - 1 ] )
max_size = offsets [ i ] - offsets [ i - 1 ] ;
if ( max_size < 1 )
max_size = 1 ;
return max_size ;
}
2019-05-16 19:39:42 +00:00
} ;
2019-05-13 23:44:55 +00:00
} ;
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
struct NameJSONHas { static constexpr auto name { " JSONHas " } ; } ;
struct NameJSONLength { static constexpr auto name { " JSONLength " } ; } ;
struct NameJSONKey { static constexpr auto name { " JSONKey " } ; } ;
struct NameJSONType { static constexpr auto name { " JSONType " } ; } ;
struct NameJSONExtractInt { static constexpr auto name { " JSONExtractInt " } ; } ;
struct NameJSONExtractUInt { static constexpr auto name { " JSONExtractUInt " } ; } ;
struct NameJSONExtractFloat { static constexpr auto name { " JSONExtractFloat " } ; } ;
struct NameJSONExtractBool { static constexpr auto name { " JSONExtractBool " } ; } ;
struct NameJSONExtractString { static constexpr auto name { " JSONExtractString " } ; } ;
struct NameJSONExtract { static constexpr auto name { " JSONExtract " } ; } ;
2019-05-15 23:56:10 +00:00
struct NameJSONExtractKeysAndValues { static constexpr auto name { " JSONExtractKeysAndValues " } ; } ;
struct NameJSONExtractRaw { static constexpr auto name { " JSONExtractRaw " } ; } ;
2019-05-13 23:44:55 +00:00
template < typename JSONParser >
class JSONHasImpl
{
public :
2019-05-15 22:00:05 +00:00
static DataTypePtr getType ( const char * , const ColumnsWithTypeAndName & ) { return std : : make_shared < DataTypeUInt8 > ( ) ; }
2019-05-13 23:44:55 +00:00
using Iterator = typename JSONParser : : Iterator ;
static bool addValueToColumn ( IColumn & dest , const Iterator & )
{
ColumnVector < UInt8 > & col_vec = static_cast < ColumnVector < UInt8 > & > ( dest ) ;
col_vec . insertValue ( 1 ) ;
2019-03-14 02:55:04 +00:00
return true ;
}
2019-05-13 23:44:55 +00:00
static constexpr size_t num_extra_arguments = 0 ;
2019-05-15 22:00:05 +00:00
static void prepare ( const char * , const Block & , const ColumnNumbers & , size_t ) { }
2019-05-13 23:44:55 +00:00
} ;
template < typename JSONParser >
class JSONLengthImpl
{
2019-03-14 02:55:04 +00:00
public :
2019-05-15 22:00:05 +00:00
static DataTypePtr getType ( const char * , const ColumnsWithTypeAndName & )
2019-05-13 23:44:55 +00:00
{
return std : : make_shared < DataTypeUInt64 > ( ) ;
}
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
using Iterator = typename JSONParser : : Iterator ;
static bool addValueToColumn ( IColumn & dest , const Iterator & it )
{
size_t size ;
if ( JSONParser : : isArray ( it ) )
2019-05-17 14:21:37 +00:00
size = JSONParser : : sizeOfArray ( it ) ;
2019-05-13 23:44:55 +00:00
else if ( JSONParser : : isObject ( it ) )
2019-05-17 14:21:37 +00:00
size = JSONParser : : sizeOfObject ( it ) ;
2019-05-13 23:44:55 +00:00
else
return false ;
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
ColumnVector < UInt64 > & col_vec = static_cast < ColumnVector < UInt64 > & > ( dest ) ;
col_vec . insertValue ( size ) ;
return true ;
}
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
static constexpr size_t num_extra_arguments = 0 ;
2019-05-15 22:00:05 +00:00
static void prepare ( const char * , const Block & , const ColumnNumbers & , size_t ) { }
2019-05-13 23:44:55 +00:00
} ;
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
template < typename JSONParser >
class JSONKeyImpl
{
public :
2019-05-15 22:00:05 +00:00
static DataTypePtr getType ( const char * , const ColumnsWithTypeAndName & )
2019-05-13 23:44:55 +00:00
{
return std : : make_shared < DataTypeString > ( ) ;
}
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
using Iterator = typename JSONParser : : Iterator ;
static bool addValueToColumn ( IColumn & dest , const Iterator & it )
2019-03-14 02:55:04 +00:00
{
2019-05-17 14:21:37 +00:00
if ( ! JSONParser : : isObjectMember ( it ) )
2019-05-13 23:44:55 +00:00
return false ;
StringRef key = JSONParser : : getKey ( it ) ;
ColumnString & col_str = static_cast < ColumnString & > ( dest ) ;
col_str . insertData ( key . data , key . size ) ;
return true ;
}
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
static constexpr size_t num_extra_arguments = 0 ;
2019-05-15 22:00:05 +00:00
static void prepare ( const char * , const Block & , const ColumnNumbers & , size_t ) { }
2019-05-13 23:44:55 +00:00
} ;
2019-03-14 05:48:29 +00:00
2019-03-14 08:30:15 +00:00
2019-05-13 23:44:55 +00:00
template < typename JSONParser >
class JSONTypeImpl
{
public :
2019-05-15 22:00:05 +00:00
static DataTypePtr getType ( const char * , const ColumnsWithTypeAndName & )
2019-05-13 23:44:55 +00:00
{
static const std : : vector < std : : pair < String , Int8 > > values = {
{ " Array " , ' [ ' } ,
{ " Object " , ' { ' } ,
{ " String " , ' " ' } ,
2019-05-16 18:22:59 +00:00
{ " Int64 " , ' i ' } ,
{ " UInt64 " , ' u ' } ,
{ " Double " , ' d ' } ,
2019-05-13 23:44:55 +00:00
{ " Bool " , ' b ' } ,
2019-05-15 22:00:05 +00:00
{ " Null " , 0 } , /// the default value for the column.
2019-05-13 23:44:55 +00:00
} ;
return std : : make_shared < DataTypeEnum < Int8 > > ( values ) ;
}
using Iterator = typename JSONParser : : Iterator ;
static bool addValueToColumn ( IColumn & dest , const Iterator & it )
{
UInt8 type ;
2019-05-16 18:22:59 +00:00
if ( JSONParser : : isInt64 ( it ) )
type = ' i ' ;
else if ( JSONParser : : isUInt64 ( it ) )
type = ' u ' ;
else if ( JSONParser : : isDouble ( it ) )
2019-05-13 23:44:55 +00:00
type = ' d ' ;
else if ( JSONParser : : isBool ( it ) )
type = ' b ' ;
else if ( JSONParser : : isString ( it ) )
type = ' " ' ;
else if ( JSONParser : : isArray ( it ) )
type = ' [ ' ;
else if ( JSONParser : : isObject ( it ) )
type = ' { ' ;
else if ( JSONParser : : isNull ( it ) )
type = 0 ;
2019-03-14 02:55:04 +00:00
else
2019-05-13 23:44:55 +00:00
return false ;
ColumnVector < Int8 > & col_vec = static_cast < ColumnVector < Int8 > & > ( dest ) ;
col_vec . insertValue ( type ) ;
return true ;
}
static constexpr size_t num_extra_arguments = 0 ;
2019-05-15 22:00:05 +00:00
static void prepare ( const char * , const Block & , const ColumnNumbers & , size_t ) { }
2019-05-13 23:44:55 +00:00
} ;
template < typename JSONParser , typename NumberType , bool convert_bool_to_integer = false >
class JSONExtractNumericImpl
{
public :
2019-05-15 22:00:05 +00:00
static DataTypePtr getType ( const char * , const ColumnsWithTypeAndName & )
2019-05-13 23:44:55 +00:00
{
return std : : make_shared < DataTypeNumber < NumberType > > ( ) ;
}
using Iterator = typename JSONParser : : Iterator ;
static bool addValueToColumn ( IColumn & dest , const Iterator & it )
{
NumberType value ;
2019-05-16 18:22:59 +00:00
if ( JSONParser : : isInt64 ( it ) )
2019-05-13 23:44:55 +00:00
{
2019-05-16 18:22:59 +00:00
if ( ! accurate : : convertNumeric ( JSONParser : : getInt64 ( it ) , value ) )
2019-05-13 23:44:55 +00:00
return false ;
}
2019-05-16 18:22:59 +00:00
else if ( JSONParser : : isUInt64 ( it ) )
2019-03-14 02:55:04 +00:00
{
2019-05-16 18:22:59 +00:00
if ( ! accurate : : convertNumeric ( JSONParser : : getUInt64 ( it ) , value ) )
return false ;
}
else if ( JSONParser : : isDouble ( it ) )
{
if ( ! accurate : : convertNumeric ( JSONParser : : getDouble ( it ) , value ) )
2019-05-13 23:44:55 +00:00
return false ;
2019-03-14 02:55:04 +00:00
}
2019-05-13 23:44:55 +00:00
else if ( JSONParser : : isBool ( it ) & & std : : is_integral_v < NumberType > & & convert_bool_to_integer )
value = static_cast < NumberType > ( JSONParser : : getBool ( it ) ) ;
else
return false ;
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
auto & col_vec = static_cast < ColumnVector < NumberType > & > ( dest ) ;
col_vec . insertValue ( value ) ;
return true ;
}
static constexpr size_t num_extra_arguments = 0 ;
2019-05-15 22:00:05 +00:00
static void prepare ( const char * , const Block & , const ColumnNumbers & , size_t ) { }
2019-05-13 23:44:55 +00:00
} ;
template < typename JSONParser >
using JSONExtractInt8Impl = JSONExtractNumericImpl < JSONParser , Int8 > ;
template < typename JSONParser >
using JSONExtractUInt8Impl = JSONExtractNumericImpl < JSONParser , UInt8 > ;
template < typename JSONParser >
using JSONExtractInt16Impl = JSONExtractNumericImpl < JSONParser , Int16 > ;
template < typename JSONParser >
using JSONExtractUInt16Impl = JSONExtractNumericImpl < JSONParser , UInt16 > ;
template < typename JSONParser >
using JSONExtractInt32Impl = JSONExtractNumericImpl < JSONParser , Int32 > ;
template < typename JSONParser >
using JSONExtractUInt32Impl = JSONExtractNumericImpl < JSONParser , UInt32 > ;
template < typename JSONParser >
using JSONExtractInt64Impl = JSONExtractNumericImpl < JSONParser , Int64 > ;
template < typename JSONParser >
using JSONExtractUInt64Impl = JSONExtractNumericImpl < JSONParser , UInt64 > ;
template < typename JSONParser >
using JSONExtractFloat32Impl = JSONExtractNumericImpl < JSONParser , Float32 > ;
template < typename JSONParser >
using JSONExtractFloat64Impl = JSONExtractNumericImpl < JSONParser , Float64 > ;
template < typename JSONParser >
class JSONExtractBoolImpl
{
public :
2019-05-15 22:00:05 +00:00
static DataTypePtr getType ( const char * , const ColumnsWithTypeAndName & )
2019-05-13 23:44:55 +00:00
{
return std : : make_shared < DataTypeUInt8 > ( ) ;
}
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
using Iterator = typename JSONParser : : Iterator ;
static bool addValueToColumn ( IColumn & dest , const Iterator & it )
{
if ( ! JSONParser : : isBool ( it ) )
return false ;
2019-03-14 02:55:04 +00:00
2019-05-13 23:44:55 +00:00
auto & col_vec = static_cast < ColumnVector < UInt8 > & > ( dest ) ;
col_vec . insertValue ( static_cast < UInt8 > ( JSONParser : : getBool ( it ) ) ) ;
return true ;
}
static constexpr size_t num_extra_arguments = 0 ;
2019-05-15 22:00:05 +00:00
static void prepare ( const char * , const Block & , const ColumnNumbers & , size_t ) { }
2019-05-13 23:44:55 +00:00
} ;
template < typename JSONParser >
class JSONExtractStringImpl
{
public :
2019-05-15 22:00:05 +00:00
static DataTypePtr getType ( const char * , const ColumnsWithTypeAndName & )
2019-05-13 23:44:55 +00:00
{
return std : : make_shared < DataTypeString > ( ) ;
}
using Iterator = typename JSONParser : : Iterator ;
static bool addValueToColumn ( IColumn & dest , const Iterator & it )
{
if ( ! JSONParser : : isString ( it ) )
return false ;
StringRef str = JSONParser : : getString ( it ) ;
ColumnString & col_str = static_cast < ColumnString & > ( dest ) ;
col_str . insertData ( str . data , str . size ) ;
return true ;
}
static constexpr size_t num_extra_arguments = 0 ;
2019-05-15 22:00:05 +00:00
static void prepare ( const char * , const Block & , const ColumnNumbers & , size_t ) { }
2019-05-13 23:44:55 +00:00
} ;
2019-05-15 23:56:10 +00:00
/// Nodes of the extract tree. We need the extract tree to extract from JSON complex values containing array, tuples or nullables.
2019-05-13 23:44:55 +00:00
template < typename JSONParser >
2019-05-15 23:56:10 +00:00
struct JSONExtractTree
2019-05-13 23:44:55 +00:00
{
using Iterator = typename JSONParser : : Iterator ;
2019-03-14 08:07:25 +00:00
2019-05-13 23:44:55 +00:00
class Node
{
public :
Node ( ) { }
virtual ~ Node ( ) { }
virtual bool addValueToColumn ( IColumn & , const Iterator & ) = 0 ;
} ;
2019-03-14 08:07:25 +00:00
2019-05-13 23:44:55 +00:00
template < typename NumberType >
class NumericNode : public Node
{
public :
bool addValueToColumn ( IColumn & dest , const Iterator & it ) override
{
return JSONExtractNumericImpl < JSONParser , NumberType , true > : : addValueToColumn ( dest , it ) ;
}
} ;
2019-03-14 08:07:25 +00:00
2019-05-13 23:44:55 +00:00
class StringNode : public Node
{
public :
bool addValueToColumn ( IColumn & dest , const Iterator & it ) override
{
return JSONExtractStringImpl < JSONParser > : : addValueToColumn ( dest , it ) ;
}
} ;
2019-03-14 08:07:25 +00:00
2019-05-13 23:44:55 +00:00
class FixedStringNode : public Node
{
public :
bool addValueToColumn ( IColumn & dest , const Iterator & it ) override
{
if ( ! JSONParser : : isString ( it ) )
return false ;
auto & col_str = static_cast < ColumnFixedString & > ( dest ) ;
StringRef str = JSONParser : : getString ( it ) ;
if ( str . size > col_str . getN ( ) )
return false ;
col_str . insertData ( str . data , str . size ) ;
return true ;
}
} ;
2019-03-14 08:07:25 +00:00
2019-05-13 23:44:55 +00:00
template < typename Type >
class EnumNode : public Node
{
public :
EnumNode ( const std : : vector < std : : pair < String , Type > > & name_value_pairs_ ) : name_value_pairs ( name_value_pairs_ )
2019-03-14 08:07:25 +00:00
{
2019-05-13 23:44:55 +00:00
for ( const auto & name_value_pair : name_value_pairs )
{
name_to_value_map . emplace ( name_value_pair . first , name_value_pair . second ) ;
only_values . emplace ( name_value_pair . second ) ;
}
}
2019-03-14 08:07:25 +00:00
2019-05-13 23:44:55 +00:00
bool addValueToColumn ( IColumn & dest , const Iterator & it ) override
{
auto & col_vec = static_cast < ColumnVector < Type > & > ( dest ) ;
2019-03-14 02:55:04 +00:00
2019-05-16 18:22:59 +00:00
if ( JSONParser : : isInt64 ( it ) )
{
Type value ;
if ( ! accurate : : convertNumeric ( JSONParser : : getInt64 ( it ) , value ) | | ! only_values . count ( value ) )
return false ;
col_vec . insertValue ( value ) ;
return true ;
}
if ( JSONParser : : isUInt64 ( it ) )
2019-03-14 02:55:04 +00:00
{
2019-05-16 18:22:59 +00:00
Type value ;
if ( ! accurate : : convertNumeric ( JSONParser : : getUInt64 ( it ) , value ) | | ! only_values . count ( value ) )
2019-05-13 23:44:55 +00:00
return false ;
col_vec . insertValue ( value ) ;
return true ;
}
2019-03-14 08:07:25 +00:00
2019-05-13 23:44:55 +00:00
if ( JSONParser : : isString ( it ) )
{
auto value = name_to_value_map . find ( JSONParser : : getString ( it ) ) ;
if ( value = = name_to_value_map . end ( ) )
return false ;
col_vec . insertValue ( value - > second ) ;
return true ;
2019-03-14 02:55:04 +00:00
}
2019-05-13 23:44:55 +00:00
return false ;
}
private :
std : : vector < std : : pair < String , Type > > name_value_pairs ;
std : : unordered_map < StringRef , Type > name_to_value_map ;
std : : unordered_set < Type > only_values ;
} ;
class NullableNode : public Node
{
public :
NullableNode ( std : : unique_ptr < Node > nested_ ) : nested ( std : : move ( nested_ ) ) { }
bool addValueToColumn ( IColumn & dest , const Iterator & it ) override
{
ColumnNullable & col_null = static_cast < ColumnNullable & > ( dest ) ;
if ( ! nested - > addValueToColumn ( col_null . getNestedColumn ( ) , it ) )
return false ;
col_null . getNullMapColumn ( ) . insertValue ( 0 ) ;
return true ;
}
private :
std : : unique_ptr < Node > nested ;
} ;
class ArrayNode : public Node
{
public :
ArrayNode ( std : : unique_ptr < Node > nested_ ) : nested ( std : : move ( nested_ ) ) { }
bool addValueToColumn ( IColumn & dest , const Iterator & it ) override
{
if ( ! JSONParser : : isArray ( it ) )
return false ;
Iterator it2 = it ;
2019-05-17 14:21:37 +00:00
if ( ! JSONParser : : firstArrayElement ( it2 ) )
2019-05-13 23:44:55 +00:00
return false ;
ColumnArray & col_arr = static_cast < ColumnArray & > ( dest ) ;
auto & data = col_arr . getData ( ) ;
size_t old_size = data . size ( ) ;
bool were_valid_elements = false ;
do
2019-03-14 02:55:04 +00:00
{
2019-05-13 23:44:55 +00:00
if ( nested - > addValueToColumn ( data , it2 ) )
were_valid_elements = true ;
2019-03-14 02:55:04 +00:00
else
2019-05-13 23:44:55 +00:00
data . insertDefault ( ) ;
2019-03-14 02:55:04 +00:00
}
2019-05-17 14:21:37 +00:00
while ( JSONParser : : nextArrayElement ( it2 ) ) ;
2019-05-13 23:44:55 +00:00
if ( ! were_valid_elements )
2019-03-14 02:55:04 +00:00
{
2019-05-13 23:44:55 +00:00
data . popBack ( data . size ( ) - old_size ) ;
return false ;
2019-03-14 02:55:04 +00:00
}
2019-05-13 23:44:55 +00:00
col_arr . getOffsets ( ) . push_back ( data . size ( ) ) ;
return true ;
2019-03-14 02:55:04 +00:00
}
2019-05-13 23:44:55 +00:00
private :
std : : unique_ptr < Node > nested ;
} ;
2019-05-07 23:31:35 +00:00
2019-05-13 23:44:55 +00:00
class TupleNode : public Node
{
public :
TupleNode ( std : : vector < std : : unique_ptr < Node > > nested_ , const std : : vector < String > & explicit_names_ ) : nested ( std : : move ( nested_ ) ) , explicit_names ( explicit_names_ )
{
for ( size_t i = 0 ; i ! = explicit_names . size ( ) ; + + i )
name_to_index_map . emplace ( explicit_names [ i ] , i ) ;
}
2019-05-07 23:31:35 +00:00
2019-05-13 23:44:55 +00:00
bool addValueToColumn ( IColumn & dest , const Iterator & it ) override
{
ColumnTuple & tuple = static_cast < ColumnTuple & > ( dest ) ;
size_t old_size = dest . size ( ) ;
bool were_valid_elements = false ;
2019-05-07 23:31:35 +00:00
2019-05-13 23:44:55 +00:00
auto set_size = [ & ] ( size_t size )
{
for ( size_t i = 0 ; i ! = tuple . tupleSize ( ) ; + + i )
{
auto & col = tuple . getColumn ( i ) ;
if ( col . size ( ) ! = size )
{
if ( col . size ( ) > size )
col . popBack ( col . size ( ) - size ) ;
else
while ( col . size ( ) < size )
col . insertDefault ( ) ;
}
}
} ;
2019-05-07 23:31:35 +00:00
2019-05-13 23:44:55 +00:00
if ( JSONParser : : isArray ( it ) )
{
Iterator it2 = it ;
2019-05-17 14:21:37 +00:00
if ( ! JSONParser : : firstArrayElement ( it2 ) )
2019-05-13 23:44:55 +00:00
return false ;
2019-05-07 23:31:35 +00:00
2019-05-13 23:44:55 +00:00
size_t index = 0 ;
do
{
if ( nested [ index ] - > addValueToColumn ( tuple . getColumn ( index ) , it2 ) )
were_valid_elements = true ;
else
tuple . getColumn ( index ) . insertDefault ( ) ;
+ + index ;
}
2019-05-17 14:21:37 +00:00
while ( JSONParser : : nextArrayElement ( it2 ) ) ;
2019-05-13 23:44:55 +00:00
set_size ( old_size + static_cast < size_t > ( were_valid_elements ) ) ;
return were_valid_elements ;
}
if ( JSONParser : : isObject ( it ) )
{
if ( name_to_index_map . empty ( ) )
{
Iterator it2 = it ;
2019-05-17 14:21:37 +00:00
if ( ! JSONParser : : firstObjectMember ( it2 ) )
2019-05-13 23:44:55 +00:00
return false ;
size_t index = 0 ;
do
{
if ( nested [ index ] - > addValueToColumn ( tuple . getColumn ( index ) , it2 ) )
were_valid_elements = true ;
else
tuple . getColumn ( index ) . insertDefault ( ) ;
+ + index ;
}
2019-05-17 14:21:37 +00:00
while ( JSONParser : : nextObjectMember ( it2 ) ) ;
2019-05-13 23:44:55 +00:00
}
else
{
Iterator it2 = it ;
StringRef key ;
2019-05-17 14:21:37 +00:00
if ( ! JSONParser : : firstObjectMember ( it2 , key ) )
2019-05-13 23:44:55 +00:00
return false ;
do
{
auto index = name_to_index_map . find ( key ) ;
if ( index ! = name_to_index_map . end ( ) )
{
if ( nested [ index - > second ] - > addValueToColumn ( tuple . getColumn ( index - > second ) , it2 ) )
were_valid_elements = true ;
}
}
2019-05-17 14:21:37 +00:00
while ( JSONParser : : nextObjectMember ( it2 , key ) ) ;
2019-05-13 23:44:55 +00:00
}
set_size ( old_size + static_cast < size_t > ( were_valid_elements ) ) ;
return were_valid_elements ;
}
return false ;
}
private :
std : : vector < std : : unique_ptr < Node > > nested ;
std : : vector < String > explicit_names ;
std : : unordered_map < StringRef , size_t > name_to_index_map ;
} ;
2019-05-15 23:56:10 +00:00
static std : : unique_ptr < Node > build ( const char * function_name , const DataTypePtr & type )
2019-05-07 23:31:35 +00:00
{
2019-05-13 23:44:55 +00:00
switch ( type - > getTypeId ( ) )
{
case TypeIndex : : UInt8 : return std : : make_unique < NumericNode < UInt8 > > ( ) ;
case TypeIndex : : UInt16 : return std : : make_unique < NumericNode < UInt16 > > ( ) ;
case TypeIndex : : UInt32 : return std : : make_unique < NumericNode < UInt32 > > ( ) ;
case TypeIndex : : UInt64 : return std : : make_unique < NumericNode < UInt64 > > ( ) ;
case TypeIndex : : Int8 : return std : : make_unique < NumericNode < Int8 > > ( ) ;
case TypeIndex : : Int16 : return std : : make_unique < NumericNode < Int16 > > ( ) ;
case TypeIndex : : Int32 : return std : : make_unique < NumericNode < Int32 > > ( ) ;
case TypeIndex : : Int64 : return std : : make_unique < NumericNode < Int64 > > ( ) ;
case TypeIndex : : Float32 : return std : : make_unique < NumericNode < Float32 > > ( ) ;
case TypeIndex : : Float64 : return std : : make_unique < NumericNode < Float64 > > ( ) ;
case TypeIndex : : String : return std : : make_unique < StringNode > ( ) ;
case TypeIndex : : FixedString : return std : : make_unique < FixedStringNode > ( ) ;
2019-05-15 22:00:05 +00:00
case TypeIndex : : Enum8 :
return std : : make_unique < EnumNode < Int8 > > ( static_cast < const DataTypeEnum8 & > ( * type ) . getValues ( ) ) ;
case TypeIndex : : Enum16 :
return std : : make_unique < EnumNode < Int16 > > ( static_cast < const DataTypeEnum16 & > ( * type ) . getValues ( ) ) ;
case TypeIndex : : Nullable :
{
2019-05-15 23:56:10 +00:00
return std : : make_unique < NullableNode > ( build ( function_name , static_cast < const DataTypeNullable & > ( * type ) . getNestedType ( ) ) ) ;
2019-05-15 22:00:05 +00:00
}
case TypeIndex : : Array :
{
2019-05-15 23:56:10 +00:00
return std : : make_unique < ArrayNode > ( build ( function_name , static_cast < const DataTypeArray & > ( * type ) . getNestedType ( ) ) ) ;
2019-05-15 22:00:05 +00:00
}
2019-05-13 23:44:55 +00:00
case TypeIndex : : Tuple :
{
const auto & tuple = static_cast < const DataTypeTuple & > ( * type ) ;
const auto & tuple_elements = tuple . getElements ( ) ;
std : : vector < std : : unique_ptr < Node > > elements ;
for ( const auto & tuple_element : tuple_elements )
2019-05-15 23:56:10 +00:00
elements . emplace_back ( build ( function_name , tuple_element ) ) ;
2019-05-13 23:44:55 +00:00
return std : : make_unique < TupleNode > ( std : : move ( elements ) , tuple . haveExplicitNames ( ) ? tuple . getElementNames ( ) : Strings { } ) ;
}
default :
2019-05-15 22:00:05 +00:00
throw Exception { " Function " + String ( function_name ) + " doesn't support the return type schema: " + type - > getName ( ) , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT } ;
2019-05-13 23:44:55 +00:00
}
2019-05-07 23:31:35 +00:00
}
2019-05-15 23:56:10 +00:00
} ;
template < typename JSONParser >
class JSONExtractImpl
{
public :
static constexpr size_t num_extra_arguments = 1 ;
static DataTypePtr getType ( const char * function_name , const ColumnsWithTypeAndName & arguments )
{
if ( arguments . size ( ) < 2 )
throw Exception { " Function " + String ( function_name ) + " requires at least two arguments " , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH } ;
const auto & col = arguments . back ( ) ;
auto col_type_const = typeid_cast < const ColumnConst * > ( col . column . get ( ) ) ;
if ( ! col_type_const | | ! isString ( col . type ) )
throw Exception { " The last argument of function " + String ( function_name )
+ " should be a constant string specifying the return data type, illegal value: " + col . column - > getName ( ) ,
ErrorCodes : : ILLEGAL_COLUMN } ;
return DataTypeFactory : : instance ( ) . get ( col_type_const - > getValue < String > ( ) ) ;
}
void prepare ( const char * function_name , const Block & block , const ColumnNumbers & , size_t result_pos )
{
extract_tree = JSONExtractTree < JSONParser > : : build ( function_name , block . getByPosition ( result_pos ) . type ) ;
}
using Iterator = typename JSONParser : : Iterator ;
bool addValueToColumn ( IColumn & dest , const Iterator & it )
{
return extract_tree - > addValueToColumn ( dest , it ) ;
}
protected :
std : : unique_ptr < typename JSONExtractTree < JSONParser > : : Node > extract_tree ;
} ;
template < typename JSONParser >
class JSONExtractKeysAndValuesImpl
{
public :
static constexpr size_t num_extra_arguments = 1 ;
static DataTypePtr getType ( const char * function_name , const ColumnsWithTypeAndName & arguments )
{
if ( arguments . size ( ) < 2 )
throw Exception { " Function " + String ( function_name ) + " requires at least two arguments " , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH } ;
const auto & col = arguments . back ( ) ;
auto col_type_const = typeid_cast < const ColumnConst * > ( col . column . get ( ) ) ;
if ( ! col_type_const | | ! isString ( col . type ) )
throw Exception { " The last argument of function " + String ( function_name )
+ " should be a constant string specifying the values' data type, illegal value: " + col . column - > getName ( ) ,
ErrorCodes : : ILLEGAL_COLUMN } ;
DataTypePtr value_type = DataTypeFactory : : instance ( ) . get ( col_type_const - > getValue < String > ( ) ) ;
DataTypePtr key_type = std : : make_unique < DataTypeString > ( ) ;
DataTypePtr tuple_type = std : : make_unique < DataTypeTuple > ( DataTypes { key_type , value_type } ) ;
return std : : make_unique < DataTypeArray > ( tuple_type ) ;
}
void prepare ( const char * function_name , const Block & block , const ColumnNumbers & , size_t result_pos )
{
const auto & result_type = block . getByPosition ( result_pos ) . type ;
const auto tuple_type = typeid_cast < const DataTypeArray * > ( result_type . get ( ) ) - > getNestedType ( ) ;
const auto value_type = typeid_cast < const DataTypeTuple * > ( tuple_type . get ( ) ) - > getElements ( ) [ 1 ] ;
extract_tree = JSONExtractTree < JSONParser > : : build ( function_name , value_type ) ;
}
using Iterator = typename JSONParser : : Iterator ;
bool addValueToColumn ( IColumn & dest , const Iterator & it )
{
if ( ! JSONParser : : isObject ( it ) )
return false ;
auto & col_arr = static_cast < ColumnArray & > ( dest ) ;
auto & col_tuple = static_cast < ColumnTuple & > ( col_arr . getData ( ) ) ;
size_t old_size = col_tuple . size ( ) ;
auto & col_key = static_cast < ColumnString & > ( col_tuple . getColumn ( 0 ) ) ;
auto & col_value = col_tuple . getColumn ( 1 ) ;
StringRef key ;
Iterator it2 = it ;
2019-05-17 14:21:37 +00:00
if ( ! JSONParser : : firstObjectMember ( it2 , key ) )
2019-05-15 23:56:10 +00:00
return false ;
do
{
if ( extract_tree - > addValueToColumn ( col_value , it2 ) )
col_key . insertData ( key . data , key . size ) ;
}
2019-05-17 14:21:37 +00:00
while ( JSONParser : : nextObjectMember ( it2 , key ) ) ;
2019-05-15 23:56:10 +00:00
if ( col_tuple . size ( ) = = old_size )
return false ;
col_arr . getOffsets ( ) . push_back ( col_tuple . size ( ) ) ;
return true ;
}
private :
std : : unique_ptr < typename JSONExtractTree < JSONParser > : : Node > extract_tree ;
} ;
template < typename JSONParser >
class JSONExtractRawImpl
{
public :
static DataTypePtr getType ( const char * , const ColumnsWithTypeAndName & )
{
return std : : make_shared < DataTypeString > ( ) ;
}
using Iterator = typename JSONParser : : Iterator ;
static bool addValueToColumn ( IColumn & dest , const Iterator & it )
{
ColumnString & col_str = static_cast < ColumnString & > ( dest ) ;
auto & chars = col_str . getChars ( ) ;
WriteBufferFromVector < ColumnString : : Chars > buf ( chars , WriteBufferFromVector < ColumnString : : Chars > : : AppendModeTag ( ) ) ;
traverse ( it , buf ) ;
buf . finish ( ) ;
chars . push_back ( 0 ) ;
col_str . getOffsets ( ) . push_back ( chars . size ( ) ) ;
return true ;
}
static constexpr size_t num_extra_arguments = 0 ;
static void prepare ( const char * , const Block & , const ColumnNumbers & , size_t ) { }
private :
static void traverse ( const Iterator & it , WriteBuffer & buf )
{
2019-05-16 18:22:59 +00:00
if ( JSONParser : : isInt64 ( it ) )
{
writeIntText ( JSONParser : : getInt64 ( it ) , buf ) ;
return ;
}
if ( JSONParser : : isUInt64 ( it ) )
2019-05-15 23:56:10 +00:00
{
2019-05-16 18:22:59 +00:00
writeIntText ( JSONParser : : getUInt64 ( it ) , buf ) ;
2019-05-15 23:56:10 +00:00
return ;
}
2019-05-16 18:22:59 +00:00
if ( JSONParser : : isDouble ( it ) )
2019-05-15 23:56:10 +00:00
{
2019-05-16 18:22:59 +00:00
writeFloatText ( JSONParser : : getDouble ( it ) , buf ) ;
2019-05-15 23:56:10 +00:00
return ;
}
if ( JSONParser : : isBool ( it ) )
{
if ( JSONParser : : getBool ( it ) )
writeCString ( " true " , buf ) ;
else
writeCString ( " false " , buf ) ;
return ;
}
if ( JSONParser : : isString ( it ) )
{
writeJSONString ( JSONParser : : getString ( it ) , buf , format_settings ( ) ) ;
return ;
}
if ( JSONParser : : isArray ( it ) )
{
writeChar ( ' [ ' , buf ) ;
Iterator it2 = it ;
2019-05-17 14:21:37 +00:00
if ( JSONParser : : firstArrayElement ( it2 ) )
2019-05-15 23:56:10 +00:00
{
traverse ( it2 , buf ) ;
2019-05-17 14:21:37 +00:00
while ( JSONParser : : nextArrayElement ( it2 ) )
2019-05-15 23:56:10 +00:00
{
writeChar ( ' , ' , buf ) ;
traverse ( it2 , buf ) ;
}
}
writeChar ( ' ] ' , buf ) ;
return ;
}
if ( JSONParser : : isObject ( it ) )
{
writeChar ( ' { ' , buf ) ;
Iterator it2 = it ;
StringRef key ;
2019-05-17 14:21:37 +00:00
if ( JSONParser : : firstObjectMember ( it2 , key ) )
2019-05-15 23:56:10 +00:00
{
writeJSONString ( key , buf , format_settings ( ) ) ;
writeChar ( ' : ' , buf ) ;
traverse ( it2 , buf ) ;
2019-05-17 14:21:37 +00:00
while ( JSONParser : : nextObjectMember ( it2 , key ) )
2019-05-15 23:56:10 +00:00
{
writeChar ( ' , ' , buf ) ;
writeJSONString ( key , buf , format_settings ( ) ) ;
writeChar ( ' : ' , buf ) ;
traverse ( it2 , buf ) ;
}
}
writeChar ( ' } ' , buf ) ;
return ;
}
if ( JSONParser : : isNull ( it ) )
{
writeCString ( " null " , buf ) ;
return ;
}
}
2019-05-13 23:44:55 +00:00
2019-05-15 23:56:10 +00:00
static const FormatSettings & format_settings ( )
{
static const FormatSettings the_instance = [ ]
{
FormatSettings settings ;
settings . json . escape_forward_slashes = false ;
return settings ;
} ( ) ;
return the_instance ;
}
2019-05-07 23:31:35 +00:00
} ;
}