Merge pull request #56741 from ClickHouse/well-known-protos

This commit is contained in:
János Benjamin Antal 2023-11-28 15:21:15 +01:00 committed by GitHub
commit 4c2451f837
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 150 additions and 186 deletions

View File

@ -385,9 +385,25 @@ endif ()
include("${ClickHouse_SOURCE_DIR}/contrib/google-protobuf-cmake/protobuf_generate.cmake")
# These files needs to be installed to make it possible that users can use well-known protobuf types
set(google_proto_files
${protobuf_source_dir}/src/google/protobuf/any.proto
${protobuf_source_dir}/src/google/protobuf/api.proto
${protobuf_source_dir}/src/google/protobuf/descriptor.proto
${protobuf_source_dir}/src/google/protobuf/duration.proto
${protobuf_source_dir}/src/google/protobuf/empty.proto
${protobuf_source_dir}/src/google/protobuf/field_mask.proto
${protobuf_source_dir}/src/google/protobuf/source_context.proto
${protobuf_source_dir}/src/google/protobuf/struct.proto
${protobuf_source_dir}/src/google/protobuf/timestamp.proto
${protobuf_source_dir}/src/google/protobuf/type.proto
${protobuf_source_dir}/src/google/protobuf/wrappers.proto
)
add_library(_protobuf INTERFACE)
target_link_libraries(_protobuf INTERFACE _libprotobuf)
target_include_directories(_protobuf INTERFACE "${Protobuf_INCLUDE_DIR}")
set_target_properties(_protobuf PROPERTIES google_proto_files "${google_proto_files}")
add_library(ch_contrib::protobuf ALIAS _protobuf)
add_library(_protoc INTERFACE)

View File

@ -44,6 +44,8 @@ contents:
dst: /usr/bin/clickhouse-odbc-bridge
- src: root/usr/share/bash-completion/completions
dst: /usr/share/bash-completion/completions
- src: root/usr/share/clickhouse
dst: /usr/share/clickhouse
# docs
- src: ../AUTHORS
dst: /usr/share/doc/clickhouse-common-static/AUTHORS

View File

@ -457,3 +457,10 @@ endif()
if (ENABLE_FUZZING)
add_compile_definitions(FUZZING_MODE=1)
endif ()
if (TARGET ch_contrib::protobuf)
get_property(google_proto_files TARGET ch_contrib::protobuf PROPERTY google_proto_files)
foreach (proto_file IN LISTS google_proto_files)
install(FILES ${proto_file} DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse/protos/google/protobuf)
endforeach()
endif ()

View File

@ -306,6 +306,10 @@ void Client::initialize(Poco::Util::Application & self)
/// Set path for format schema files
if (config().has("format_schema_path"))
global_context->setFormatSchemaPath(fs::weakly_canonical(config().getString("format_schema_path")));
/// Set the path for google proto files
if (config().has("google_protos_path"))
global_context->setGoogleProtosPath(fs::weakly_canonical(config().getString("google_protos_path")));
}

View File

@ -37,7 +37,7 @@
<production>{display_name} \e[1;31m:)\e[0m </production> <!-- if it matched to the substring "production" in the server display name -->
</prompt_by_server_display_name>
<!--
<!--
Settings adjustable via command-line parameters
can take their defaults from that config file, see examples:
@ -58,6 +58,9 @@
The same can be done on user-level configuration, just create & adjust: ~/.clickhouse-client/config.xml
-->
<!-- Directory containing the proto files for the well-known Protobuf types.
-->
<google_protos_path>/usr/share/clickhouse/protos/</google_protos_path>
<!-- Analog of .netrc -->
<![CDATA[

View File

@ -1577,6 +1577,10 @@ try
global_context->setFormatSchemaPath(format_schema_path);
fs::create_directories(format_schema_path);
/// Set the path for google proto files
if (config().has("google_protos_path"))
global_context->setGoogleProtosPath(fs::weakly_canonical(config().getString("google_protos_path")));
/// Set path for filesystem caches
fs::path filesystem_caches_path(config().getString("filesystem_caches_path", ""));
if (!filesystem_caches_path.empty())

View File

@ -3,6 +3,7 @@
<tmp_path replace="replace">./tmp/</tmp_path>
<user_files_path replace="replace">./user_files/</user_files_path>
<format_schema_path replace="replace">./format_schemas/</format_schema_path>
<google_protos_path replace="replace">../../contrib/google-protobuf/src/</google_protos_path>
<access_control_path replace="replace">./access/</access_control_path>
<top_level_domains_path replace="replace">./top_level_domains/</top_level_domains_path>
</clickhouse>

View File

@ -1428,6 +1428,10 @@
-->
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
<!-- Directory containing the proto files for the well-known Protobuf types.
-->
<google_protos_path>/usr/share/clickhouse/protos/</google_protos_path>
<!-- Default query masking rules, matching lines would be replaced with something else in the logs
(both text logs and system.query_log).
name - name for the rule (optional)

View File

@ -151,6 +151,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers;
format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference;
format_settings.protobuf.use_autogenerated_schema = settings.format_protobuf_use_autogenerated_schema;
format_settings.protobuf.google_protos_path = context->getGoogleProtosPath();
format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule;
format_settings.regexp.regexp = settings.format_regexp;
format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched;

View File

@ -295,6 +295,7 @@ struct FormatSettings
bool allow_multiple_rows_without_delimiter = false;
bool skip_fields_with_unsupported_types_in_schema_inference = false;
bool use_autogenerated_schema = true;
std::string google_protos_path;
} protobuf;
struct

View File

@ -30,11 +30,11 @@ void ProtobufSchemas::clear()
class ProtobufSchemas::ImporterWithSourceTree : public google::protobuf::compiler::MultiFileErrorCollector
{
public:
explicit ImporterWithSourceTree(const String & schema_directory, WithEnvelope with_envelope_)
: importer(&disk_source_tree, this)
, with_envelope(with_envelope_)
explicit ImporterWithSourceTree(const String & schema_directory, const String & google_protos_path, WithEnvelope with_envelope_)
: importer(&disk_source_tree, this), with_envelope(with_envelope_)
{
disk_source_tree.MapPath("", schema_directory);
disk_source_tree.MapPath("", google_protos_path);
}
~ImporterWithSourceTree() override = default;
@ -112,12 +112,17 @@ private:
};
const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo & info, WithEnvelope with_envelope)
const google::protobuf::Descriptor *
ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo & info, WithEnvelope with_envelope, const String & google_protos_path)
{
std::lock_guard lock(mutex);
auto it = importers.find(info.schemaDirectory());
if (it == importers.end())
it = importers.emplace(info.schemaDirectory(), std::make_unique<ImporterWithSourceTree>(info.schemaDirectory(), with_envelope)).first;
it = importers
.emplace(
info.schemaDirectory(),
std::make_unique<ImporterWithSourceTree>(info.schemaDirectory(), google_protos_path, with_envelope))
.first;
auto * importer = it->second.get();
return importer->import(info.schemaPath(), info.messageName());
}

View File

@ -59,7 +59,8 @@ public:
/// Parses the format schema, then parses the corresponding proto file, and returns the descriptor of the message type.
/// The function never returns nullptr, it throws an exception if it cannot load or parse the file.
const google::protobuf::Descriptor * getMessageTypeForFormatSchema(const FormatSchemaInfo & info, WithEnvelope with_envelope);
const google::protobuf::Descriptor *
getMessageTypeForFormatSchema(const FormatSchemaInfo & info, WithEnvelope with_envelope, const String & google_protos_path);
private:
class ImporterWithSourceTree;

View File

@ -328,6 +328,7 @@ struct ContextSharedPart : boost::noncopyable
std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
/// No lock required for format_schema_path modified only during initialization
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
String google_protos_path; /// Path to a directory that contains the proto files for the well-known Protobuf types.
mutable OnceFlag action_locks_manager_initialized;
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
OnceFlag system_logs_initialized;
@ -4141,6 +4142,16 @@ void Context::setFormatSchemaPath(const String & path)
shared->format_schema_path = path;
}
String Context::getGoogleProtosPath() const
{
return shared->google_protos_path;
}
void Context::setGoogleProtosPath(const String & path)
{
shared->google_protos_path = path;
}
Context::SampleBlockCache & Context::getSampleBlockCache() const
{
assert(hasQueryContext());

View File

@ -1147,6 +1147,10 @@ public:
String getFormatSchemaPath() const;
void setFormatSchemaPath(const String & path);
/// Path to the folder containing the proto files for the well-known Protobuf types
String getGoogleProtosPath() const;
void setGoogleProtosPath(const String & path);
SampleBlockCache & getSampleBlockCache() const;
/// Query parameters for prepared statements.

View File

@ -15,18 +15,20 @@ ProtobufListInputFormat::ProtobufListInputFormat(
const Block & header_,
const Params & params_,
const ProtobufSchemaInfo & schema_info_,
bool flatten_google_wrappers_)
bool flatten_google_wrappers_,
const String & google_protos_path)
: IRowInputFormat(header_, in_, params_)
, reader(std::make_unique<ProtobufReader>(in_))
, serializer(ProtobufSerializer::create(
header_.getNames(),
header_.getDataTypes(),
missing_column_indices,
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
flatten_google_wrappers_,
*reader))
header_.getNames(),
header_.getDataTypes(),
missing_column_indices,
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(
schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes, google_protos_path),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
flatten_google_wrappers_,
*reader))
{
}
@ -83,33 +85,33 @@ size_t ProtobufListInputFormat::countRows(size_t max_block_size)
ProtobufListSchemaReader::ProtobufListSchemaReader(const FormatSettings & format_settings)
: schema_info(
format_settings.schema.format_schema,
"Protobuf",
true,
format_settings.schema.is_server,
format_settings.schema.format_schema_path)
format_settings.schema.format_schema, "Protobuf", true, format_settings.schema.is_server, format_settings.schema.format_schema_path)
, skip_unsopported_fields(format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference)
, google_protos_path(format_settings.protobuf.google_protos_path)
{
}
NamesAndTypesList ProtobufListSchemaReader::readSchema()
{
const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::Yes);
const auto * message_descriptor
= ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::Yes, google_protos_path);
return protobufSchemaToCHSchema(message_descriptor, skip_unsopported_fields);
}
void registerInputFormatProtobufList(FormatFactory & factory)
{
factory.registerInputFormat(
"ProtobufList",
[](ReadBuffer &buf,
const Block & sample,
RowInputFormatParams params,
const FormatSettings & settings)
{
return std::make_shared<ProtobufListInputFormat>(buf, sample, std::move(params),
ProtobufSchemaInfo(settings, "Protobuf", sample, settings.protobuf.use_autogenerated_schema), settings.protobuf.input_flatten_google_wrappers);
});
"ProtobufList",
[](ReadBuffer & buf, const Block & sample, RowInputFormatParams params, const FormatSettings & settings)
{
return std::make_shared<ProtobufListInputFormat>(
buf,
sample,
std::move(params),
ProtobufSchemaInfo(settings, "Protobuf", sample, settings.protobuf.use_autogenerated_schema),
settings.protobuf.input_flatten_google_wrappers,
settings.protobuf.google_protos_path);
});
factory.markFormatSupportsSubsetOfColumns("ProtobufList");
factory.registerAdditionalInfoForSchemaCacheGetter(
"ProtobufList",

View File

@ -29,7 +29,8 @@ public:
const Block & header_,
const Params & params_,
const ProtobufSchemaInfo & schema_info_,
bool flatten_google_wrappers_);
bool flatten_google_wrappers_,
const String & google_protos_path);
String getName() const override { return "ProtobufListInputFormat"; }
@ -56,6 +57,7 @@ public:
private:
const FormatSchemaInfo schema_info;
bool skip_unsopported_fields;
const String google_protos_path;
};
}

View File

@ -13,13 +13,15 @@ ProtobufListOutputFormat::ProtobufListOutputFormat(
WriteBuffer & out_,
const Block & header_,
const ProtobufSchemaInfo & schema_info_,
bool defaults_for_nullable_google_wrappers_)
bool defaults_for_nullable_google_wrappers_,
const String & google_protos_path)
: IRowOutputFormat(header_, out_)
, writer(std::make_unique<ProtobufWriter>(out))
, serializer(ProtobufSerializer::create(
header_.getNames(),
header_.getDataTypes(),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(
schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes, google_protos_path),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
defaults_for_nullable_google_wrappers_,
@ -49,13 +51,14 @@ void registerOutputFormatProtobufList(FormatFactory & factory)
{
factory.registerOutputFormat(
"ProtobufList",
[](WriteBuffer & buf,
const Block & header,
const FormatSettings & settings)
[](WriteBuffer & buf, const Block & header, const FormatSettings & settings)
{
return std::make_shared<ProtobufListOutputFormat>(
buf, header, ProtobufSchemaInfo(settings, "Protobuf", header, settings.protobuf.use_autogenerated_schema),
settings.protobuf.output_nullables_with_google_wrappers);
buf,
header,
ProtobufSchemaInfo(settings, "Protobuf", header, settings.protobuf.use_autogenerated_schema),
settings.protobuf.output_nullables_with_google_wrappers,
settings.protobuf.google_protos_path);
});
}

View File

@ -27,7 +27,8 @@ public:
WriteBuffer & out_,
const Block & header_,
const ProtobufSchemaInfo & schema_info_,
bool defaults_for_nullable_google_wrappers_);
bool defaults_for_nullable_google_wrappers_,
const String & google_protos_path);
String getName() const override { return "ProtobufListOutputFormat"; }

View File

@ -10,10 +10,17 @@
namespace DB
{
ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_,
const ProtobufSchemaInfo & schema_info_, bool with_length_delimiter_, bool flatten_google_wrappers_)
ProtobufRowInputFormat::ProtobufRowInputFormat(
ReadBuffer & in_,
const Block & header_,
const Params & params_,
const ProtobufSchemaInfo & schema_info_,
bool with_length_delimiter_,
bool flatten_google_wrappers_,
const String & google_protos_path)
: IRowInputFormat(header_, in_, params_)
, message_descriptor(ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No))
, message_descriptor(ProtobufSchemas::instance().getMessageTypeForFormatSchema(
schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No, google_protos_path))
, with_length_delimiter(with_length_delimiter_)
, flatten_google_wrappers(flatten_google_wrappers_)
{
@ -98,34 +105,35 @@ void registerInputFormatProtobuf(FormatFactory & factory)
{
for (bool with_length_delimiter : {false, true})
{
factory.registerInputFormat(with_length_delimiter ? "Protobuf" : "ProtobufSingle", [with_length_delimiter](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<ProtobufRowInputFormat>(buf, sample, std::move(params),
ProtobufSchemaInfo(settings, "Protobuf", sample, settings.protobuf.use_autogenerated_schema),
with_length_delimiter,
settings.protobuf.input_flatten_google_wrappers);
});
factory.registerInputFormat(
with_length_delimiter ? "Protobuf" : "ProtobufSingle",
[with_length_delimiter](ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings)
{
return std::make_shared<ProtobufRowInputFormat>(
buf,
sample,
std::move(params),
ProtobufSchemaInfo(settings, "Protobuf", sample, settings.protobuf.use_autogenerated_schema),
with_length_delimiter,
settings.protobuf.input_flatten_google_wrappers,
settings.protobuf.google_protos_path);
});
factory.markFormatSupportsSubsetOfColumns(with_length_delimiter ? "Protobuf" : "ProtobufSingle");
}
}
ProtobufSchemaReader::ProtobufSchemaReader(const FormatSettings & format_settings)
: schema_info(
format_settings.schema.format_schema,
"Protobuf",
true,
format_settings.schema.is_server, format_settings.schema.format_schema_path)
format_settings.schema.format_schema, "Protobuf", true, format_settings.schema.is_server, format_settings.schema.format_schema_path)
, skip_unsupported_fields(format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference)
, google_protos_path(format_settings.protobuf.google_protos_path)
{
}
NamesAndTypesList ProtobufSchemaReader::readSchema()
{
const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::No);
const auto * message_descriptor
= ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::No, google_protos_path);
return protobufSchemaToCHSchema(message_descriptor, skip_unsupported_fields);
}

View File

@ -35,7 +35,8 @@ public:
const Params & params_,
const ProtobufSchemaInfo & schema_info_,
bool with_length_delimiter_,
bool flatten_google_wrappers_);
bool flatten_google_wrappers_,
const String & google_protos_path);
String getName() const override { return "ProtobufRowInputFormat"; }
@ -71,6 +72,7 @@ public:
private:
const FormatSchemaInfo schema_info;
bool skip_unsupported_fields;
String google_protos_path;
};
}

View File

@ -27,7 +27,8 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat(
, serializer(ProtobufSerializer::create(
header_.getNames(),
header_.getDataTypes(),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(
schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No, settings_.protobuf.google_protos_path),
with_length_delimiter_,
/* with_envelope = */ false,
settings_.protobuf.output_nullables_with_google_wrappers,

View File

@ -14,4 +14,8 @@
<!-- Default timeout is 5 sec. Set it to 10 to avoid tests flakiness with slow builds (debug, tsan) -->
<sync_request_timeout>10</sync_request_timeout>
<!-- Directory containing the proto files for the well-known Protobuf types. -->
<google_protos_path>/usr/share/clickhouse/protos/</google_protos_path>
</config>

View File

@ -90,7 +90,7 @@ hexdump -C $BINARY_FILE_PATH
echo
echo "Decoded with protoc:"
(cd $SCHEMADIR && $PROTOC_BINARY --decode Message "$PROTOBUF_FILE_NAME".proto) < $BINARY_FILE_PATH
(cd $SCHEMADIR && $PROTOC_BINARY --proto_path=. --proto_path=/usr/share/clickhouse/protos --decode Message "$PROTOBUF_FILE_NAME".proto) < $BINARY_FILE_PATH
echo
echo "Proto message with wrapper for (NULL, 1), ('', 2), ('str', 3):"

View File

@ -1,6 +1,6 @@
syntax = "proto3";
import "wrappers.proto";
import "google/protobuf/wrappers.proto";
message Message {
google.protobuf.StringValue str = 1;

View File

@ -1,123 +0,0 @@
// Protocol Buffers - Google's data interchange format
// Copyright 2008 Google Inc. All rights reserved.
// https://developers.google.com/protocol-buffers/
//
// Redistribution and use in source and binary forms, with or without
// modification, are permitted provided that the following conditions are
// met:
//
// * Redistributions of source code must retain the above copyright
// notice, this list of conditions and the following disclaimer.
// * Redistributions in binary form must reproduce the above
// copyright notice, this list of conditions and the following disclaimer
// in the documentation and/or other materials provided with the
// distribution.
// * Neither the name of Google Inc. nor the names of its
// contributors may be used to endorse or promote products derived from
// this software without specific prior written permission.
//
// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
// Wrappers for primitive (non-message) types. These types are useful
// for embedding primitives in the `google.protobuf.Any` type and for places
// where we need to distinguish between the absence of a primitive
// typed field and its default value.
//
// These wrappers have no meaningful use within repeated fields as they lack
// the ability to detect presence on individual elements.
// These wrappers have no meaningful use within a map or a oneof since
// individual entries of a map or fields of a oneof can already detect presence.
syntax = "proto3";
package google.protobuf;
option csharp_namespace = "Google.Protobuf.WellKnownTypes";
option cc_enable_arenas = true;
option go_package = "google.golang.org/protobuf/types/known/wrapperspb";
option java_package = "com.google.protobuf";
option java_outer_classname = "WrappersProto";
option java_multiple_files = true;
option objc_class_prefix = "GPB";
// Wrapper message for `double`.
//
// The JSON representation for `DoubleValue` is JSON number.
message DoubleValue {
// The double value.
double value = 1;
}
// Wrapper message for `float`.
//
// The JSON representation for `FloatValue` is JSON number.
message FloatValue {
// The float value.
float value = 1;
}
// Wrapper message for `int64`.
//
// The JSON representation for `Int64Value` is JSON string.
message Int64Value {
// The int64 value.
int64 value = 1;
}
// Wrapper message for `uint64`.
//
// The JSON representation for `UInt64Value` is JSON string.
message UInt64Value {
// The uint64 value.
uint64 value = 1;
}
// Wrapper message for `int32`.
//
// The JSON representation for `Int32Value` is JSON number.
message Int32Value {
// The int32 value.
int32 value = 1;
}
// Wrapper message for `uint32`.
//
// The JSON representation for `UInt32Value` is JSON number.
message UInt32Value {
// The uint32 value.
uint32 value = 1;
}
// Wrapper message for `bool`.
//
// The JSON representation for `BoolValue` is JSON `true` and `false`.
message BoolValue {
// The bool value.
bool value = 1;
}
// Wrapper message for `string`.
//
// The JSON representation for `StringValue` is JSON string.
message StringValue {
// The string value.
string value = 1;
}
// Wrapper message for `bytes`.
//
// The JSON representation for `BytesValue` is JSON string.
message BytesValue {
// The bytes value.
bytes value = 1;
}