[FEAT MERGE] load local files
This commit is contained in:
		
							
								
								
									
										1
									
								
								deps/oblib/src/rpc/CMakeLists.txt
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										1
									
								
								deps/oblib/src/rpc/CMakeLists.txt
									
									
									
									
										vendored
									
									
								
							@ -48,6 +48,7 @@ ob_set_subtarget(oblib_rpc obmysql_packet
 | 
			
		||||
  obmysql/packet/ompk_field.cpp
 | 
			
		||||
  obmysql/packet/ompk_handshake.cpp
 | 
			
		||||
  obmysql/packet/ompk_handshake_response.cpp
 | 
			
		||||
  obmysql/packet/ompk_local_infile.cpp
 | 
			
		||||
  obmysql/packet/ompk_ok.cpp
 | 
			
		||||
  obmysql/packet/ompk_piece.cpp
 | 
			
		||||
  obmysql/packet/ompk_prepare.cpp
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										73
									
								
								deps/oblib/src/rpc/ob_sql_request_operator.h
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										73
									
								
								deps/oblib/src/rpc/ob_sql_request_operator.h
									
									
									
									
										vendored
									
									
								
							@ -19,9 +19,17 @@
 | 
			
		||||
 | 
			
		||||
namespace oceanbase
 | 
			
		||||
{
 | 
			
		||||
namespace obmysql
 | 
			
		||||
{
 | 
			
		||||
class ObICSMemPool;
 | 
			
		||||
class ObSqlSockProcessor;
 | 
			
		||||
} // namespace obmysql
 | 
			
		||||
 | 
			
		||||
namespace rpc
 | 
			
		||||
{
 | 
			
		||||
class ObRequest;
 | 
			
		||||
class ObPacket;
 | 
			
		||||
 | 
			
		||||
struct ObSqlSockDesc
 | 
			
		||||
{
 | 
			
		||||
  ObSqlSockDesc(): type_(0), sock_desc_(NULL) {}
 | 
			
		||||
@ -52,6 +60,49 @@ public:
 | 
			
		||||
  virtual common::ObAddr get_peer(const ObRequest* req) = 0;
 | 
			
		||||
  virtual void disconnect_sql_conn(ObRequest* req) = 0;
 | 
			
		||||
  virtual void finish_sql_request(ObRequest* req) = 0;
 | 
			
		||||
  virtual int create_read_handle(ObRequest* req, void *& read_handle) { return OB_NOT_SUPPORTED; }
 | 
			
		||||
 | 
			
		||||
  /**
 | 
			
		||||
   * read a packet from socket channel
 | 
			
		||||
   *
 | 
			
		||||
   * @param req the `req` received before
 | 
			
		||||
   * @param mem_pool The memory manager to hold the packet returned
 | 
			
		||||
   * @param read_handle Read data from socket by this handle
 | 
			
		||||
   * @param[out] pkt The packet received or null if no message
 | 
			
		||||
   * @return OB_SUCCESS success to read a packet
 | 
			
		||||
   *         Other failed
 | 
			
		||||
   *
 | 
			
		||||
   * read_packet only supported by nio now.
 | 
			
		||||
   * In NIO, the message reader will cache the message data and the packet in
 | 
			
		||||
   * it's memory buffer when receiving a request. The request would be hold in
 | 
			
		||||
   * the memory until the reuqest done.
 | 
			
		||||
   * We can not read new packet during the processing of request if using the
 | 
			
		||||
   * normal method. We need to create a new buffer to hold new packet and keep
 | 
			
		||||
   * the request live.
 | 
			
		||||
   * We should `create_read_handle` before we read new packet and
 | 
			
		||||
   * `release_read_handle` after things done.
 | 
			
		||||
   * You cannot read new packet until the last packet received is dead and you
 | 
			
		||||
   * can use `release_packet` to kill it.
 | 
			
		||||
   *
 | 
			
		||||
   * The whole flow likes below:
 | 
			
		||||
   * 1. get a request from client. (`load data local infile`)
 | 
			
		||||
   * 2. call `create_read_handle`
 | 
			
		||||
   * 3. `read_packet`
 | 
			
		||||
   * 4. `release_packet`
 | 
			
		||||
   * 5. goto 3 if we should read more packets
 | 
			
		||||
   * 6. call `release_read_handle`
 | 
			
		||||
   * 7. finish_request. request (`load data local`) will be released.
 | 
			
		||||
   */
 | 
			
		||||
  virtual int read_packet(ObRequest* req,
 | 
			
		||||
                          obmysql::ObICSMemPool& mem_pool,
 | 
			
		||||
                          void* read_handle,
 | 
			
		||||
                          obmysql::ObSqlSockProcessor& sock_processor,
 | 
			
		||||
                          ObPacket*& pkt)
 | 
			
		||||
  {
 | 
			
		||||
    return OB_NOT_SUPPORTED;
 | 
			
		||||
  }
 | 
			
		||||
  virtual int release_packet(ObRequest* req, void* read_handle, ObPacket* pkt) { return OB_NOT_SUPPORTED; }
 | 
			
		||||
  virtual int release_read_handle(ObRequest* req, void* read_handle) { return OB_NOT_SUPPORTED; }
 | 
			
		||||
  virtual int write_response(ObRequest* req, const char* buf, int64_t sz) = 0;
 | 
			
		||||
  virtual int async_write_response(ObRequest* req, const char* buf, int64_t sz) = 0;
 | 
			
		||||
  virtual void get_sock_desc(ObRequest* req, ObSqlSockDesc& desc) = 0;
 | 
			
		||||
@ -84,6 +135,21 @@ public:
 | 
			
		||||
  void finish_sql_request(ObRequest* req) {
 | 
			
		||||
    return get_operator(req).finish_sql_request(req);
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  int create_read_handle(ObRequest* req, void *& read_handle) {
 | 
			
		||||
    return get_operator(req).create_read_handle(req, read_handle);
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  int release_read_handle(ObRequest* req, void * read_handle) {
 | 
			
		||||
    return get_operator(req).release_read_handle(req, read_handle);
 | 
			
		||||
  }
 | 
			
		||||
  int read_packet(ObRequest* req, obmysql::ObICSMemPool& mem_pool, void *read_handle, ObPacket*& pkt) {
 | 
			
		||||
    return get_operator(req).read_packet(req, mem_pool, read_handle, *sock_processor_, pkt);
 | 
			
		||||
  }
 | 
			
		||||
  int release_packet(ObRequest* req, void* read_handle, ObPacket* pkt) {
 | 
			
		||||
    return get_operator(req).release_packet(req, read_handle, pkt);
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  int write_response(ObRequest* req, const char* buf, int64_t sz) {
 | 
			
		||||
    return get_operator(req).write_response(req, buf, sz);
 | 
			
		||||
  }
 | 
			
		||||
@ -102,9 +168,16 @@ public:
 | 
			
		||||
  void set_sql_session_to_sock_desc(ObRequest* req, void* sess) {
 | 
			
		||||
    return get_operator(req).set_sql_session_to_sock_desc(req, sess);
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  void set_sql_sock_processor(obmysql::ObSqlSockProcessor& sock_processor) {
 | 
			
		||||
    sock_processor_ = &sock_processor;
 | 
			
		||||
  }
 | 
			
		||||
private:
 | 
			
		||||
  ObISqlRequestOperator& get_operator(const ObRequest* req);
 | 
			
		||||
  ObISqlRequestOperator& get_operator(const ObSqlSockDesc& desc);
 | 
			
		||||
 | 
			
		||||
private:
 | 
			
		||||
  obmysql::ObSqlSockProcessor *sock_processor_;
 | 
			
		||||
};
 | 
			
		||||
extern ObSqlRequestOperator global_sql_req_operator;
 | 
			
		||||
#define SQL_REQ_OP (oceanbase::rpc::global_sql_req_operator)
 | 
			
		||||
 | 
			
		||||
@ -410,7 +410,9 @@ inline int Ob20ProtocolProcessor::process_ob20_packet(ObProto20PktContext& conte
 | 
			
		||||
        && pkt20->get_extra_info().exist_extra_info()) {
 | 
			
		||||
      char* tmp_buffer = NULL;
 | 
			
		||||
      int64_t total_len = pkt20->get_extra_info().get_total_len();
 | 
			
		||||
      if (OB_ISNULL(tmp_buffer = reinterpret_cast<char *>(context.arena_.alloc(total_len)))) {
 | 
			
		||||
      if (total_len <= 0) {
 | 
			
		||||
        // empty extra info, do nothing
 | 
			
		||||
      } else if (OB_ISNULL(tmp_buffer = reinterpret_cast<char *>(context.arena_.alloc(total_len)))) {
 | 
			
		||||
        ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
			
		||||
        LOG_ERROR("no memory available", "alloc_size", total_len, K(ret));
 | 
			
		||||
      } else if (OB_FAIL(context.extra_info_.assign(pkt20->get_extra_info(), tmp_buffer, total_len))) {
 | 
			
		||||
@ -440,7 +442,9 @@ inline int Ob20ProtocolProcessor::process_ob20_packet(ObProto20PktContext& conte
 | 
			
		||||
        input_packet->set_proxy_switch_route(pkt20->get_flags().proxy_switch_route());
 | 
			
		||||
        const int64_t t_len = context.extra_info_.get_total_len();
 | 
			
		||||
        char *t_buffer = NULL;
 | 
			
		||||
        if (OB_ISNULL(t_buffer = reinterpret_cast<char *>(pool.alloc(t_len)))) {
 | 
			
		||||
        if (t_len <= 0) {
 | 
			
		||||
          // empty extra info, do nothing
 | 
			
		||||
        } else if (OB_ISNULL(t_buffer = reinterpret_cast<char *>(pool.alloc(t_len)))) {
 | 
			
		||||
          ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
			
		||||
          LOG_ERROR("no memory available", "alloc_size", t_len, K(ret));
 | 
			
		||||
        } else if (OB_FAIL(input_packet->extra_info_.assign(context.extra_info_, t_buffer, t_len))) {
 | 
			
		||||
 | 
			
		||||
@ -556,7 +556,8 @@ inline int ObProto20Utils::fill_proto20_header(ObProtoEncodeParam ¶m) {
 | 
			
		||||
  Ob20ProtocolFlags flag;
 | 
			
		||||
  flag.st_flags_.OB_EXTRA_INFO_EXIST = param.proto20_context_->has_extra_info_;
 | 
			
		||||
 | 
			
		||||
  flag.st_flags_.OB_IS_LAST_PACKET = (ObProto20Utils::is_the_last_packet(param) ? 1 : 0);
 | 
			
		||||
  flag.st_flags_.OB_IS_LAST_PACKET = (ObProto20Utils::is_the_last_packet(param)
 | 
			
		||||
                                      || OB_UNLIKELY(param.proto20_context_->is_filename_packet_) ? 1 : 0);
 | 
			
		||||
  flag.st_flags_.OB_IS_NEW_EXTRA_INFO = proto20_context.is_new_extra_info_;
 | 
			
		||||
  flag.st_flags_.OB_TXN_FREE_ROUTE = proto20_context.txn_free_route_ ? 1 : 0;
 | 
			
		||||
  uint16_t reserved = 0;
 | 
			
		||||
 | 
			
		||||
@ -67,7 +67,8 @@ public:
 | 
			
		||||
      tailer_len_(0), next_step_(START_TO_FILL_STEP),
 | 
			
		||||
      is_proto20_used_(false), is_checksum_off_(false),
 | 
			
		||||
      has_extra_info_(false), is_new_extra_info_(false),
 | 
			
		||||
      curr_proto20_packet_start_pos_(0), txn_free_route_(false) {}
 | 
			
		||||
      curr_proto20_packet_start_pos_(0), txn_free_route_(false),
 | 
			
		||||
      is_filename_packet_(false) {}
 | 
			
		||||
  ~ObProto20Context() {}
 | 
			
		||||
 | 
			
		||||
  inline void reset() { MEMSET(this, 0, sizeof(ObProto20Context)); }
 | 
			
		||||
@ -84,7 +85,8 @@ public:
 | 
			
		||||
                K_(has_extra_info),
 | 
			
		||||
                K_(is_new_extra_info),
 | 
			
		||||
                K_(txn_free_route),
 | 
			
		||||
                K_(curr_proto20_packet_start_pos));
 | 
			
		||||
                K_(curr_proto20_packet_start_pos),
 | 
			
		||||
                K_(is_filename_packet));
 | 
			
		||||
 | 
			
		||||
public:
 | 
			
		||||
  uint8_t comp_seq_;
 | 
			
		||||
@ -99,6 +101,9 @@ public:
 | 
			
		||||
  bool is_new_extra_info_;
 | 
			
		||||
  int64_t curr_proto20_packet_start_pos_;
 | 
			
		||||
  bool txn_free_route_;
 | 
			
		||||
  // used in local local.
 | 
			
		||||
  // We should set `is_filename_packet_` when sending PKT_FILENAME packet.
 | 
			
		||||
  bool is_filename_packet_;
 | 
			
		||||
private:
 | 
			
		||||
  DISALLOW_COPY_AND_ASSIGN(ObProto20Context);
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
@ -198,7 +198,8 @@ inline int ObMysqlCompressProtocolProcessor::process_compressed_packet(
 | 
			
		||||
              pkt_rec_wrapper.record_recieve_comp_packet(*iraw_pkt, *raw_pkt);
 | 
			
		||||
            }
 | 
			
		||||
          }
 | 
			
		||||
          context.reset();
 | 
			
		||||
          // reset all context except the compress packet sequence
 | 
			
		||||
          context.reuse();
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										19
									
								
								deps/oblib/src/rpc/obmysql/ob_mysql_packet.h
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										19
									
								
								deps/oblib/src/rpc/obmysql/ob_mysql_packet.h
									
									
									
									
										vendored
									
									
								
							@ -105,7 +105,8 @@ enum class ObMySQLPacketType
 | 
			
		||||
  PKT_PREPARE,   // 9 -> prepare packet;
 | 
			
		||||
  PKT_RESHEAD,   // 10 -> result header packet
 | 
			
		||||
  PKT_PREXEC,    // 11 -> prepare execute packet;
 | 
			
		||||
  PKT_END        // 12 -> end of packet type
 | 
			
		||||
  PKT_FILENAME,  // 12 -> send file name to client(load local infile)
 | 
			
		||||
  PKT_END        // 13 -> end of packet type
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
union ObServerStatusFlags
 | 
			
		||||
@ -161,6 +162,7 @@ union ObProxyCapabilityFlags
 | 
			
		||||
  bool is_weak_stale_feedback() const { return 1 == cap_flags_.OB_CAP_PROXY_WEAK_STALE_FEEDBACK; }
 | 
			
		||||
  bool is_flt_show_trace_support() const { return 1 == cap_flags_.OB_CAP_PROXY_FULL_LINK_TRACING_EXT
 | 
			
		||||
                                                        && is_ob_protocol_v2_support(); }
 | 
			
		||||
  bool is_load_local_support() const { return 1 == cap_flags_.OB_CAP_LOCAL_FILES; }
 | 
			
		||||
  bool is_client_sessid_support() const { return 1 == cap_flags_.OB_CAP_PROXY_CLIENT_SESSION_ID; }
 | 
			
		||||
 | 
			
		||||
  uint64_t capability_;
 | 
			
		||||
@ -522,6 +524,7 @@ public:
 | 
			
		||||
      is_weak_read_(false),
 | 
			
		||||
      txn_free_route_(false),
 | 
			
		||||
      proxy_switch_route_(false),
 | 
			
		||||
      consume_size_(0),
 | 
			
		||||
      extra_info_()
 | 
			
		||||
  {}
 | 
			
		||||
 | 
			
		||||
@ -551,6 +554,9 @@ public:
 | 
			
		||||
  const common::ObString &get_trace_info() const { return extra_info_.trace_info_; }
 | 
			
		||||
  virtual int64_t get_serialize_size() const;
 | 
			
		||||
 | 
			
		||||
  void set_consume_size(int64_t consume_size) { consume_size_ = consume_size; }
 | 
			
		||||
  int64_t get_consume_size() const { return consume_size_; }
 | 
			
		||||
 | 
			
		||||
  virtual void reset() {
 | 
			
		||||
    ObMySQLPacket::reset();
 | 
			
		||||
    cmd_ = COM_MAX_NUM;
 | 
			
		||||
@ -559,6 +565,7 @@ public:
 | 
			
		||||
    txn_free_route_ = false;
 | 
			
		||||
    proxy_switch_route_ = false;
 | 
			
		||||
    extra_info_.reset();
 | 
			
		||||
    consume_size_ = 0;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  virtual void assign(const ObMySQLRawPacket &other)
 | 
			
		||||
@ -570,10 +577,12 @@ public:
 | 
			
		||||
    txn_free_route_ = other.txn_free_route_;
 | 
			
		||||
    extra_info_ = other.extra_info_;
 | 
			
		||||
    proxy_switch_route_ = other.proxy_switch_route_;
 | 
			
		||||
    consume_size_ = other.consume_size_;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  TO_STRING_KV("header", hdr_, "can_reroute", can_reroute_pkt_, "weak_read", is_weak_read_,
 | 
			
		||||
            "txn_free_route_", txn_free_route_, "proxy_switch_route", proxy_switch_route_);
 | 
			
		||||
            "txn_free_route_", txn_free_route_, "proxy_switch_route", proxy_switch_route_,
 | 
			
		||||
            "consume_size", consume_size_);
 | 
			
		||||
protected:
 | 
			
		||||
  virtual int serialize(char*, const int64_t, int64_t&) const;
 | 
			
		||||
 | 
			
		||||
@ -585,6 +594,12 @@ private:
 | 
			
		||||
  bool is_weak_read_;
 | 
			
		||||
  bool txn_free_route_;
 | 
			
		||||
  bool proxy_switch_route_;
 | 
			
		||||
 | 
			
		||||
  // In load local scenario, we should tell the NIO to consume specific size data.
 | 
			
		||||
  // The size is a packet size in usually. But the mysql packet size if not equal
 | 
			
		||||
  // to the packet that we received if we use ob20 or compress protocol.
 | 
			
		||||
  // NOTE: one ob20 or compress packet has only one mysql packet in request message.
 | 
			
		||||
  int64_t consume_size_;
 | 
			
		||||
public:
 | 
			
		||||
  Ob20ExtraInfo extra_info_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
@ -123,6 +123,12 @@ public:
 | 
			
		||||
    is_multi_pkt_ = false;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  void reuse()
 | 
			
		||||
  {
 | 
			
		||||
    // keep the last_pkt_seq_ here
 | 
			
		||||
    is_multi_pkt_ = false;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  TO_STRING_KV(K_(last_pkt_seq),
 | 
			
		||||
               K_(is_multi_pkt));
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										68
									
								
								deps/oblib/src/rpc/obmysql/ob_packet_record.cpp
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										68
									
								
								deps/oblib/src/rpc/obmysql/ob_packet_record.cpp
									
									
									
									
										vendored
									
									
								
							@ -27,7 +27,7 @@ bool __attribute__((weak)) enable_proto_dia()
 | 
			
		||||
}
 | 
			
		||||
namespace obmysql
 | 
			
		||||
{
 | 
			
		||||
static const char* pkt_type_name[13] =
 | 
			
		||||
static const char* pkt_type_name[14] =
 | 
			
		||||
{
 | 
			
		||||
  "INVALID_PKT",
 | 
			
		||||
  "PKT_MYSQL",     // 1 -> mysql packet;
 | 
			
		||||
@ -41,9 +41,70 @@ static const char* pkt_type_name[13] =
 | 
			
		||||
  "PKT_PREPARE",   // 9 -> prepare packet;
 | 
			
		||||
  "PKT_RESHEAD",   // 10 -> result header packet
 | 
			
		||||
  "PKT_PREXEC",    // 11 -> prepare execute packet;
 | 
			
		||||
  "PKT_END"        // 12 -> end of packet type
 | 
			
		||||
  "PKT_FILENAME",  // 12 -> file name packet(load local infile)
 | 
			
		||||
  "PKT_END"        // 13 -> end of packet type
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
static const char *get_receive_pkt_type_name(const ObpMysqHeader &header)
 | 
			
		||||
{
 | 
			
		||||
#define PKT_TYPE_NAME(command)  case command: { name = #command; } break
 | 
			
		||||
 | 
			
		||||
  const char *name = "UNKNOWN_PKT";
 | 
			
		||||
 | 
			
		||||
  if (header.is_file_content_) {
 | 
			
		||||
    name = "FILE_CONTENT";
 | 
			
		||||
  } else {
 | 
			
		||||
    uint8_t type = header.type_;
 | 
			
		||||
 | 
			
		||||
    switch (type) {
 | 
			
		||||
      PKT_TYPE_NAME(COM_SLEEP);
 | 
			
		||||
      PKT_TYPE_NAME(COM_QUIT);
 | 
			
		||||
      PKT_TYPE_NAME(COM_INIT_DB);
 | 
			
		||||
      PKT_TYPE_NAME(COM_QUERY);
 | 
			
		||||
      PKT_TYPE_NAME(COM_FIELD_LIST);
 | 
			
		||||
      PKT_TYPE_NAME(COM_CREATE_DB);
 | 
			
		||||
      PKT_TYPE_NAME(COM_DROP_DB);
 | 
			
		||||
      PKT_TYPE_NAME(COM_REFRESH);
 | 
			
		||||
      PKT_TYPE_NAME(COM_SHUTDOWN);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STATISTICS);
 | 
			
		||||
      PKT_TYPE_NAME(COM_PROCESS_INFO);
 | 
			
		||||
      PKT_TYPE_NAME(COM_CONNECT);
 | 
			
		||||
      PKT_TYPE_NAME(COM_PROCESS_KILL);
 | 
			
		||||
      PKT_TYPE_NAME(COM_DEBUG);
 | 
			
		||||
      PKT_TYPE_NAME(COM_PING);
 | 
			
		||||
      PKT_TYPE_NAME(COM_TIME);
 | 
			
		||||
      PKT_TYPE_NAME(COM_DELAYED_INSERT);
 | 
			
		||||
      PKT_TYPE_NAME(COM_CHANGE_USER);
 | 
			
		||||
      PKT_TYPE_NAME(COM_BINLOG_DUMP);
 | 
			
		||||
      PKT_TYPE_NAME(COM_TABLE_DUMP);
 | 
			
		||||
      PKT_TYPE_NAME(COM_CONNECT_OUT);
 | 
			
		||||
      PKT_TYPE_NAME(COM_REGISTER_SLAVE);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_PREPARE);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_EXECUTE);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_SEND_LONG_DATA);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_CLOSE);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_RESET);
 | 
			
		||||
      PKT_TYPE_NAME(COM_SET_OPTION);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_FETCH);
 | 
			
		||||
      PKT_TYPE_NAME(COM_DAEMON);
 | 
			
		||||
      PKT_TYPE_NAME(COM_BINLOG_DUMP_GTID);
 | 
			
		||||
      PKT_TYPE_NAME(COM_RESET_CONNECTION);
 | 
			
		||||
      PKT_TYPE_NAME(COM_END);
 | 
			
		||||
      PKT_TYPE_NAME(COM_DELETE_SESSION);
 | 
			
		||||
      PKT_TYPE_NAME(COM_HANDSHAKE);
 | 
			
		||||
      PKT_TYPE_NAME(COM_LOGIN);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_PREXECUTE);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_SEND_PIECE_DATA);
 | 
			
		||||
      PKT_TYPE_NAME(COM_STMT_GET_PIECE_DATA);
 | 
			
		||||
      default: {
 | 
			
		||||
      } break;
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
#undef PKT_TYPE_NAME
 | 
			
		||||
 | 
			
		||||
  return name;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int64_t ObPacketRecord::to_string(char *buf, const int64_t buf_len) const
 | 
			
		||||
{
 | 
			
		||||
  int64_t pos = 0;
 | 
			
		||||
@ -91,7 +152,8 @@ int64_t ObPacketRecord::to_string(char *buf, const int64_t buf_len) const
 | 
			
		||||
         "rec_", obp_mysql_header_.rec_, "seq_", obp_mysql_header_.seq_);
 | 
			
		||||
    J_OBJ_END();
 | 
			
		||||
    J_COMMA();
 | 
			
		||||
    J_KV(K(obp_mysql_header_.type_), K(obp_mysql_header_.is_send_));
 | 
			
		||||
    J_KV("pkt_name", get_receive_pkt_type_name(obp_mysql_header_),
 | 
			
		||||
         K(obp_mysql_header_.type_), K(obp_mysql_header_.is_send_));
 | 
			
		||||
  }
 | 
			
		||||
  J_OBJ_END();
 | 
			
		||||
  return pos;
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										38
									
								
								deps/oblib/src/rpc/obmysql/ob_packet_record.h
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										38
									
								
								deps/oblib/src/rpc/obmysql/ob_packet_record.h
									
									
									
									
										vendored
									
									
								
							@ -72,11 +72,13 @@ struct ObpMysqHeader {
 | 
			
		||||
  uint8_t seq_;
 | 
			
		||||
  uint8_t type_;
 | 
			
		||||
  uint8_t com_seq_; // compress head sequence
 | 
			
		||||
  uint8_t is_send_;
 | 
			
		||||
  uint8_t is_send_:1;
 | 
			
		||||
  uint8_t is_file_content_:1;
 | 
			
		||||
  ObpMysqHeader() {
 | 
			
		||||
    rec_ = 0;
 | 
			
		||||
    seq_ = 0;
 | 
			
		||||
    mysql_header_.len_ = 0;
 | 
			
		||||
    is_file_content_ = 0;
 | 
			
		||||
  }
 | 
			
		||||
  ~ObpMysqHeader() {}
 | 
			
		||||
 | 
			
		||||
@ -168,6 +170,10 @@ public:
 | 
			
		||||
    obp_mysql_header_.type_ = static_cast<uint8_t>(type);
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  inline void set_file_content() __restrict__ {
 | 
			
		||||
    obp_mysql_header_.is_file_content_ = 1;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  int64_t to_string(char *buf, const int64_t buf_len) const;
 | 
			
		||||
  Obp20Header obp20_header_;         // 16 byte
 | 
			
		||||
  ObpMysqHeader obp_mysql_header_;   // 16  byte
 | 
			
		||||
@ -182,6 +188,7 @@ class ObPacketRecordWrapper {
 | 
			
		||||
      cur_pkt_pos_ = 0;
 | 
			
		||||
      last_type_ = obmysql::ObMySQLPacketType::INVALID_PKT;
 | 
			
		||||
      enable_proto_dia_ = false;
 | 
			
		||||
      receiving_file_contents_ = false;
 | 
			
		||||
    }
 | 
			
		||||
    ~ObPacketRecordWrapper() {}
 | 
			
		||||
    void init() {
 | 
			
		||||
@ -189,6 +196,7 @@ class ObPacketRecordWrapper {
 | 
			
		||||
      cur_pkt_pos_ = 0;
 | 
			
		||||
      last_type_ = obmysql::ObMySQLPacketType::INVALID_PKT;
 | 
			
		||||
      enable_proto_dia_ = observer::enable_proto_dia();
 | 
			
		||||
      receiving_file_contents_ = false;
 | 
			
		||||
    }
 | 
			
		||||
    int64_t to_string(char *buf, int64_t buf_len) const;
 | 
			
		||||
 | 
			
		||||
@ -214,6 +222,13 @@ class ObPacketRecordWrapper {
 | 
			
		||||
      rec.record_recieve_obp20_packet(obp20_pkt);
 | 
			
		||||
      rec.record_recieve_mysql_packet(pkt);
 | 
			
		||||
      cur_pkt_pos_++;
 | 
			
		||||
 | 
			
		||||
      if (OB_UNLIKELY(receiving_file_contents_)) {
 | 
			
		||||
        pkt_rec_[idx].set_file_content();
 | 
			
		||||
        if (0 == pkt.get_clen()) {
 | 
			
		||||
          receiving_file_contents_ = false;
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    // for 20 protocol end
 | 
			
		||||
 | 
			
		||||
@ -236,6 +251,13 @@ class ObPacketRecordWrapper {
 | 
			
		||||
      rec.record_recieve_comp_packet(com_pkt);
 | 
			
		||||
      rec.record_recieve_mysql_packet(pkt);
 | 
			
		||||
      cur_pkt_pos_++;
 | 
			
		||||
 | 
			
		||||
      if (OB_UNLIKELY(receiving_file_contents_)) {
 | 
			
		||||
        pkt_rec_[idx].set_file_content();
 | 
			
		||||
        if (0 == pkt.get_clen()) {
 | 
			
		||||
          receiving_file_contents_ = false;
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    // for compress protocol end
 | 
			
		||||
 | 
			
		||||
@ -251,12 +273,23 @@ class ObPacketRecordWrapper {
 | 
			
		||||
      int64_t idx = (cur_pkt_pos_-1) % ObPacketRecordWrapper::REC_BUF_SIZE;
 | 
			
		||||
      pkt_rec_[idx].record_send_mysql_packet(pkt, len);
 | 
			
		||||
      last_type_ = pkt.get_mysql_packet_type();
 | 
			
		||||
 | 
			
		||||
      if (OB_UNLIKELY(pkt.get_mysql_packet_type() == ObMySQLPacketType::PKT_FILENAME)) {
 | 
			
		||||
        receiving_file_contents_ = true;
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    inline void record_recieve_mysql_packet(obmysql::ObMySQLRawPacket &__restrict__ pkt) __restrict__
 | 
			
		||||
    {
 | 
			
		||||
      int64_t idx = cur_pkt_pos_ % ObPacketRecordWrapper::REC_BUF_SIZE;
 | 
			
		||||
      pkt_rec_[idx].record_recieve_mysql_packet(pkt);
 | 
			
		||||
      cur_pkt_pos_++;
 | 
			
		||||
 | 
			
		||||
      if (OB_UNLIKELY(receiving_file_contents_)) {
 | 
			
		||||
        pkt_rec_[idx].set_file_content();
 | 
			
		||||
        if (0 == pkt.get_clen()) {
 | 
			
		||||
          receiving_file_contents_ = false;
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    inline void record_recieve_mysql_pkt_fragment(int32_t recive) __restrict__
 | 
			
		||||
    {
 | 
			
		||||
@ -274,6 +307,9 @@ class ObPacketRecordWrapper {
 | 
			
		||||
    uint32_t cur_pkt_pos_;
 | 
			
		||||
    obmysql::ObMySQLPacketType last_type_;
 | 
			
		||||
    bool enable_proto_dia_;
 | 
			
		||||
    // in load local infile, we will receive some file content packets and there is no `cmd` in the packet.
 | 
			
		||||
    // so we use a flag to mark the context.
 | 
			
		||||
    bool receiving_file_contents_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
@ -13,6 +13,7 @@
 | 
			
		||||
#include "rpc/obmysql/ob_poc_sql_request_operator.h"
 | 
			
		||||
#include "rpc/obmysql/ob_sql_sock_session.h"
 | 
			
		||||
#include "rpc/obrpc/ob_rpc_opts.h"
 | 
			
		||||
#include "rpc/obmysql/ob_sql_sock_processor.h"
 | 
			
		||||
 | 
			
		||||
namespace oceanbase
 | 
			
		||||
{
 | 
			
		||||
@ -72,6 +73,35 @@ void ObPocSqlRequestOperator::finish_sql_request(ObRequest* req)
 | 
			
		||||
  obmysql::request_finish_callback();
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObPocSqlRequestOperator::create_read_handle(ObRequest* req, void*& read_handle)
 | 
			
		||||
{
 | 
			
		||||
  ObSqlSockSession* sess = (ObSqlSockSession*)req->get_server_handle_context();
 | 
			
		||||
  return sess->create_read_handle(read_handle);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObPocSqlRequestOperator::release_read_handle(ObRequest* req, void* read_handle)
 | 
			
		||||
{
 | 
			
		||||
  ObSqlSockSession* sess = (ObSqlSockSession*)req->get_server_handle_context();
 | 
			
		||||
  return sess->release_read_handle(read_handle);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObPocSqlRequestOperator::read_packet(ObRequest* req,
 | 
			
		||||
                                         ObICSMemPool& mem_pool,
 | 
			
		||||
                                         void* read_handle,
 | 
			
		||||
                                         ObSqlSockProcessor &sock_processor,
 | 
			
		||||
                                         ObPacket*& pkt)
 | 
			
		||||
{
 | 
			
		||||
  ObSqlSockSession* sess = (ObSqlSockSession*)req->get_server_handle_context();
 | 
			
		||||
  return sock_processor.decode_sql_packet(mem_pool, *sess, read_handle, pkt);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObPocSqlRequestOperator::release_packet(ObRequest* req, void* read_handle, ObPacket* pkt)
 | 
			
		||||
{
 | 
			
		||||
  ObSqlSockSession* sess = (ObSqlSockSession*)req->get_server_handle_context();
 | 
			
		||||
  obmysql::ObMySQLRawPacket* mysql_packet = static_cast<obmysql::ObMySQLRawPacket*>(pkt);
 | 
			
		||||
  return sess->consume_data(read_handle, mysql_packet->get_consume_size());
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObPocSqlRequestOperator::write_response(ObRequest* req, const char* buf, int64_t sz)
 | 
			
		||||
{
 | 
			
		||||
  ObSqlSockSession* sess = (ObSqlSockSession*)req->get_server_handle_context();
 | 
			
		||||
 | 
			
		||||
@ -31,6 +31,14 @@ public:
 | 
			
		||||
  virtual common::ObAddr get_peer(const rpc::ObRequest* req) override;
 | 
			
		||||
  virtual void disconnect_sql_conn(rpc::ObRequest* req) override;
 | 
			
		||||
  virtual void finish_sql_request(rpc::ObRequest* req) override;
 | 
			
		||||
  virtual int create_read_handle(rpc::ObRequest* req, void*& read_handle) override;
 | 
			
		||||
  virtual int release_read_handle(rpc::ObRequest* req, void* read_handle) override;
 | 
			
		||||
  virtual int read_packet(rpc::ObRequest* req,
 | 
			
		||||
                          ObICSMemPool& mem_pool,
 | 
			
		||||
                          void* read_handle,
 | 
			
		||||
                          ObSqlSockProcessor& sock_processor,
 | 
			
		||||
                          rpc::ObPacket*& pkt) override; // TODO change to mysql packet
 | 
			
		||||
  virtual int release_packet(rpc::ObRequest* req, void* read_handle, rpc::ObPacket* pkt) override;
 | 
			
		||||
  virtual int write_response(rpc::ObRequest* req, const char* buf, int64_t sz) override;
 | 
			
		||||
  virtual int async_write_response(rpc::ObRequest* req, const char* buf, int64_t sz) override;
 | 
			
		||||
  virtual void get_sock_desc(rpc::ObRequest* req, rpc::ObSqlSockDesc& desc) override;
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										174
									
								
								deps/oblib/src/rpc/obmysql/ob_sql_nio.cpp
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										174
									
								
								deps/oblib/src/rpc/obmysql/ob_sql_nio.cpp
									
									
									
									
										vendored
									
									
								
							@ -48,6 +48,8 @@ namespace oceanbase
 | 
			
		||||
{
 | 
			
		||||
namespace obmysql
 | 
			
		||||
{
 | 
			
		||||
static const char *MEMORY_MODEL_NAME = "SqlNio";
 | 
			
		||||
 | 
			
		||||
class ObDList
 | 
			
		||||
{
 | 
			
		||||
 | 
			
		||||
@ -135,13 +137,63 @@ private:
 | 
			
		||||
  int32_t ready_ CACHE_ALIGNED;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
class SocketReader
 | 
			
		||||
{
 | 
			
		||||
public:
 | 
			
		||||
  SocketReader(int fd)
 | 
			
		||||
      : fd_(fd),
 | 
			
		||||
        has_EAGAIN_(false)
 | 
			
		||||
  {}
 | 
			
		||||
 | 
			
		||||
  ~SocketReader()
 | 
			
		||||
  {
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  TO_STRING_KV(K_(fd));
 | 
			
		||||
 | 
			
		||||
  void set_fd(int fd) { fd_ = fd; }
 | 
			
		||||
  int  get_fd() const { return fd_; }
 | 
			
		||||
  bool has_EAGAIN() const { return has_EAGAIN_; }
 | 
			
		||||
  void clear_EAGAIN() { has_EAGAIN_ = false; }
 | 
			
		||||
 | 
			
		||||
  int read(char* buf, int64_t buf_size, int64_t& read_size)
 | 
			
		||||
  {
 | 
			
		||||
    int ret = OB_SUCCESS;
 | 
			
		||||
    int64_t read_ret = ob_read_regard_ssl(fd_, buf, buf_size);
 | 
			
		||||
    if (read_ret > 0) {
 | 
			
		||||
      read_size = read_ret;
 | 
			
		||||
    } else if (0 == read_ret) {
 | 
			
		||||
      LOG_INFO("read fd return EOF", K_(fd));
 | 
			
		||||
      has_EAGAIN_ = true;
 | 
			
		||||
      ret = OB_IO_ERROR;
 | 
			
		||||
    } else if (EAGAIN == errno || EWOULDBLOCK == errno) {
 | 
			
		||||
      has_EAGAIN_ = true;
 | 
			
		||||
    } else if (EINTR == errno) {
 | 
			
		||||
      // pass
 | 
			
		||||
    } else {
 | 
			
		||||
      ret = OB_IO_ERROR;
 | 
			
		||||
      LOG_WARN("read fd has error", K_(fd), K(errno));
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
private:
 | 
			
		||||
  int  fd_;
 | 
			
		||||
  bool has_EAGAIN_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
class ReadBuffer
 | 
			
		||||
{
 | 
			
		||||
public:
 | 
			
		||||
  enum { IO_BUFFER_SIZE = (1<<15) - 128};
 | 
			
		||||
  ReadBuffer(int fd): fd_(fd), has_EAGAIN_(false), request_more_data_(false),
 | 
			
		||||
                alloc_buf_(NULL), buf_end_(NULL), cur_buf_(NULL), data_end_(NULL),
 | 
			
		||||
                consume_sz_(0)
 | 
			
		||||
  ReadBuffer(SocketReader& reader)
 | 
			
		||||
      : reader_(reader),
 | 
			
		||||
        request_more_data_(false),
 | 
			
		||||
        alloc_buf_(NULL),
 | 
			
		||||
        buf_end_(NULL),
 | 
			
		||||
        cur_buf_(NULL),
 | 
			
		||||
        data_end_(NULL),
 | 
			
		||||
        consume_sz_(0)
 | 
			
		||||
  {}
 | 
			
		||||
  ~ReadBuffer()
 | 
			
		||||
  {
 | 
			
		||||
@ -150,18 +202,18 @@ public:
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  int64_t get_remain_sz() const { return remain(); }
 | 
			
		||||
  void set_fd(int fd) { fd_ = fd; }
 | 
			
		||||
 | 
			
		||||
  int peek_data(int64_t limit, const char*& buf, int64_t& sz) {
 | 
			
		||||
    int ret = OB_SUCCESS;
 | 
			
		||||
    if (OB_FAIL(try_read_fd(limit))) {
 | 
			
		||||
      LOG_WARN("read fail", K(ret), K_(fd), K(limit));
 | 
			
		||||
      LOG_WARN("read fail", K(ret), K_(reader), K(limit));
 | 
			
		||||
    } else {
 | 
			
		||||
      buf = cur_buf_;
 | 
			
		||||
      sz = remain();
 | 
			
		||||
      if (sz < limit) {
 | 
			
		||||
        request_more_data_ = true;
 | 
			
		||||
      }
 | 
			
		||||
      LOG_DEBUG("peek data", K_(fd), K(limit), K(sz));
 | 
			
		||||
      LOG_DEBUG("peek data", K_(reader), K(limit), K(sz));
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
@ -170,16 +222,16 @@ public:
 | 
			
		||||
    if (sz > 0 && sz <= remain()) {
 | 
			
		||||
      cur_buf_ += sz;
 | 
			
		||||
      consume_sz_ += sz;
 | 
			
		||||
      LOG_DEBUG("consume data", K_(fd), K(sz));
 | 
			
		||||
      LOG_DEBUG("consume data", K_(reader), K(sz));
 | 
			
		||||
    } else {
 | 
			
		||||
      ret = OB_INVALID_ARGUMENT;
 | 
			
		||||
      LOG_WARN("consume data, invalid argument", K_(fd), K(sz));
 | 
			
		||||
      LOG_WARN("consume data, invalid argument", K_(reader), K(sz));
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
  bool clear_EAGAIN() {
 | 
			
		||||
    bool ret = (has_EAGAIN_ && (remain() <= 0 || request_more_data_));
 | 
			
		||||
    has_EAGAIN_ = false;
 | 
			
		||||
    bool ret = (reader_.has_EAGAIN() && (remain() <= 0 || request_more_data_));
 | 
			
		||||
    reader_.clear_EAGAIN();
 | 
			
		||||
    request_more_data_ = false;
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
@ -192,9 +244,9 @@ private:
 | 
			
		||||
    } else if (remain() >= limit) {
 | 
			
		||||
 | 
			
		||||
    } else if (cur_buf_ + limit > buf_end_ && OB_FAIL(switch_buffer(limit))) {
 | 
			
		||||
      LOG_ERROR("alloc read buffer fail", K_(fd), K(ret));
 | 
			
		||||
      LOG_ERROR("alloc read buffer fail", K_(reader), K(ret));
 | 
			
		||||
    } else if (OB_FAIL(do_read_fd(limit))) {
 | 
			
		||||
      LOG_WARN("do_read_fd fail", K(ret), K_(fd), K(limit));
 | 
			
		||||
      LOG_WARN("do_read_fd fail", K(ret), K_(reader), K(limit));
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
@ -210,7 +262,7 @@ private:
 | 
			
		||||
      data_end_ = cur_buf_ + rsz;
 | 
			
		||||
    } else if (NULL == (new_buf = (char*)alloc_io_buffer(alloc_size))) {
 | 
			
		||||
      ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
			
		||||
      LOG_WARN("alloc buffer fail", K(ret), K_(fd), K(alloc_size));
 | 
			
		||||
      LOG_WARN("alloc buffer fail", K(ret), K(alloc_size), K_(reader), K(alloc_size));
 | 
			
		||||
    } else {
 | 
			
		||||
      char* old_buffer = alloc_buf_;
 | 
			
		||||
      int64_t rsz = remain();
 | 
			
		||||
@ -228,10 +280,10 @@ private:
 | 
			
		||||
  int do_read_fd(int64_t sz) {
 | 
			
		||||
    int ret = OB_SUCCESS;
 | 
			
		||||
    const int MAX_SSL_REQ_PKT_SIZE = 36;
 | 
			
		||||
    while(remain() < sz && OB_SUCCESS == ret) {
 | 
			
		||||
    while(remain() < sz && OB_SUCCESS == ret && !reader_.has_EAGAIN()) {
 | 
			
		||||
      int64_t rbytes = 0;
 | 
			
		||||
      size_t read_size = 0;
 | 
			
		||||
      if (OB_UNLIKELY(0 == consume_sz_)) {
 | 
			
		||||
      if (OB_UNLIKELY(0 == consume_sz_) && data_end_ == alloc_buf_) {
 | 
			
		||||
        /*
 | 
			
		||||
          set read size for ssl, when client want to open ssl, it will send a 36 bytes
 | 
			
		||||
          incomplete Login Request packet and then do SSL_connect, the data flow will be
 | 
			
		||||
@ -242,21 +294,9 @@ private:
 | 
			
		||||
      } else {
 | 
			
		||||
        read_size = buf_end_ - data_end_;
 | 
			
		||||
      }
 | 
			
		||||
      if ((rbytes = ob_read_regard_ssl(fd_, data_end_, read_size)) > 0) {
 | 
			
		||||
      ret = reader_.read(data_end_, read_size, rbytes);
 | 
			
		||||
      if (OB_SUCC(ret)) {
 | 
			
		||||
        data_end_ += rbytes;
 | 
			
		||||
      } else if (0 == rbytes) {
 | 
			
		||||
        LOG_INFO("read fd return EOF", K_(fd));
 | 
			
		||||
        has_EAGAIN_ = true;
 | 
			
		||||
        ret = OB_IO_ERROR; // for mysql protocol, it is not prossible
 | 
			
		||||
        break;
 | 
			
		||||
      } else if (EAGAIN == errno || EWOULDBLOCK == errno) {
 | 
			
		||||
        has_EAGAIN_ = true;
 | 
			
		||||
        break;
 | 
			
		||||
      } else if (EINTR == errno) {
 | 
			
		||||
        // pass
 | 
			
		||||
      } else {
 | 
			
		||||
        ret = OB_IO_ERROR;
 | 
			
		||||
        LOG_WARN("read fd has error", K_(fd), K(errno));
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
@ -268,8 +308,7 @@ private:
 | 
			
		||||
  }
 | 
			
		||||
  static void direct_free(void* p) { ob_free(p); }
 | 
			
		||||
private:
 | 
			
		||||
  int fd_;
 | 
			
		||||
  bool has_EAGAIN_;
 | 
			
		||||
  SocketReader& reader_;
 | 
			
		||||
  bool request_more_data_;
 | 
			
		||||
  char* alloc_buf_;
 | 
			
		||||
  char* buf_end_;
 | 
			
		||||
@ -341,8 +380,8 @@ private:
 | 
			
		||||
class ObSqlSock: public ObLink
 | 
			
		||||
{
 | 
			
		||||
public:
 | 
			
		||||
  ObSqlSock(ObSqlNioImpl *nio, int fd): dlink_(), all_list_link_(), write_task_link_(), nio_impl_(nio),
 | 
			
		||||
            fd_(fd), err_(0), read_buffer_(fd), need_epoll_trigger_write_(false), may_handling_(true),
 | 
			
		||||
  ObSqlSock(ObSqlNioImpl *nio, int fd): dlink_(), all_list_link_(), write_task_link_(), nio_impl_(nio), fd_(fd),
 | 
			
		||||
            err_(0), reader_(fd), read_buffer_(reader_), need_epoll_trigger_write_(false), may_handling_(true),
 | 
			
		||||
            handler_close_flag_(false), need_shutdown_(false), last_decode_time_(0), last_write_time_(0),
 | 
			
		||||
            sql_session_info_(NULL), tls_verion_option_(SSL_OP_NO_SSLv2|SSL_OP_NO_SSLv3) {
 | 
			
		||||
    memset(sess_, 0, sizeof(sess_));
 | 
			
		||||
@ -350,7 +389,7 @@ public:
 | 
			
		||||
  ~ObSqlSock() {}
 | 
			
		||||
  int64_t get_remain_sz() const { return read_buffer_.get_remain_sz(); }
 | 
			
		||||
  TO_STRING_KV(KP(this), "session_id", get_sql_session_id(), "trace_id", get_trace_id(), "sql_handling_stage", get_sql_request_execute_state(), "sql_initiative_shutdown", need_shutdown_,
 | 
			
		||||
              K_(fd), K_(err), K_(last_decode_time), K_(last_write_time), K_(pending_write_task), K_(need_epoll_trigger_write),
 | 
			
		||||
              K_(reader), K_(err), K_(last_decode_time), K_(last_write_time), K_(pending_write_task), K_(need_epoll_trigger_write),
 | 
			
		||||
              "consume_size", read_buffer_.get_consume_sz(), "pending_flag", get_pending_flag(), "may_handling_flag", get_may_handling_flag(), K_(handler_close_flag));
 | 
			
		||||
  ObSqlNioImpl *get_nio_impl() { return nio_impl_; }
 | 
			
		||||
  void set_nio_impl(ObSqlNioImpl *impl) { nio_impl_ = impl; }
 | 
			
		||||
@ -361,21 +400,58 @@ public:
 | 
			
		||||
    if (fd_ >= 0) {
 | 
			
		||||
      ob_fd_disable_ssl(fd_);
 | 
			
		||||
      close(fd_);
 | 
			
		||||
      read_buffer_.set_fd(-1);
 | 
			
		||||
      reader_.set_fd(-1);
 | 
			
		||||
      fd_ = -1;
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  void set_last_decode_succ_time(int64_t time) { last_decode_time_ = time;  }
 | 
			
		||||
  int64_t get_consume_sz() { return read_buffer_.get_consume_sz(); }
 | 
			
		||||
 | 
			
		||||
  int peek_data(int64_t limit, const char*& buf, int64_t& sz) {
 | 
			
		||||
    return  read_buffer_.peek_data(limit ,buf, sz);
 | 
			
		||||
  int peek_data(void* read_handle, int64_t limit, const char*& buf, int64_t& sz)
 | 
			
		||||
  {
 | 
			
		||||
    ReadBuffer *read_buffer = &read_buffer_;
 | 
			
		||||
    if (OB_UNLIKELY(OB_NOT_NULL(read_handle))) {
 | 
			
		||||
      read_buffer = static_cast<ReadBuffer *>(read_handle);
 | 
			
		||||
      // The application layer try to read packet but it may failed as the EAGAIN flag is set.
 | 
			
		||||
      read_buffer->clear_EAGAIN();
 | 
			
		||||
    }
 | 
			
		||||
    return read_buffer->peek_data(limit, buf, sz);
 | 
			
		||||
  }
 | 
			
		||||
  int consume_data(int64_t sz) { return read_buffer_.consume_data(sz); }
 | 
			
		||||
 | 
			
		||||
  int consume_data(void* read_handle, int64_t sz)
 | 
			
		||||
  {
 | 
			
		||||
    ReadBuffer *read_buffer = (OB_ISNULL(read_handle)) ? &read_buffer_ : static_cast<ReadBuffer *>(read_handle);
 | 
			
		||||
    return read_buffer->consume_data(sz);
 | 
			
		||||
   }
 | 
			
		||||
  void init_write_task(const char* buf, int64_t sz) {
 | 
			
		||||
    pending_write_task_.init(buf, sz);
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  int create_read_handle(void *& handle)
 | 
			
		||||
  {
 | 
			
		||||
    int ret = OB_SUCCESS;
 | 
			
		||||
    ReadBuffer * buffer = OB_NEW(ReadBuffer, MEMORY_MODEL_NAME, reader_);
 | 
			
		||||
    if (OB_ISNULL(buffer)) {
 | 
			
		||||
      ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
			
		||||
    } else {
 | 
			
		||||
      handle = static_cast<void *>(buffer);
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  int release_read_handle(void * handle)
 | 
			
		||||
  {
 | 
			
		||||
    int ret = OB_SUCCESS;
 | 
			
		||||
    if (OB_ISNULL(handle)) {
 | 
			
		||||
      ret = OB_INVALID_ARGUMENT;
 | 
			
		||||
    } else {
 | 
			
		||||
      ReadBuffer* read_handle = static_cast<ReadBuffer*>(handle);
 | 
			
		||||
      OB_DELETE(ReadBuffer, MEMORY_MODEL_NAME, read_handle);
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
  bool is_need_epoll_trigger_write() const { return need_epoll_trigger_write_; }
 | 
			
		||||
  int do_pending_write(bool& become_clean) {
 | 
			
		||||
    int ret = OB_SUCCESS;
 | 
			
		||||
@ -453,6 +529,7 @@ private:
 | 
			
		||||
  ObSqlNioImpl *nio_impl_;
 | 
			
		||||
  int fd_;
 | 
			
		||||
  int err_;
 | 
			
		||||
  SocketReader reader_;
 | 
			
		||||
  ReadBuffer read_buffer_;
 | 
			
		||||
  ReadyFlag ready_flag_;
 | 
			
		||||
  SingleWaitCond write_cond_;
 | 
			
		||||
@ -505,7 +582,7 @@ int ObSqlSock::set_ssl_enabled()
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  if (OB_FAIL(ob_fd_enable_ssl_for_server(fd_, OB_SSL_CTX_ID_SQL_NIO, tls_verion_option_))) {
 | 
			
		||||
    LOG_WARN("sqlnio enable ssl for server failed", K(ret), K(fd_));
 | 
			
		||||
    LOG_WARN("sqlnio enable ssl for server failed", K(ret), K(reader_));
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
@ -529,7 +606,7 @@ int ObSqlSock::write_handshake_packet(const char* buf, int64_t sz) {
 | 
			
		||||
      //will be treated as error
 | 
			
		||||
      IGNORE_RETURN(set_error(EIO));
 | 
			
		||||
      ret = OB_IO_ERROR;
 | 
			
		||||
      LOG_WARN("write data error", K_(fd), K(errno));
 | 
			
		||||
      LOG_WARN("write data error", K(fd_), K(errno));
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  last_write_time_ = ObTimeUtility::current_time();
 | 
			
		||||
@ -1159,14 +1236,23 @@ bool ObSqlNio::has_error(void* sess)
 | 
			
		||||
  return sess2sock(sess)->has_error();
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlNio::peek_data(void* sess, int64_t limit, const char*& buf, int64_t& sz)
 | 
			
		||||
int ObSqlNio::create_read_handle(void* sess, void*& read_handle)
 | 
			
		||||
{
 | 
			
		||||
  return sess2sock(sess)->peek_data(limit, buf, sz);
 | 
			
		||||
  return sess2sock(sess)->create_read_handle(read_handle);
 | 
			
		||||
}
 | 
			
		||||
int ObSqlNio::release_read_handle(void* sess, void* read_handle)
 | 
			
		||||
{
 | 
			
		||||
  return sess2sock(sess)->release_read_handle(read_handle);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlNio::consume_data(void* sess, int64_t sz)
 | 
			
		||||
int ObSqlNio::peek_data(void* sess, void* read_handle, int64_t limit, const char*& buf, int64_t& sz)
 | 
			
		||||
{
 | 
			
		||||
  return sess2sock(sess)->consume_data(sz);
 | 
			
		||||
 return sess2sock(sess)->peek_data(read_handle, limit, buf, sz);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlNio::consume_data(void* sess, void* read_handle, int64_t sz)
 | 
			
		||||
{
 | 
			
		||||
 return sess2sock(sess)->consume_data(read_handle, sz);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlNio::write_data(void* sess, const char* buf, int64_t sz)
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										6
									
								
								deps/oblib/src/rpc/obmysql/ob_sql_nio.h
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										6
									
								
								deps/oblib/src/rpc/obmysql/ob_sql_nio.h
									
									
									
									
										vendored
									
									
								
							@ -37,8 +37,10 @@ public:
 | 
			
		||||
  bool has_error(void* sess);
 | 
			
		||||
  void destroy_sock(void* sess);
 | 
			
		||||
  void revert_sock(void* sess);
 | 
			
		||||
  int peek_data(void* sess, int64_t limit, const char*& buf, int64_t& sz);
 | 
			
		||||
  int consume_data(void* sess, int64_t sz);
 | 
			
		||||
  int create_read_handle(void* sess, void*& read_handle);
 | 
			
		||||
  int release_read_handle(void* sess, void* read_handle);
 | 
			
		||||
  int peek_data(void* sess, void* read_handle, int64_t limit, const char*& buf, int64_t& sz);
 | 
			
		||||
  int consume_data(void* sess, void* read_handle, int64_t sz);
 | 
			
		||||
  int write_data(void* sess, const char* buf, int64_t sz);
 | 
			
		||||
  void async_write_data(void* sess, const char* buf, int64_t sz);
 | 
			
		||||
  void stop();
 | 
			
		||||
 | 
			
		||||
@ -59,6 +59,11 @@ void ObSqlNioServer::update_tcp_keepalive_params(int keepalive_enabled, uint32_t
 | 
			
		||||
  nio_.update_tcp_keepalive_params(keepalive_enabled, tcp_keepidle, tcp_keepintvl, tcp_keepcnt);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
ObSqlSockProcessor& ObSqlNioServer::get_sql_sock_processor()
 | 
			
		||||
{
 | 
			
		||||
  return thread_processor_;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
ObSqlNioServer* global_sql_nio_server = NULL;
 | 
			
		||||
}; // end namespace obmysql
 | 
			
		||||
}; // end namespace oceanbase
 | 
			
		||||
 | 
			
		||||
@ -40,6 +40,9 @@ public:
 | 
			
		||||
  void wait();
 | 
			
		||||
  void destroy();
 | 
			
		||||
  void update_tcp_keepalive_params(int keepalive_enabled, uint32_t tcp_keepidle, uint32_t tcp_keepintvl, uint32_t tcp_keepcnt);
 | 
			
		||||
 | 
			
		||||
  ObSqlSockProcessor& get_sql_sock_processor();
 | 
			
		||||
 | 
			
		||||
private:
 | 
			
		||||
  ObSqlSockProcessor thread_processor_; // for tenant worker
 | 
			
		||||
  ObSqlSockHandler io_handler_; // for io thread
 | 
			
		||||
 | 
			
		||||
@ -130,7 +130,7 @@ int ObSqlSockHandler::on_readable(void* udata)
 | 
			
		||||
  if (NULL == sess) {
 | 
			
		||||
    ret = OB_INVALID_ARGUMENT;
 | 
			
		||||
    LOG_ERROR("sess is null!", K(ret));
 | 
			
		||||
  } else if (OB_FAIL(sock_processor_.decode_sql_packet(*sess, pkt))) {
 | 
			
		||||
  } else if (OB_FAIL(sock_processor_.decode_sql_packet(sess->pool_, *sess, NULL, pkt))) {
 | 
			
		||||
    LOG_WARN("decode sql req fail", K(ret));
 | 
			
		||||
  } else if (NULL == pkt) {
 | 
			
		||||
    sess->revert_sock();
 | 
			
		||||
 | 
			
		||||
@ -37,7 +37,10 @@ static int processor_do_decode(ObVirtualCSProtocolProcessor& processor, ObSMConn
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlSockProcessor::decode_sql_packet(ObSqlSockSession& sess, rpc::ObPacket*& ret_pkt)
 | 
			
		||||
int ObSqlSockProcessor::decode_sql_packet(ObICSMemPool& mem_pool,
 | 
			
		||||
                                          ObSqlSockSession& sess,
 | 
			
		||||
                                          void* read_handle,
 | 
			
		||||
                                          rpc::ObPacket*& ret_pkt)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  ObSMConnection& conn = sess.conn_;
 | 
			
		||||
@ -46,17 +49,17 @@ int ObSqlSockProcessor::decode_sql_packet(ObSqlSockSession& sess, rpc::ObPacket*
 | 
			
		||||
  const char* start = NULL;
 | 
			
		||||
  int64_t limit = 1;
 | 
			
		||||
  int64_t read_sz = 0;
 | 
			
		||||
  char* buf = NULL;
 | 
			
		||||
  ret_pkt = NULL;
 | 
			
		||||
 | 
			
		||||
  while(OB_SUCCESS == ret && NULL == ret_pkt) {
 | 
			
		||||
    bool need_read_more = false;
 | 
			
		||||
    int64_t consume_sz = 0;
 | 
			
		||||
    
 | 
			
		||||
    if (OB_FAIL(sess.peek_data(limit, start, read_sz))) {
 | 
			
		||||
    if (OB_FAIL(sess.peek_data(read_handle, limit, start, read_sz))) {
 | 
			
		||||
      LOG_WARN("peed data fail", K(ret));
 | 
			
		||||
    } else if (read_sz < limit) {
 | 
			
		||||
      break;
 | 
			
		||||
    } else if (OB_FAIL(processor_do_decode(*processor, conn, sess.pool_, start, read_sz, pkt, limit, consume_sz))) {
 | 
			
		||||
    } else if (OB_FAIL(processor_do_decode(*processor, conn, mem_pool, start, read_sz, pkt, limit, consume_sz))) {
 | 
			
		||||
      LOG_WARN("do_decode fail", K(ret));
 | 
			
		||||
    } else if (NULL == pkt) {
 | 
			
		||||
      // try read more
 | 
			
		||||
@ -70,13 +73,18 @@ int ObSqlSockProcessor::decode_sql_packet(ObSqlSockSession& sess, rpc::ObPacket*
 | 
			
		||||
    } else if (!conn.is_in_authed_phase()) {
 | 
			
		||||
      ret_pkt = pkt;
 | 
			
		||||
      sess.set_last_pkt_sz(consume_sz);
 | 
			
		||||
    } else if (OB_FAIL(processor->do_splice(conn, sess.pool_, (void*&)pkt, need_read_more))) {
 | 
			
		||||
    } else if (OB_FAIL(processor->do_splice(conn, mem_pool, (void*&)pkt, need_read_more))) {
 | 
			
		||||
      LOG_WARN("do_splice fail");
 | 
			
		||||
    } else if (!need_read_more) {
 | 
			
		||||
      ret_pkt = pkt;
 | 
			
		||||
      sess.set_last_pkt_sz(consume_sz);
 | 
			
		||||
      if (NULL == read_handle) {
 | 
			
		||||
        sess.set_last_pkt_sz(consume_sz);
 | 
			
		||||
      } else if (OB_LIKELY(ret_pkt != NULL)) {
 | 
			
		||||
        ObMySQLRawPacket *raw_packet = static_cast<ObMySQLRawPacket *>(ret_pkt);
 | 
			
		||||
        raw_packet->set_consume_size(consume_sz);
 | 
			
		||||
      }
 | 
			
		||||
    } else {
 | 
			
		||||
      sess.consume_data(consume_sz);
 | 
			
		||||
      sess.consume_data(read_handle, consume_sz); // assert read_handle == NULL
 | 
			
		||||
      limit = 1; // new pkt need more data
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
@ -32,7 +32,7 @@ public:
 | 
			
		||||
  ObSqlSockProcessor(ObMySQLHandler& handler):
 | 
			
		||||
      mysql_processor_(), compress_processor_(), ob_2_0_processor_() {}
 | 
			
		||||
  ~ObSqlSockProcessor() {}
 | 
			
		||||
  int decode_sql_packet(ObSqlSockSession& sess, rpc::ObPacket*& pkt);
 | 
			
		||||
  int decode_sql_packet(ObICSMemPool& mem_pool, ObSqlSockSession& sess, void* read_handle, rpc::ObPacket*& pkt);
 | 
			
		||||
  int build_sql_req(ObSqlSockSession& sess, rpc::ObPacket* pkt, rpc::ObRequest*& sql_req);
 | 
			
		||||
private:
 | 
			
		||||
  ObVirtualCSProtocolProcessor *get_protocol_processor(common::ObCSProtocolType type);
 | 
			
		||||
 | 
			
		||||
@ -71,7 +71,7 @@ void ObSqlSockSession::shutdown()
 | 
			
		||||
void ObSqlSockSession::revert_sock()
 | 
			
		||||
{
 | 
			
		||||
  if (last_pkt_sz_ > 0) {
 | 
			
		||||
    nio_->consume_data((void*)this, last_pkt_sz_);
 | 
			
		||||
    nio_->consume_data((void*)this, NULL, last_pkt_sz_);
 | 
			
		||||
    last_pkt_sz_ = 0;
 | 
			
		||||
  }
 | 
			
		||||
  sql_req_.reset_trace_id();
 | 
			
		||||
@ -100,13 +100,23 @@ bool ObSqlSockSession::has_error()
 | 
			
		||||
  return nio_->has_error((void*)this);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlSockSession::peek_data(int64_t limit, const char*& buf, int64_t& sz)
 | 
			
		||||
int ObSqlSockSession::create_read_handle(void *& read_handle)
 | 
			
		||||
{
 | 
			
		||||
  return nio_->create_read_handle((void*)this, read_handle);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlSockSession::release_read_handle(void * read_handle)
 | 
			
		||||
{
 | 
			
		||||
  return nio_->release_read_handle((void*)this, read_handle);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlSockSession::peek_data(void *handle, int64_t limit, const char*& buf, int64_t& sz)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  if (has_error()) {
 | 
			
		||||
    ret = OB_IO_ERROR;
 | 
			
		||||
    LOG_WARN("sock has error", K(ret));
 | 
			
		||||
  } else if (OB_FAIL(nio_->peek_data((void*)this,  limit, buf, sz))) {
 | 
			
		||||
  } else if (OB_FAIL(nio_->peek_data((void*)this, handle, limit, buf, sz))) {
 | 
			
		||||
    destroy_sock();
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
@ -117,13 +127,13 @@ void ObSqlSockSession::clear_sql_session_info()
 | 
			
		||||
  nio_->reset_sql_session_info(this);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObSqlSockSession::consume_data(int64_t sz)
 | 
			
		||||
int ObSqlSockSession::consume_data(void* read_handle, int64_t sz)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  if (has_error()) {
 | 
			
		||||
    ret = OB_IO_ERROR;
 | 
			
		||||
    LOG_WARN("sock has error", K(ret));
 | 
			
		||||
  } else if (OB_FAIL(nio_->consume_data((void*)this, sz))) {
 | 
			
		||||
  } else if (OB_FAIL(nio_->consume_data((void*)this, read_handle, sz))) {
 | 
			
		||||
    destroy_sock();
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
 | 
			
		||||
@ -46,8 +46,10 @@ public:
 | 
			
		||||
  void destroy();
 | 
			
		||||
  void destroy_sock();
 | 
			
		||||
  bool has_error();
 | 
			
		||||
  int peek_data(int64_t limit, const char*& buf, int64_t& sz);
 | 
			
		||||
  int consume_data(int64_t sz);
 | 
			
		||||
  int create_read_handle(void*& read_handle);
 | 
			
		||||
  int release_read_handle(void* read_handle);
 | 
			
		||||
  int peek_data(void* read_handle, int64_t limit, const char*& buf, int64_t& sz);
 | 
			
		||||
  int consume_data(void* read_handle, int64_t sz);
 | 
			
		||||
  int write_data(const char* buf, int64_t sz);
 | 
			
		||||
  int async_write_data(const char* buf, int64_t sz);
 | 
			
		||||
  void on_flushed();
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										72
									
								
								deps/oblib/src/rpc/obmysql/packet/ompk_local_infile.cpp
									
									
									
									
										vendored
									
									
										Normal file
									
								
							
							
						
						
									
										72
									
								
								deps/oblib/src/rpc/obmysql/packet/ompk_local_infile.cpp
									
									
									
									
										vendored
									
									
										Normal file
									
								
							@ -0,0 +1,72 @@
 | 
			
		||||
/**
 | 
			
		||||
 * Copyright (c) 2021 OceanBase
 | 
			
		||||
 * OceanBase CE is licensed under Mulan PubL v2.
 | 
			
		||||
 * You can use this software according to the terms and conditions of the Mulan PubL v2.
 | 
			
		||||
 * You may obtain a copy of Mulan PubL v2 at:
 | 
			
		||||
 *          http://license.coscl.org.cn/MulanPubL-2.0
 | 
			
		||||
 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
 | 
			
		||||
 * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
 | 
			
		||||
 * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
 | 
			
		||||
 * See the Mulan PubL v2 for more details.
 | 
			
		||||
 */
 | 
			
		||||
 | 
			
		||||
#define USING_LOG_PREFIX RPC_OBMYSQL
 | 
			
		||||
 | 
			
		||||
#include "rpc/obmysql/packet/ompk_local_infile.h"
 | 
			
		||||
#include "rpc/obmysql/ob_mysql_util.h"
 | 
			
		||||
 | 
			
		||||
namespace oceanbase
 | 
			
		||||
{
 | 
			
		||||
namespace obmysql
 | 
			
		||||
{
 | 
			
		||||
 | 
			
		||||
OMPKLocalInfile::OMPKLocalInfile() :
 | 
			
		||||
    packet_type_(0xFB)
 | 
			
		||||
{}
 | 
			
		||||
 | 
			
		||||
OMPKLocalInfile::~OMPKLocalInfile()
 | 
			
		||||
{}
 | 
			
		||||
 | 
			
		||||
int OMPKLocalInfile::serialize(char *buffer, int64_t len, int64_t &pos) const
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
 | 
			
		||||
  if (OB_ISNULL(buffer) || OB_UNLIKELY(len - pos < 0)) {
 | 
			
		||||
    ret = OB_INVALID_ARGUMENT;
 | 
			
		||||
    LOG_WARN("invalid argument", KP(buffer), K(len), K(pos), K(ret));
 | 
			
		||||
  } else if (OB_UNLIKELY(len - pos < static_cast<int64_t>(get_serialize_size()))) {
 | 
			
		||||
    ret = OB_SIZE_OVERFLOW;
 | 
			
		||||
    LOG_WARN("invalid argument", K(len), K(pos), "need_size", get_serialize_size());
 | 
			
		||||
  } else if (OB_FAIL(ObMySQLUtil::store_int1(buffer, len, packet_type_, pos))) {
 | 
			
		||||
    LOG_WARN("store fail", KP(buffer), K(len), K(pos), K(ret));
 | 
			
		||||
  } else if (OB_FAIL(ObMySQLUtil::store_obstr_nzt(buffer, len, filename_, pos))) {
 | 
			
		||||
    LOG_WARN("store fail", KP(buffer), K(len), K(pos), K(ret));
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int64_t OMPKLocalInfile::get_serialize_size() const
 | 
			
		||||
{
 | 
			
		||||
  int64_t len = 1/* packet type*/ + filename_.length();
 | 
			
		||||
  return len;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int64_t OMPKLocalInfile::to_string(char *buf, const int64_t buf_len) const
 | 
			
		||||
{
 | 
			
		||||
  int64_t pos = 0;
 | 
			
		||||
  J_OBJ_START();
 | 
			
		||||
  J_KV("header", hdr_,
 | 
			
		||||
       "packet_type", "LOCAL INFILE",
 | 
			
		||||
       K_(filename));
 | 
			
		||||
  J_OBJ_END();
 | 
			
		||||
  return pos;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
void OMPKLocalInfile::set_filename(const ObString &filename)
 | 
			
		||||
{
 | 
			
		||||
  filename_ = filename;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
} // namespace obmysql
 | 
			
		||||
} // namespace oceanbase
 | 
			
		||||
							
								
								
									
										54
									
								
								deps/oblib/src/rpc/obmysql/packet/ompk_local_infile.h
									
									
									
									
										vendored
									
									
										Normal file
									
								
							
							
						
						
									
										54
									
								
								deps/oblib/src/rpc/obmysql/packet/ompk_local_infile.h
									
									
									
									
										vendored
									
									
										Normal file
									
								
							@ -0,0 +1,54 @@
 | 
			
		||||
/**
 | 
			
		||||
 * Copyright (c) 2021 OceanBase
 | 
			
		||||
 * OceanBase CE is licensed under Mulan PubL v2.
 | 
			
		||||
 * You can use this software according to the terms and conditions of the Mulan PubL v2.
 | 
			
		||||
 * You may obtain a copy of Mulan PubL v2 at:
 | 
			
		||||
 *          http://license.coscl.org.cn/MulanPubL-2.0
 | 
			
		||||
 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
 | 
			
		||||
 * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
 | 
			
		||||
 * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
 | 
			
		||||
 * See the Mulan PubL v2 for more details.
 | 
			
		||||
 */
 | 
			
		||||
 | 
			
		||||
#ifndef _OMPK_LOCAL_INFILE_H_
 | 
			
		||||
#define _OMPK_LOCAL_INFILE_H_
 | 
			
		||||
 | 
			
		||||
#include "rpc/obmysql/ob_mysql_packet.h"
 | 
			
		||||
#include "lib/ob_define.h"
 | 
			
		||||
 | 
			
		||||
namespace oceanbase
 | 
			
		||||
{
 | 
			
		||||
namespace obmysql
 | 
			
		||||
{
 | 
			
		||||
 | 
			
		||||
// In the MySQL client/server protocol, server send a `local infile` message to client
 | 
			
		||||
// to specific the file name to load.
 | 
			
		||||
// format:
 | 
			
		||||
// int<1>      | packet type  | 0xFB: LOCAL INFILE
 | 
			
		||||
// string<EOF> | filename     | the path to the file the client shall send
 | 
			
		||||
// The notation is "string<EOF>" Strings whose length will be calculated by the packet remaining length.
 | 
			
		||||
class OMPKLocalInfile : public ObMySQLRawPacket
 | 
			
		||||
{
 | 
			
		||||
public:
 | 
			
		||||
  OMPKLocalInfile();
 | 
			
		||||
  virtual ~OMPKLocalInfile();
 | 
			
		||||
 | 
			
		||||
  virtual int serialize(char *buffer, int64_t len, int64_t &pos) const override;
 | 
			
		||||
  virtual int64_t get_serialize_size() const override;
 | 
			
		||||
 | 
			
		||||
  virtual int64_t to_string(char *buf, const int64_t buf_len) const override;
 | 
			
		||||
 | 
			
		||||
  void set_filename(const ObString &filename);
 | 
			
		||||
 | 
			
		||||
  inline ObMySQLPacketType get_mysql_packet_type() override { return ObMySQLPacketType::PKT_FILENAME; }
 | 
			
		||||
 | 
			
		||||
private:
 | 
			
		||||
  DISALLOW_COPY_AND_ASSIGN(OMPKLocalInfile);
 | 
			
		||||
  int8_t   packet_type_;
 | 
			
		||||
  ObString filename_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
} // end of namespace obmysql
 | 
			
		||||
} // end of namespace oceanbase
 | 
			
		||||
 | 
			
		||||
#endif /* _OMPK_LOCAL_INFILE_H_ */
 | 
			
		||||
		Reference in New Issue
	
	Block a user