You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by ma...@apache.org on 2018/04/13 06:58:31 UTC

[trafficserver] branch quic-latest updated (fadeb54 -> 6212927)

This is an automated email from the ASF dual-hosted git repository.

masaori pushed a change to branch quic-latest
in repository https://gitbox.apache.org/repos/asf/trafficserver.git.


    from fadeb54  QUIC: Adding total offset in Stream Manager
     new e1e5e86  Rename QUICConnection in QUICApplication
     new 643f742  Add create_uni/bidi_stream to QUICStreamManager
     new 6212927  Add HTTP/0.9 exchange support on quic client

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 cmd/traffic_quic/quic_client.cc      | 80 +++++++++++++++++++++++++++++++++++-
 cmd/traffic_quic/quic_client.h       | 19 +++++++--
 cmd/traffic_quic/traffic_quic.cc     |  6 ++-
 iocore/net/QUICNetVConnection.cc     |  3 +-
 iocore/net/quic/QUICApplication.cc   |  5 +--
 iocore/net/quic/QUICApplication.h    |  2 +-
 iocore/net/quic/QUICHandshake.cc     |  8 ++--
 iocore/net/quic/QUICStreamManager.cc | 37 ++++++++++++++++-
 iocore/net/quic/QUICStreamManager.h  |  8 +++-
 proxy/hq/QUICSimpleApp.cc            |  6 +--
 10 files changed, 153 insertions(+), 21 deletions(-)

-- 
To stop receiving notification emails like this one, please contact
masaori@apache.org.

[trafficserver] 01/03: Rename QUICConnection in QUICApplication

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

masaori pushed a commit to branch quic-latest
in repository https://gitbox.apache.org/repos/asf/trafficserver.git

commit e1e5e86f896bae893a6507dc5f1df61d8e37e8b3
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Fri Apr 13 15:39:40 2018 +0900

    Rename QUICConnection in QUICApplication
---
 iocore/net/quic/QUICApplication.cc | 5 ++---
 iocore/net/quic/QUICApplication.h  | 2 +-
 iocore/net/quic/QUICHandshake.cc   | 8 ++++----
 proxy/hq/QUICSimpleApp.cc          | 6 +++---
 4 files changed, 10 insertions(+), 11 deletions(-)

diff --git a/iocore/net/quic/QUICApplication.cc b/iocore/net/quic/QUICApplication.cc
index c15dd93..4283819 100644
--- a/iocore/net/quic/QUICApplication.cc
+++ b/iocore/net/quic/QUICApplication.cc
@@ -133,7 +133,7 @@ QUICStreamIO::get_transaction_id() const
 //
 QUICApplication::QUICApplication(QUICConnection *qc) : Continuation(new_ProxyMutex())
 {
-  this->_client_qc = qc;
+  this->_qc = qc;
 }
 
 // @brief Bind stream and application
@@ -162,8 +162,7 @@ QUICApplication::reenable(QUICStream *stream)
     stream_io->read_reenable();
     stream_io->write_reenable();
   } else {
-    Debug(tag, "[%" PRIx64 "] Unknown Stream, id: %" PRIx64, static_cast<uint64_t>(this->_client_qc->connection_id()),
-          stream->id());
+    Debug(tag, "[%" PRIx64 "] Unknown Stream, id: %" PRIx64, static_cast<uint64_t>(this->_qc->connection_id()), stream->id());
   }
 
   return;
diff --git a/iocore/net/quic/QUICApplication.h b/iocore/net/quic/QUICApplication.h
index d1f94de..eef90a0 100644
--- a/iocore/net/quic/QUICApplication.h
+++ b/iocore/net/quic/QUICApplication.h
@@ -86,7 +86,7 @@ protected:
   QUICStreamIO *_find_stream_io(QUICStreamId id);
   QUICStreamIO *_find_stream_io(VIO *vio);
 
-  QUICConnection *_client_qc = nullptr;
+  QUICConnection *_qc = nullptr;
 
 private:
   std::map<QUICStreamId, QUICStreamIO *> _stream_map;
diff --git a/iocore/net/quic/QUICHandshake.cc b/iocore/net/quic/QUICHandshake.cc
index 37ec334..831da56 100644
--- a/iocore/net/quic/QUICHandshake.cc
+++ b/iocore/net/quic/QUICHandshake.cc
@@ -37,7 +37,7 @@
 static constexpr char dump_tag[] = "v_quic_handshake_dump_pkt";
 
 #define QUICHSDebug(fmt, ...) \
-  Debug("quic_handshake", "[%" PRIx64 "] " fmt, static_cast<uint64_t>(this->_client_qc->connection_id()), ##__VA_ARGS__)
+  Debug("quic_handshake", "[%" PRIx64 "] " fmt, static_cast<uint64_t>(this->_qc->connection_id()), ##__VA_ARGS__)
 
 #define I_WANNA_DUMP_THIS_BUF(buf, len)                                                                                            \
   {                                                                                                                                \
@@ -101,7 +101,7 @@ QUICHandshake::QUICHandshake(QUICConnection *qc, SSL_CTX *ssl_ctx, QUICStateless
 {
   SSL_set_ex_data(this->_ssl, QUIC::ssl_quic_qc_index, qc);
   SSL_set_ex_data(this->_ssl, QUIC::ssl_quic_hs_index, this);
-  this->_hs_protocol->initialize_key_materials(this->_client_qc->original_connection_id());
+  this->_hs_protocol->initialize_key_materials(this->_qc->original_connection_id());
 
   if (this->_netvc_context == NET_VCONNECTION_OUT) {
     this->_initial = true;
@@ -143,7 +143,7 @@ QUICHandshake::start(const QUICPacket *initial_packet, QUICPacketFactory *packet
         this->_load_local_server_transport_parameters(initial_packet->version());
         packet_factory->set_version(this->_version_negotiator->negotiated_version());
       } else {
-        this->_client_qc->transmit_packet(packet_factory->create_version_negotiation_packet(initial_packet));
+        this->_qc->transmit_packet(packet_factory->create_version_negotiation_packet(initial_packet));
         QUICHSDebug("Version negotiation failed: %x", initial_packet->version());
       }
     } else {
@@ -525,7 +525,7 @@ QUICHandshake::_complete_handshake()
 void
 QUICHandshake::_abort_handshake(QUICTransErrorCode code)
 {
-  this->_client_qc->close(QUICConnectionErrorUPtr(new QUICConnectionError(code)));
+  this->_qc->close(QUICConnectionErrorUPtr(new QUICConnectionError(code)));
 
   QUICHSDebug("Enter state_closed");
   SET_HANDLER(&QUICHandshake::state_closed);
diff --git a/proxy/hq/QUICSimpleApp.cc b/proxy/hq/QUICSimpleApp.cc
index 36fd7d7..b06821d 100644
--- a/proxy/hq/QUICSimpleApp.cc
+++ b/proxy/hq/QUICSimpleApp.cc
@@ -42,7 +42,7 @@ QUICSimpleApp::QUICSimpleApp(QUICNetVConnection *client_vc) : QUICApplication(cl
   this->_client_session->acl_record = session_acl_record;
   this->_client_session->new_connection(client_vc, nullptr, nullptr, false);
 
-  this->_client_qc->stream_manager()->set_default_application(this);
+  this->_qc->stream_manager()->set_default_application(this);
 
   SET_HANDLER(&QUICSimpleApp::main_event_handler);
 }
@@ -55,13 +55,13 @@ QUICSimpleApp::~QUICSimpleApp()
 int
 QUICSimpleApp::main_event_handler(int event, Event *data)
 {
-  Debug(tag, "[%" PRIx64 "] %s (%d)", static_cast<uint64_t>(this->_client_qc->connection_id()), get_vc_event_name(event), event);
+  Debug(tag, "[%" PRIx64 "] %s (%d)", static_cast<uint64_t>(this->_qc->connection_id()), get_vc_event_name(event), event);
 
   VIO *vio                = reinterpret_cast<VIO *>(data);
   QUICStreamIO *stream_io = this->_find_stream_io(vio);
 
   if (stream_io == nullptr) {
-    Debug(tag, "[%" PRIx64 "] Unknown Stream", static_cast<uint64_t>(this->_client_qc->connection_id()));
+    Debug(tag, "[%" PRIx64 "] Unknown Stream", static_cast<uint64_t>(this->_qc->connection_id()));
     return -1;
   }
 

-- 
To stop receiving notification emails like this one, please contact
masaori@apache.org.

[trafficserver] 02/03: Add create_uni/bidi_stream to QUICStreamManager

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

masaori pushed a commit to branch quic-latest
in repository https://gitbox.apache.org/repos/asf/trafficserver.git

commit 643f7420aeedea9602c5f941df0ca443c30721d6
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Fri Apr 13 15:40:55 2018 +0900

    Add create_uni/bidi_stream to QUICStreamManager
---
 iocore/net/QUICNetVConnection.cc     |  3 ++-
 iocore/net/quic/QUICStreamManager.cc | 37 ++++++++++++++++++++++++++++++++++--
 iocore/net/quic/QUICStreamManager.h  |  8 +++++++-
 3 files changed, 44 insertions(+), 4 deletions(-)

diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index 1463056..506223d 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -204,7 +204,8 @@ QUICNetVConnection::start()
   this->_remote_flow_controller = new QUICRemoteConnectionFlowController(UINT64_MAX);
   this->_local_flow_controller  = new QUICLocalConnectionFlowController(this->_loss_detector, UINT64_MAX);
   this->_path_validator         = new QUICPathValidator();
-  this->_stream_manager         = new QUICStreamManager(this->_loss_detector, this->connection_id(), this->_application_map);
+  this->_stream_manager =
+    new QUICStreamManager(this->_loss_detector, this->connection_id(), this->_application_map, this->netvc_context);
 
   this->_frame_dispatcher->add_handler(this);
   this->_frame_dispatcher->add_handler(this->_stream_manager);
diff --git a/iocore/net/quic/QUICStreamManager.cc b/iocore/net/quic/QUICStreamManager.cc
index a01ffce..f52cf52 100644
--- a/iocore/net/quic/QUICStreamManager.cc
+++ b/iocore/net/quic/QUICStreamManager.cc
@@ -32,9 +32,18 @@ static constexpr char tag[] = "quic_stream_manager";
 ClassAllocator<QUICStreamManager> quicStreamManagerAllocator("quicStreamManagerAllocator");
 ClassAllocator<QUICStream> quicStreamAllocator("quicStreamAllocator");
 
-QUICStreamManager::QUICStreamManager(QUICRTTProvider *rtt_provider, QUICConnectionId cid, QUICApplicationMap *app_map)
-  : _connection_id(cid), _app_map(app_map), _rtt_provider(rtt_provider)
+QUICStreamManager::QUICStreamManager(QUICRTTProvider *rtt_provider, QUICConnectionId cid, QUICApplicationMap *app_map,
+                                     NetVConnectionContext_t context)
+  : _connection_id(cid), _app_map(app_map), _netvc_context(context), _rtt_provider(rtt_provider)
 {
+  if (this->_netvc_context == NET_VCONNECTION_OUT) {
+    // stream 0 is for handshake, smallest client bidi stream id is 4
+    this->_next_stream_id_bidi = static_cast<uint32_t>(QUICStreamType::CLIENT_BIDI) + 4;
+    this->_next_stream_id_uni  = static_cast<uint32_t>(QUICStreamType::CLIENT_UNI);
+  } else {
+    this->_next_stream_id_bidi = static_cast<uint32_t>(QUICStreamType::SERVER_BIDI);
+    this->_next_stream_id_uni  = static_cast<uint32_t>(QUICStreamType::SERVER_UNI);
+  }
 }
 
 std::vector<QUICFrameType>
@@ -103,6 +112,30 @@ QUICStreamManager::create_stream(QUICStreamId stream_id)
 }
 
 QUICErrorUPtr
+QUICStreamManager::create_uni_stream(QUICStreamId &new_stream_id)
+{
+  QUICErrorUPtr error = this->create_stream(this->_next_stream_id_uni);
+  if (error->cls == QUICErrorClass::NONE) {
+    new_stream_id = this->_next_stream_id_uni;
+    this->_next_stream_id_uni += 2;
+  }
+
+  return error;
+}
+
+QUICErrorUPtr
+QUICStreamManager::create_bidi_stream(QUICStreamId &new_stream_id)
+{
+  QUICErrorUPtr error = this->create_stream(this->_next_stream_id_bidi);
+  if (error->cls == QUICErrorClass::NONE) {
+    new_stream_id = this->_next_stream_id_bidi;
+    this->_next_stream_id_bidi += 2;
+  }
+
+  return error;
+}
+
+QUICErrorUPtr
 QUICStreamManager::handle_frame(std::shared_ptr<const QUICFrame> frame)
 {
   QUICErrorUPtr error = QUICErrorUPtr(new QUICNoError());
diff --git a/iocore/net/quic/QUICStreamManager.h b/iocore/net/quic/QUICStreamManager.h
index 9f8851a..3b14b41 100644
--- a/iocore/net/quic/QUICStreamManager.h
+++ b/iocore/net/quic/QUICStreamManager.h
@@ -38,7 +38,8 @@ class QUICStreamManager : public QUICFrameHandler, public QUICFrameGenerator
 {
 public:
   QUICStreamManager(){};
-  QUICStreamManager(QUICRTTProvider *rtt_provider, QUICConnectionId cid, QUICApplicationMap *app_map);
+  QUICStreamManager(QUICRTTProvider *rtt_provider, QUICConnectionId cid, QUICApplicationMap *app_map,
+                    NetVConnectionContext_t context);
 
   void init_flow_control_params(const std::shared_ptr<const QUICTransportParameters> &local_tp,
                                 const std::shared_ptr<const QUICTransportParameters> &remote_tp);
@@ -49,6 +50,8 @@ public:
 
   uint32_t stream_count() const;
   QUICErrorUPtr create_stream(QUICStreamId stream_id);
+  QUICErrorUPtr create_uni_stream(QUICStreamId &new_stream_id);
+  QUICErrorUPtr create_bidi_stream(QUICStreamId &new_stream_id);
 
   void set_default_application(QUICApplication *app);
   void reset_send_offset();
@@ -76,12 +79,15 @@ private:
 
   QUICConnectionId _connection_id                           = 0;
   QUICApplicationMap *_app_map                              = nullptr;
+  NetVConnectionContext_t _netvc_context                    = NET_VCONNECTION_UNSET;
   std::shared_ptr<const QUICTransportParameters> _local_tp  = nullptr;
   std::shared_ptr<const QUICTransportParameters> _remote_tp = nullptr;
   QUICStreamId _local_maximum_stream_id_bidi                = 0;
   QUICStreamId _local_maximum_stream_id_uni                 = 0;
   QUICStreamId _remote_maximum_stream_id_bidi               = 0;
   QUICStreamId _remote_maximum_stream_id_uni                = 0;
+  QUICStreamId _next_stream_id_uni                          = 0;
+  QUICStreamId _next_stream_id_bidi                         = 0;
   uint64_t _total_offset_sent                               = 0;
   QUICRTTProvider *_rtt_provider                            = nullptr;
 };

-- 
To stop receiving notification emails like this one, please contact
masaori@apache.org.

[trafficserver] 03/03: Add HTTP/0.9 exchange support on quic client

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

masaori pushed a commit to branch quic-latest
in repository https://gitbox.apache.org/repos/asf/trafficserver.git

commit 6212927aa508e8c365a405e7e0e2db47547f54a5
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Fri Apr 13 15:55:58 2018 +0900

    Add HTTP/0.9 exchange support on quic client
    
    Out of scope: starting session or transaction of HTTP/QUIC
---
 cmd/traffic_quic/quic_client.cc  | 80 +++++++++++++++++++++++++++++++++++++++-
 cmd/traffic_quic/quic_client.h   | 19 ++++++++--
 cmd/traffic_quic/traffic_quic.cc |  6 ++-
 3 files changed, 99 insertions(+), 6 deletions(-)

diff --git a/cmd/traffic_quic/quic_client.cc b/cmd/traffic_quic/quic_client.cc
index 3a2e661..2396414 100644
--- a/cmd/traffic_quic/quic_client.cc
+++ b/cmd/traffic_quic/quic_client.cc
@@ -72,8 +72,12 @@ QUICClient::state_http_server_open(int event, void *data)
   switch (event) {
   case NET_EVENT_OPEN: {
     // TODO: create ProxyServerSession / ProxyServerTransaction
-    // TODO: send HTTP/0.9 message
     Debug("quic_client", "start proxy server ssn/txn");
+
+    QUICNetVConnection *conn = static_cast<QUICNetVConnection *>(data);
+    QUICClientApp *app       = new QUICClientApp(conn);
+    app->start(this->_path);
+
     break;
   }
   case NET_EVENT_OPEN_FAILED: {
@@ -90,3 +94,77 @@ QUICClient::state_http_server_open(int event, void *data)
 
   return 0;
 }
+
+//
+// QUICClientApp
+//
+#define QUICClientAppDebug(fmt, ...) \
+  Debug("quic_client_app", "[%" PRIx64 "] " fmt, static_cast<uint64_t>(this->_qc->connection_id()), ##__VA_ARGS__)
+
+QUICClientApp::QUICClientApp(QUICNetVConnection *qvc) : QUICApplication(qvc)
+{
+  this->_qc->stream_manager()->set_default_application(this);
+
+  SET_HANDLER(&QUICClientApp::main_event_handler);
+}
+
+void
+QUICClientApp::start(const char *path)
+{
+  QUICStreamId stream_id;
+  QUICErrorUPtr error = this->_qc->stream_manager()->create_bidi_stream(stream_id);
+
+  if (error->cls != QUICErrorClass::NONE) {
+    Error("%s", error->msg);
+    ink_assert(abort);
+  }
+
+  // TODO: move to transaction
+  char request[1024] = {0};
+  int request_len    = snprintf(request, sizeof(request), "GET %s\r\n", path);
+
+  QUICClientAppDebug("\n%s", request);
+
+  QUICStreamIO *stream_io = this->_find_stream_io(stream_id);
+
+  stream_io->write(reinterpret_cast<uint8_t *>(request), request_len);
+  stream_io->write_reenable();
+}
+
+int
+QUICClientApp::main_event_handler(int event, Event *data)
+{
+  QUICClientAppDebug("%s (%d)", get_vc_event_name(event), event);
+
+  VIO *vio                = reinterpret_cast<VIO *>(data);
+  QUICStreamIO *stream_io = this->_find_stream_io(vio);
+
+  if (stream_io == nullptr) {
+    QUICClientAppDebug("Unknown Stream");
+    return -1;
+  }
+
+  switch (event) {
+  case VC_EVENT_READ_READY:
+  case VC_EVENT_READ_COMPLETE:
+    if (stream_io->is_read_avail_more_than(0)) {
+      uint8_t response[1024] = {0};
+      stream_io->read(response, sizeof(response));
+      QUICClientAppDebug("\n%s", response);
+    }
+    break;
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE:
+    break;
+  case VC_EVENT_EOS:
+  case VC_EVENT_ERROR:
+  case VC_EVENT_INACTIVITY_TIMEOUT:
+  case VC_EVENT_ACTIVE_TIMEOUT:
+    ink_assert(false);
+    break;
+  default:
+    break;
+  }
+
+  return EVENT_CONT;
+}
diff --git a/cmd/traffic_quic/quic_client.h b/cmd/traffic_quic/quic_client.h
index ea71739..15071c3 100644
--- a/cmd/traffic_quic/quic_client.h
+++ b/cmd/traffic_quic/quic_client.h
@@ -28,10 +28,13 @@
 #include "I_NetVConnection.h"
 #include "P_QUICNetProcessor.h"
 
+#include "QUICApplication.h"
+
 class QUICClient : public Continuation
 {
 public:
-  QUICClient(const char *addr, const char *port) : Continuation(new_ProxyMutex()), _remote_addr(addr), _remote_port(port)
+  QUICClient(const char *addr, const char *port, const char *path)
+    : Continuation(new_ProxyMutex()), _remote_addr(addr), _remote_port(port), _path(path)
   {
     SET_HANDLER(&QUICClient::start);
   };
@@ -41,7 +44,17 @@ public:
   int state_http_server_open(int event, void *data);
 
 private:
-  const char *_remote_addr;
-  const char *_remote_port;
+  const char *_remote_addr           = nullptr;
+  const char *_remote_port           = nullptr;
   struct addrinfo *_remote_addr_info = nullptr;
+  const char *_path                  = nullptr;
+};
+
+class QUICClientApp : public QUICApplication
+{
+public:
+  QUICClientApp(QUICNetVConnection *qvc);
+
+  void start(const char *path);
+  int main_event_handler(int event, Event *data);
 };
diff --git a/cmd/traffic_quic/traffic_quic.cc b/cmd/traffic_quic/traffic_quic.cc
index c46e80f..2098d49 100644
--- a/cmd/traffic_quic/traffic_quic.cc
+++ b/cmd/traffic_quic/traffic_quic.cc
@@ -51,11 +51,13 @@ main(int argc, const char **argv)
 
   char addr[1024]       = "127.0.0.1";
   char port[16]         = "4433";
-  char debug_tags[1024] = "quic|udp";
+  char path[1018]       = "/";
+  char debug_tags[1024] = "quic";
 
   const ArgumentDescription argument_descriptions[] = {
     {"addr", 'a', "Address", "S1023", addr, nullptr, nullptr},
     {"port", 'p', "Port", "S15", port, nullptr, nullptr},
+    {"path", 'P', "Path", "S1017", path, nullptr, nullptr},
     {"debug", 'T', "Vertical-bar-separated Debug Tags", "S1023", debug_tags, nullptr, nullptr},
     HELP_ARGUMENT_DESCRIPTION(),
     VERSION_ARGUMENT_DESCRIPTION(),
@@ -87,7 +89,7 @@ main(int argc, const char **argv)
   udpNet.start(1, stacksize);
   quic_NetProcessor.start(-1, stacksize);
 
-  QUICClient client(addr, port);
+  QUICClient client(addr, port, path);
   eventProcessor.schedule_in(&client, 1, ET_NET);
 
   this_thread()->execute();

-- 
To stop receiving notification emails like this one, please contact
masaori@apache.org.