mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Enforce checking read output
This commit is contained in:
parent
41056bb990
commit
54db7c6520
@ -143,7 +143,10 @@ void read(std::string & s, ReadBuffer & in)
|
||||
throw Exception("Too large string size while reading from ZooKeeper", Error::ZMARSHALLINGERROR);
|
||||
|
||||
s.resize(size);
|
||||
in.read(s.data(), size);
|
||||
size_t read_bytes = in.read(s.data(), size);
|
||||
if (read_bytes != static_cast<size_t>(size))
|
||||
throw Exception(
|
||||
Error::ZMARSHALLINGERROR, "Buffer size read from Zookeeper is not big enough. Expected {}. Got {}", size, read_bytes);
|
||||
}
|
||||
|
||||
void read(ACL & acl, ReadBuffer & in)
|
||||
|
@ -67,7 +67,7 @@ void read(std::array<char, N> & s, ReadBuffer & in)
|
||||
read(size, in);
|
||||
if (size != N)
|
||||
throw Exception("Unexpected array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR);
|
||||
in.read(s.data(), N);
|
||||
in.readStrict(s.data(), N);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
size_t readBig(char * to, size_t n) override;
|
||||
[[nodiscard]] size_t readBig(char * to, size_t n) override;
|
||||
|
||||
/// The compressed size of the current block.
|
||||
size_t getSizeCompressed() const
|
||||
|
@ -53,7 +53,7 @@ public:
|
||||
/// we store this offset inside nextimpl_working_buffer_offset.
|
||||
void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) override;
|
||||
|
||||
size_t readBig(char * to, size_t n) override;
|
||||
[[nodiscard]] size_t readBig(char * to, size_t n) override;
|
||||
|
||||
void setProfileCallback(const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE)
|
||||
{
|
||||
|
@ -349,7 +349,7 @@ nuraft::ptr<nuraft::srv_state> KeeperStateManager::read_state()
|
||||
auto buffer_size = content_size - sizeof read_checksum - sizeof version;
|
||||
|
||||
auto state_buf = nuraft::buffer::alloc(buffer_size);
|
||||
read_buf.read(reinterpret_cast<char *>(state_buf->data_begin()), buffer_size);
|
||||
read_buf.readStrict(reinterpret_cast<char *>(state_buf->data_begin()), buffer_size);
|
||||
|
||||
SipHash hash;
|
||||
hash.update(version);
|
||||
|
@ -175,7 +175,7 @@ public:
|
||||
FrontMessageType receiveMessageType()
|
||||
{
|
||||
char type = 0;
|
||||
in->read(type);
|
||||
in->readStrict(type);
|
||||
return static_cast<FrontMessageType>(type);
|
||||
}
|
||||
|
||||
|
@ -47,7 +47,7 @@ void SerializationIPv4::deserializeText(IColumn & column, ReadBuffer & istr, con
|
||||
}
|
||||
|
||||
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
istr.read(buffer, sizeof(buffer) - 1);
|
||||
istr.readStrict(buffer, sizeof(buffer) - 1);
|
||||
UInt32 ipv4_value = 0;
|
||||
|
||||
bool parse_result = parseIPv4(buffer, reinterpret_cast<unsigned char *>(&ipv4_value));
|
||||
@ -90,7 +90,7 @@ void SerializationIPv6::deserializeText(IColumn & column, ReadBuffer & istr, con
|
||||
}
|
||||
|
||||
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
istr.read(buffer, sizeof(buffer) - 1);
|
||||
[[maybe_unused]] size_t read_bytes = istr.read(buffer, sizeof(buffer) - 1);
|
||||
|
||||
std::string ipv6_value(IPV6_BINARY_LENGTH, '\0');
|
||||
|
||||
|
@ -30,7 +30,7 @@ bool MySQLPacketPayloadReadBuffer::nextImpl()
|
||||
"Received packet with payload larger than max_packet_size: {}", payload_length);
|
||||
|
||||
size_t packet_sequence_id = 0;
|
||||
in.read(reinterpret_cast<char &>(packet_sequence_id));
|
||||
in.readStrict(reinterpret_cast<char &>(packet_sequence_id));
|
||||
if (packet_sequence_id != sequence_id)
|
||||
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT,
|
||||
"Received packet with wrong sequence-id: {}. Expected: {}.", packet_sequence_id, static_cast<unsigned int>(sequence_id));
|
||||
|
@ -149,7 +149,7 @@ public:
|
||||
}
|
||||
|
||||
/// Reads a single byte.
|
||||
bool ALWAYS_INLINE read(char & c)
|
||||
[[nodiscard]] bool ALWAYS_INLINE read(char & c)
|
||||
{
|
||||
if (peek(c))
|
||||
{
|
||||
@ -168,7 +168,7 @@ public:
|
||||
}
|
||||
|
||||
/** Reads as many as there are, no more than n bytes. */
|
||||
size_t read(char * to, size_t n)
|
||||
[[nodiscard]] size_t read(char * to, size_t n)
|
||||
{
|
||||
size_t bytes_copied = 0;
|
||||
|
||||
@ -197,10 +197,7 @@ public:
|
||||
* By default - the same as read.
|
||||
* Don't use for small reads.
|
||||
*/
|
||||
virtual size_t readBig(char * to, size_t n)
|
||||
{
|
||||
return read(to, n);
|
||||
}
|
||||
[[nodiscard]] virtual size_t readBig(char * to, size_t n) { return read(to, n); }
|
||||
|
||||
/** Do something to allow faster subsequent call to 'nextImpl' if possible.
|
||||
* It's used for asynchronous readers with double-buffering.
|
||||
|
@ -72,7 +72,7 @@ void TraceCollector::run()
|
||||
UInt8 query_id_size = 0;
|
||||
readBinary(query_id_size, in);
|
||||
query_id.resize(query_id_size);
|
||||
in.read(query_id.data(), query_id_size);
|
||||
in.readStrict(query_id.data(), query_id_size);
|
||||
|
||||
UInt8 trace_size = 0;
|
||||
readIntBinary(trace_size, in);
|
||||
|
@ -44,7 +44,7 @@ void AnnoyIndex<Dist>::deserialize(ReadBuffer& istr)
|
||||
readIntBinary(Base::_seed, istr);
|
||||
readVectorBinary(Base::_roots, istr);
|
||||
Base::_nodes = realloc(Base::_nodes, Base::_s * Base::_n_nodes);
|
||||
istr.read(reinterpret_cast<char*>(Base::_nodes), Base::_s * Base::_n_nodes);
|
||||
istr.readStrict(reinterpret_cast<char *>(Base::_nodes), Base::_s * Base::_n_nodes);
|
||||
|
||||
Base::_fd = 0;
|
||||
// set flags
|
||||
|
@ -59,8 +59,7 @@ void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTr
|
||||
|
||||
for (auto & bloom_filter : bloom_filters)
|
||||
{
|
||||
istr.read(reinterpret_cast<char *>(
|
||||
bloom_filter.getFilter().data()), params.filter_size);
|
||||
istr.readStrict(reinterpret_cast<char *>(bloom_filter.getFilter().data()), params.filter_size);
|
||||
}
|
||||
has_elems = true;
|
||||
}
|
||||
|
@ -96,7 +96,7 @@ void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr, Merg
|
||||
static size_t atom_size = 8;
|
||||
size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size;
|
||||
filter = std::make_shared<BloomFilter>(bytes_size, hash_functions, 0);
|
||||
istr.read(reinterpret_cast<char *>(filter->getFilter().data()), bytes_size);
|
||||
istr.readStrict(reinterpret_cast<char *>(filter->getFilter().data()), bytes_size);
|
||||
}
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user