Skip to content

Commit

Permalink
improve json path compatibility
Browse files Browse the repository at this point in the history
  • Loading branch information
lgbo-ustc committed Apr 12, 2023
1 parent 0417ceb commit 437bfcd
Show file tree
Hide file tree
Showing 8 changed files with 278 additions and 10 deletions.
50 changes: 42 additions & 8 deletions src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp
Original file line number Diff line number Diff line change
@@ -1,9 +1,13 @@
#include <cstdint>
#include <Functions/JSONPath/ASTs/ASTJSONPathMemberAccess.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.h>

#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIdentifier_fwd.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/Lexer.h>
#include <Common/StringUtils/StringUtils.h>


namespace DB
{
Expand All @@ -16,18 +20,48 @@ namespace DB
*/
bool ParserJSONPathMemberAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::Dot)
// There's a specical case, that a path member can begin with number
if (pos->type != TokenType::Dot && pos->type != TokenType::Number)
return false;
if (pos->type != TokenType::Number)
++pos;

++pos;
ASTPtr member_name;

if (pos->type != TokenType::BareWord && pos->type !=TokenType::QuotedIdentifier)
return false;
if (pos->type == TokenType::Number)[[unlikely]]
{
for (const auto * c = pos->begin; c != pos->end; ++c)
{
if (*c == '.' && c == pos->begin)
continue;
if (!isNumericASCII(*c))
{
return false;
}
}
const auto * last_begin = *pos->begin == '.' ? pos->begin + 1 : pos->begin;
const auto * last_end = pos->end;
++pos;

ParserIdentifier name_p;
ASTPtr member_name;
if (!name_p.parse(pos, member_name, expected))
return false;
if (pos.isValid() && pos->type == TokenType::BareWord && pos->begin == last_end)
{
member_name = std::make_shared<ASTIdentifier>(String(last_begin, pos->end));
++pos;
}
else
{
return false;
}
}
else
{
if (pos->type != TokenType::BareWord && pos->type != TokenType::QuotedIdentifier)
return false;

ParserIdentifier name_p;
if (!name_p.parse(pos, member_name, expected))
return false;
}

auto member_access = std::make_shared<ASTJSONPathMemberAccess>();
node = member_access;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
#include "ParserJSONPathMemberSquareBracketAccess.h"
#include <memory>
#include <Functions/JSONPath/ASTs/ASTJSONPathMemberAccess.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ExpressionElementParsers.h>

namespace DB
{
bool ParserJSONPathMemberSquareBracketAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::OpeningSquareBracket)
return false;
++pos;
ASTPtr member_name;
if (pos->type == TokenType::BareWord || pos->type == TokenType::QuotedIdentifier)
{
ParserIdentifier name_p;
if (!name_p.parse(pos, member_name, expected))
return false;
}
else if (pos->type == TokenType::StringLiteral)
{
try
{
ReadBufferFromMemory in(pos->begin, pos->size());
String name;
readQuotedStringWithSQLStyle(name, in);
member_name = std::make_shared<ASTIdentifier>(name);
++pos;
}
catch (const Exception &)
{
return false;
}
}
else
{
return false;
}
if (pos->type != TokenType::ClosingSquareBracket)
{
return false;
}
++pos;
auto member_access = std::make_shared<ASTJSONPathMemberAccess>();
node = member_access;
return tryGetIdentifierNameInto(member_name, member_access->member_name);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#pragma once
#include <Parsers/IParserBase.h>
// cases
// - [ident]
// - ['ident']
// - ["ident"]
namespace DB
{
class ParserJSONPathMemberSquareBracketAccess : public IParserBase
{
private:
const char * getName() const override { return "ParserJSONPathMemberSquareBracketAccess"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
public:
explicit ParserJSONPathMemberSquareBracketAccess() = default;
};
}
3 changes: 3 additions & 0 deletions src/Functions/JSONPath/Parsers/ParserJSONPathQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
#include <Functions/JSONPath/Parsers/ParserJSONPathQuery.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathRoot.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathRange.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathStar.h>

Expand All @@ -19,6 +20,7 @@ bool ParserJSONPathQuery::parseImpl(Pos & pos, ASTPtr & query, Expected & expect
{
query = std::make_shared<ASTJSONPathQuery>();
ParserJSONPathMemberAccess parser_jsonpath_member_access;
ParserJSONPathMemberSquareBracketAccess parser_jsonpath_member_square_bracket_access;
ParserJSONPathRange parser_jsonpath_range;
ParserJSONPathStar parser_jsonpath_star;
ParserJSONPathRoot parser_jsonpath_root;
Expand All @@ -32,6 +34,7 @@ bool ParserJSONPathQuery::parseImpl(Pos & pos, ASTPtr & query, Expected & expect

ASTPtr accessor;
while (parser_jsonpath_member_access.parse(pos, accessor, expected)
|| parser_jsonpath_member_square_bracket_access.parse(pos, accessor, expected)
|| parser_jsonpath_range.parse(pos, accessor, expected)
|| parser_jsonpath_star.parse(pos, accessor, expected))
{
Expand Down
15 changes: 15 additions & 0 deletions utils/local-engine/Functions/FunctionGetJsonObject.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
#include "FunctionGetJsonObject.h"
#include <Functions/FunctionFactory.h>


using DB::Token;
using DB::TokenType;

namespace local_engine
{

REGISTER_FUNCTION(GetJsonObject)
{
factory.registerFunction<DB::FunctionSQLJSON<GetJsonOject, GetJsonObjectImpl>>();
}
}
148 changes: 148 additions & 0 deletions utils/local-engine/Functions/FunctionGetJsonObject.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,148 @@
#pragma once
#include <memory>
#include <string_view>
#include <Columns/ColumnNullable.h>
#include <DataTypes/DataTypeNullable.h>
#include <Functions/FunctionSQLJSON.h>
#include <Parsers/Lexer.h>
#include <Poco/Logger.h>
#include <Common/logger_useful.h>

namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
}
namespace local_engine
{
// We notice that, `get_json_object` have different behavior with `JSON_VALUE/JSON_QUERY`.
// - ('{"x":[{"y":1},{"y":2}]}' '$.x[*].y'), `json_value` return only one element, but `get_json_object` return
// return a list.
// - ('{"x":[{"y":1}]}' '$.x[*].y'), `json_query`'s result is '[1]',
// but `get_json_object`'s result is '1'
//


struct GetJsonOject
{
static constexpr auto name{"get_json_object"};
};

template <typename JSONParser>
class GetJsonObjectImpl
{
public:
using Element = typename JSONParser::Element;

static DB::DataTypePtr getReturnType(const char *, const DB::ColumnsWithTypeAndName &)
{
auto nested_type = std::make_shared<DB::DataTypeString>();
return std::make_shared<DB::DataTypeNullable>(nested_type);
}

static size_t getNumberOfIndexArguments(const DB::ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }

bool insertResultToColumn(DB::IColumn & dest, const Element & root, DB::ASTPtr & query_ptr)
{
if (!(has_array_wildcard_flag & 0x01)) [[unlikely]]
{
setupArrayWildcardFlag(query_ptr);
}
DB::GeneratorJSONPath<JSONParser> generator_json_path(query_ptr);
Element current_element = root;
DB::VisitorStatus status;
std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
/// Create json array of results: [res1, res2, ...]
bool success = false;
size_t element_count = 0;
out << "[";
while ((status = generator_json_path.getNextItem(current_element)) != DB::VisitorStatus::Exhausted)
{
if (status == DB::VisitorStatus::Ok)
{
if (success)
{
out << ", ";
}
success = true;
element_count++;
out << current_element.getElement();
}
else if (status == DB::VisitorStatus::Error)
{
/// ON ERROR
/// Here it is possible to handle errors with ON ERROR (as described in ISO/IEC TR 19075-6),
/// however this functionality is not implemented yet
}
current_element = root;
}
out << "]";
if (!success)
{
return false;
}
DB::ColumnNullable & col_str = assert_cast<DB::ColumnNullable &>(dest);
auto output_str = out.str();
std::string_view final_out_str;
assert(elelement_count);
if (element_count == 1)
{
std::string_view output_str_view(output_str.data() + 1, output_str.size() - 2);
if (output_str_view.size() >= 2 && output_str_view.front() == '\"' && output_str_view.back() == '\"')
{
final_out_str = std::string_view(output_str_view.data() + 1, output_str_view.size() - 2);
}
else
final_out_str = std::string_view(output_str);
}
else
{
final_out_str = std::string_view(output_str);
}
col_str.insertData(final_out_str.data(), final_out_str.size());
return true;
}
private:
UInt8 has_array_wildcard_flag = 0;

void setupArrayWildcardFlag(DB::ASTPtr & query_ptr)
{
has_array_wildcard_flag |= 0x01;
const auto * path = query_ptr->as<DB::ASTJSONPath>();
if (!path)
{
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Invalid path");
}
const auto * query = path->jsonpath_query;

for (const auto & child_ast : query->children)
{
if (auto * range_ast = typeid_cast<DB::ASTJSONPathRange *>(child_ast.get()))
{
if (range_ast->is_star)
{
has_array_wildcard_flag |= 0x02;
break;
}
for (const auto & range : range_ast->ranges)
{
if (range.first != range.second - 1)
{
has_array_wildcard_flag |= 0x02;
break;
}
}
}
else if (typeid_cast<DB::ASTJSONPathStar *>(child_ast.get()))
{
has_array_wildcard_flag |= 0x02;
break;
}
}
}
};


}
2 changes: 1 addition & 1 deletion utils/local-engine/Parser/SerializedPlanParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1527,7 +1527,7 @@ const ActionsDAG::Node * SerializedPlanParser::parseFunctionWithDAG(
SerializedPlanParser::parseType(rel.scalar_function().output_type())->getName(),
function_node->result_name);
}
if (function_name == "JSON_VALUE")
if (function_name == "get_json_object")
{
result_node->function->setResolver(function_builder);
}
Expand Down
2 changes: 1 addition & 1 deletion utils/local-engine/Parser/SerializedPlanParser.h
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ static const std::map<std::string, std::string> SCALAR_FUNCTIONS = {
{"posexplode", "arrayJoin"},

// json functions
{"get_json_object", "JSON_VALUE"},
{"get_json_object", "get_json_object"},
{"to_json", "toJSONString"},
{"from_json", "JSONExtract"},
};
Expand Down

0 comments on commit 437bfcd

Please sign in to comment.