You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by ki...@apache.org on 2015/03/31 10:07:49 UTC

[1/2] trafficserver git commit: TS-1611: Async request support in lua plugin

Repository: trafficserver
Updated Branches:
  refs/heads/master bdf84381d -> 68b4d8536


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_transform.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_transform.c b/plugins/experimental/ts_lua/ts_lua_transform.c
index 3be1932..ce7db9b 100644
--- a/plugins/experimental/ts_lua/ts_lua_transform.c
+++ b/plugins/experimental/ts_lua/ts_lua_transform.c
@@ -20,22 +20,26 @@
 #include "ts_lua_util.h"
 
 
-static int ts_lua_transform_handler(TSCont contp, ts_lua_transform_ctx *transform_ctx);
+static int ts_lua_transform_handler(TSCont contp, ts_lua_http_transform_ctx *transform_ctx, TSEvent event, int n);
 
 
 int
-ts_lua_transform_entry(TSCont contp, TSEvent event, void *edata ATS_UNUSED)
+ts_lua_transform_entry(TSCont contp, TSEvent ev, void *edata)
 {
+  int n, event;
   TSVIO input_vio;
+  ts_lua_http_transform_ctx *transform_ctx;
 
-  ts_lua_transform_ctx *transform_ctx = (ts_lua_transform_ctx *)TSContDataGet(contp);
+  event = (int)ev;
+  transform_ctx = (ts_lua_http_transform_ctx *)TSContDataGet(contp);
 
   if (TSVConnClosedGet(contp)) {
-    TSContDestroy(contp);
-    ts_lua_destroy_transform_ctx(transform_ctx);
+    ts_lua_destroy_http_transform_ctx(transform_ctx);
     return 0;
   }
 
+  n = 0;
+
   switch (event) {
   case TS_EVENT_ERROR:
     input_vio = TSVConnWriteVIOGet(contp);
@@ -46,9 +50,12 @@ ts_lua_transform_entry(TSCont contp, TSEvent event, void *edata ATS_UNUSED)
     TSVConnShutdown(TSTransformOutputVConnGet(contp), 0, 1);
     break;
 
+  case TS_LUA_EVENT_COROUTINE_CONT:
+    n = (intptr_t)edata;
+
   case TS_EVENT_VCONN_WRITE_READY:
   default:
-    ts_lua_transform_handler(contp, transform_ctx);
+    ts_lua_transform_handler(contp, transform_ctx, event, n);
     break;
   }
 
@@ -56,61 +63,91 @@ ts_lua_transform_entry(TSCont contp, TSEvent event, void *edata ATS_UNUSED)
 }
 
 static int
-ts_lua_transform_handler(TSCont contp, ts_lua_transform_ctx *transform_ctx)
+ts_lua_transform_handler(TSCont contp, ts_lua_http_transform_ctx *transform_ctx, TSEvent event, int n)
 {
   TSVConn output_conn;
   TSVIO input_vio;
   TSIOBufferReader input_reader;
   TSIOBufferBlock blk;
-  int64_t towrite, blk_len, upstream_done, avail, left;
+  int64_t toread, towrite, blk_len, upstream_done, input_avail, l;
   const char *start;
   const char *res;
   size_t res_len;
-  int ret, eos;
+  int ret, eos, write_down, rc, top;
+  ts_lua_coroutine *crt;
+  ts_lua_cont_info *ci;
 
   lua_State *L;
   TSMutex mtxp;
 
-  L = transform_ctx->hctx->lua;
-  mtxp = transform_ctx->hctx->mctx->mutexp;
+  ci = &transform_ctx->cinfo;
+  crt = &ci->routine;
+
+  mtxp = crt->mctx->mutexp;
+  L = crt->lua;
 
   output_conn = TSTransformOutputVConnGet(contp);
   input_vio = TSVConnWriteVIOGet(contp);
-  input_reader = TSVIOReaderGet(input_vio);
-
-  if (!transform_ctx->output_buffer) {
-    transform_ctx->output_buffer = TSIOBufferCreate();
-    transform_ctx->output_reader = TSIOBufferReaderAlloc(transform_ctx->output_buffer);
-    transform_ctx->output_vio = TSVConnWrite(output_conn, contp, transform_ctx->output_reader, INT64_MAX);
-  }
 
   if (!TSVIOBufferGet(input_vio)) {
-    TSVIONBytesSet(transform_ctx->output_vio, transform_ctx->total);
-    TSVIOReenable(transform_ctx->output_vio);
-    return 1;
+    if (transform_ctx->output.vio) {
+      TSVIONBytesSet(transform_ctx->output.vio, transform_ctx->total);
+      TSVIOReenable(transform_ctx->output.vio);
+    }
+    return 0;
   }
 
-  if (transform_ctx->eos) {
-    return 1;
+  input_reader = TSVIOReaderGet(input_vio);
+
+  if (!transform_ctx->output.buffer) {
+    transform_ctx->output.buffer = TSIOBufferCreate();
+    transform_ctx->output.reader = TSIOBufferReaderAlloc(transform_ctx->output.buffer);
+
+    transform_ctx->reserved.buffer = TSIOBufferCreate();
+    transform_ctx->reserved.reader = TSIOBufferReaderAlloc(transform_ctx->reserved.buffer);
+
+    transform_ctx->upstream_bytes = TSVIONBytesGet(input_vio);
+    transform_ctx->downstream_bytes = INT64_MAX;
   }
 
-  left = towrite = TSVIONTodoGet(input_vio);
+  input_avail = TSIOBufferReaderAvail(input_reader);
   upstream_done = TSVIONDoneGet(input_vio);
-  avail = TSIOBufferReaderAvail(input_reader);
-  eos = 0;
+  toread = TSVIONTodoGet(input_vio);
 
-  if (left <= avail)
+  if (toread <= input_avail) { // upstream finished
     eos = 1;
+  } else {
+    eos = 0;
+  }
 
-  if (towrite > avail)
-    towrite = avail;
+  if (input_avail > 0) {
+    // move to the reserved.buffer
+    TSIOBufferCopy(transform_ctx->reserved.buffer, input_reader, input_avail, 0);
 
-  TSMutexLock(mtxp);
+    // reset input
+    TSIOBufferReaderConsume(input_reader, input_avail);
+    TSVIONDoneSet(input_vio, upstream_done + input_avail);
+  }
 
-  blk = TSIOBufferReaderStart(input_reader);
+  write_down = 0;
+  towrite = TSIOBufferReaderAvail(transform_ctx->reserved.reader);
+
+  TSMutexLock(mtxp);
+  ts_lua_set_cont_info(L, ci);
 
   do {
-    start = TSIOBufferBlockReadStart(blk, input_reader, &blk_len);
+    if (event == TS_LUA_EVENT_COROUTINE_CONT) {
+      event = 0;
+      goto launch;
+    } else {
+      n = 2;
+    }
+
+    if (towrite == 0)
+      break;
+
+    blk = TSIOBufferReaderStart(transform_ctx->reserved.reader);
+    start = TSIOBufferBlockReadStart(blk, transform_ctx->reserved.reader, &blk_len);
 
     lua_pushlightuserdata(L, transform_ctx);
     lua_rawget(L, LUA_GLOBALSINDEX); /* push function */
@@ -118,54 +155,90 @@ ts_lua_transform_handler(TSCont contp, ts_lua_transform_ctx *transform_ctx)
     if (towrite > blk_len) {
       lua_pushlstring(L, start, (size_t)blk_len);
       towrite -= blk_len;
+      TSIOBufferReaderConsume(transform_ctx->reserved.reader, blk_len);
     } else {
       lua_pushlstring(L, start, (size_t)towrite);
+      TSIOBufferReaderConsume(transform_ctx->reserved.reader, towrite);
       towrite = 0;
     }
 
     if (!towrite && eos) {
-      lua_pushinteger(L, 1); /* second param, not finish */
+      lua_pushinteger(L, 1); /* second param, data finished */
     } else {
-      lua_pushinteger(L, 0); /* second param, not finish */
+      lua_pushinteger(L, 0); /* second param, data not finish */
     }
 
-    if (lua_pcall(L, 2, 2, 0)) {
-      TSError("lua_pcall failed: %s", lua_tostring(L, -1));
+  launch:
+    rc = lua_resume(L, n);
+    top = lua_gettop(L);
+
+    switch (rc) {
+    case LUA_YIELD: // coroutine yield
+      TSMutexUnlock(mtxp);
+      return 0;
+
+    case 0: // coroutine success
+      if (top == 2) {
+        ret = lua_tointeger(L, -1); /* 0 is not finished, 1 is finished */
+        res = lua_tolstring(L, -2, &res_len);
+      } else { // what hells code are you writing ?
+        ret = 0;
+        res = NULL;
+        res_len = 0;
+      }
+      break;
+
+    default: // coroutine failed
+      ee("lua_resume failed: %s", lua_tostring(L, -1));
+      ret = 1;
+      res = NULL;
+      res_len = 0;
+      break;
     }
 
-    ret = lua_tointeger(L, -1); /* 0 is not finished, 1 is finished */
-    res = lua_tolstring(L, -2, &res_len);
+    if (res && res_len > 0) {
+      if (!transform_ctx->output.vio) {
+        l = transform_ctx->downstream_bytes;
+        if (ret) {
+          l = res_len;
+        }
+
+        transform_ctx->output.vio = TSVConnWrite(output_conn, contp, transform_ctx->output.reader, l); // HttpSM go on
+      }
 
-    if (res && res_len) {
-      TSIOBufferWrite(transform_ctx->output_buffer, res, res_len);
+      TSIOBufferWrite(transform_ctx->output.buffer, res, res_len);
       transform_ctx->total += res_len;
+      write_down = 1;
     }
 
-    lua_pop(L, 2);
+    lua_pop(L, top);
 
     if (ret || (eos && !towrite)) { // EOS
       eos = 1;
       break;
     }
 
-    blk = TSIOBufferBlockNext(blk);
-
-  } while (blk && towrite > 0);
+  } while (towrite > 0);
 
   TSMutexUnlock(mtxp);
 
-  TSIOBufferReaderConsume(input_reader, avail);
-  TSVIONDoneSet(input_vio, upstream_done + avail);
+  if (eos && !transform_ctx->output.vio)
+    transform_ctx->output.vio = TSVConnWrite(output_conn, contp, transform_ctx->output.reader, 0);
+
+  if (write_down || eos)
+    TSVIOReenable(transform_ctx->output.vio);
 
-  if (eos) {
-    transform_ctx->eos = 1;
-    TSVIONBytesSet(transform_ctx->output_vio, transform_ctx->total);
-    TSVIOReenable(transform_ctx->output_vio);
+  if (toread > input_avail) { // upstream not finished.
+    if (eos) {
+      TSVIONBytesSet(transform_ctx->output.vio, transform_ctx->total);
+      TSContCall(TSVIOContGet(input_vio), TS_EVENT_VCONN_EOS, input_vio);
+    } else {
+      TSContCall(TSVIOContGet(input_vio), TS_EVENT_VCONN_WRITE_READY, input_vio);
+    }
+  } else { // upstream is finished.
+    TSVIONBytesSet(transform_ctx->output.vio, transform_ctx->total);
     TSContCall(TSVIOContGet(input_vio), TS_EVENT_VCONN_WRITE_COMPLETE, input_vio);
-  } else {
-    TSVIOReenable(transform_ctx->output_vio);
-    TSContCall(TSVIOContGet(input_vio), TS_EVENT_VCONN_WRITE_READY, input_vio);
   }
 
-  return 1;
+  return 0;
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_util.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_util.c b/plugins/experimental/ts_lua/ts_lua_util.c
index c7833c8..31bf80f 100644
--- a/plugins/experimental/ts_lua/ts_lua_util.c
+++ b/plugins/experimental/ts_lua/ts_lua_util.c
@@ -19,6 +19,7 @@
 
 #include "ts_lua_util.h"
 #include "ts_lua_remap.h"
+#include "ts_lua_constant.h"
 #include "ts_lua_client_request.h"
 #include "ts_lua_server_request.h"
 #include "ts_lua_server_response.h"
@@ -33,6 +34,8 @@
 #include "ts_lua_mgmt.h"
 #include "ts_lua_package.h"
 #include "ts_lua_stat.h"
+#include "ts_lua_fetch.h"
+#include "ts_lua_http_intercept.h"
 
 static lua_State *ts_lua_new_state();
 static void ts_lua_init_registry(lua_State *L);
@@ -261,6 +264,7 @@ ts_lua_inject_ts_api(lua_State *L)
   lua_newtable(L);
 
   ts_lua_inject_remap_api(L);
+  ts_lua_inject_constant_api(L);
 
   ts_lua_inject_client_request_api(L);
   ts_lua_inject_server_request_api(L);
@@ -273,11 +277,13 @@ ts_lua_inject_ts_api(lua_State *L)
   ts_lua_inject_hook_api(L);
 
   ts_lua_inject_http_api(L);
+  ts_lua_inject_intercept_api(L);
   ts_lua_inject_misc_api(L);
   ts_lua_inject_crypto_api(L);
   ts_lua_inject_mgmt_api(L);
   ts_lua_inject_package_api(L);
   ts_lua_inject_stat_api(L);
+  ts_lua_inject_fetch_api(L);
 
   lua_getglobal(L, "package");
   lua_getfield(L, -1, "loaded");
@@ -311,6 +317,28 @@ ts_lua_get_instance_conf(lua_State *L)
 }
 
 void
+ts_lua_set_cont_info(lua_State *L, ts_lua_cont_info *ci)
+{
+  lua_pushliteral(L, "__ts_cont_info");
+  lua_pushlightuserdata(L, ci);
+  lua_rawset(L, LUA_GLOBALSINDEX);
+}
+
+ts_lua_cont_info *
+ts_lua_get_cont_info(lua_State *L)
+{
+  ts_lua_cont_info *ci;
+
+  lua_pushliteral(L, "__ts_cont_info");
+  lua_rawget(L, LUA_GLOBALSINDEX);
+  ci = lua_touserdata(L, -1);
+
+  lua_pop(L, 1); // pop the coroutine out
+
+  return ci;
+}
+
+void
 ts_lua_set_http_ctx(lua_State *L, ts_lua_http_ctx *ctx)
 {
   lua_pushliteral(L, "__ts_http_ctx");
@@ -335,22 +363,19 @@ ts_lua_get_http_ctx(lua_State *L)
 ts_lua_http_ctx *
 ts_lua_create_http_ctx(ts_lua_main_ctx *main_ctx, ts_lua_instance_conf *conf)
 {
-  size_t i, size;
+  ts_lua_coroutine *crt;
   ts_lua_http_ctx *http_ctx;
   lua_State *L;
   lua_State *l;
 
   L = main_ctx->lua;
 
-  size = TS_LUA_MEM_ALIGN(sizeof(ts_lua_http_ctx));
-  http_ctx = TSmalloc(size);
-
-  for (i = 0; i < TS_LUA_ALIGN_COUNT(size); i++) {
-    ((void **)http_ctx)[i] = 0;
-  }
+  http_ctx = TSmalloc(sizeof(ts_lua_http_ctx));
+  memset(http_ctx, 0, sizeof(ts_lua_http_ctx));
 
-  http_ctx->lua = lua_newthread(L);
-  l = http_ctx->lua;
+  // create coroutine for http_ctx
+  crt = &http_ctx->cinfo.routine;
+  l = lua_newthread(L);
 
   lua_pushlightuserdata(L, conf);
   lua_rawget(L, LUA_REGISTRYINDEX);
@@ -366,13 +391,15 @@ ts_lua_create_http_ctx(ts_lua_main_ctx *main_ctx, ts_lua_instance_conf *conf)
 
   lua_replace(l, LUA_GLOBALSINDEX);
 
-  http_ctx->ref = luaL_ref(L, LUA_REGISTRYINDEX);
+  // init coroutine
+  crt->ref = luaL_ref(L, LUA_REGISTRYINDEX);
+  crt->lua = l;
+  crt->mctx = main_ctx;
 
-  http_ctx->mctx = main_ctx;
   http_ctx->instance_conf = conf;
 
-  ts_lua_set_http_ctx(http_ctx->lua, http_ctx);
-  ts_lua_create_context_table(http_ctx->lua);
+  ts_lua_set_http_ctx(l, http_ctx);
+  ts_lua_create_context_table(l);
 
   return http_ctx;
 }
@@ -381,9 +408,9 @@ ts_lua_create_http_ctx(ts_lua_main_ctx *main_ctx, ts_lua_instance_conf *conf)
 void
 ts_lua_destroy_http_ctx(ts_lua_http_ctx *http_ctx)
 {
-  ts_lua_main_ctx *main_ctx;
+  ts_lua_cont_info *ci;
 
-  main_ctx = http_ctx->mctx;
+  ci = &http_ctx->cinfo;
 
   if (!http_ctx->remap) {
     if (http_ctx->client_request_bufp) {
@@ -413,7 +440,7 @@ ts_lua_destroy_http_ctx(ts_lua_http_ctx *http_ctx)
     TSMBufferDestroy(http_ctx->cached_response_bufp);
   }
 
-  luaL_unref(main_ctx->lua, LUA_REGISTRYINDEX, http_ctx->ref);
+  ts_lua_release_cont_info(ci);
   TSfree(http_ctx);
 }
 
@@ -440,29 +467,40 @@ ts_lua_get_http_intercept_ctx(lua_State *L)
 }
 
 ts_lua_http_intercept_ctx *
-ts_lua_create_http_intercept_ctx(ts_lua_http_ctx *http_ctx)
+ts_lua_create_http_intercept_ctx(lua_State *L, ts_lua_http_ctx *http_ctx, int n)
 {
-  size_t i, size;
-  lua_State *L;
+  int i;
+  lua_State *l;
+  ts_lua_cont_info *hci;
+  ts_lua_coroutine *crt;
   ts_lua_http_intercept_ctx *ictx;
 
-  L = http_ctx->lua;
+  hci = &http_ctx->cinfo;
 
-  size = TS_LUA_MEM_ALIGN(sizeof(ts_lua_http_intercept_ctx));
-  ictx = TSmalloc(size);
+  ictx = TSmalloc(sizeof(ts_lua_http_intercept_ctx));
+  memset(ictx, 0, sizeof(ts_lua_http_intercept_ctx));
 
-  for (i = 0; i < TS_LUA_ALIGN_COUNT(size); i++) {
-    ((void **)ictx)[i] = 0;
-  }
+  ictx->hctx = http_ctx;
 
-  ictx->lua = lua_newthread(L);
+  // create lua_thread
+  l = lua_newthread(L);
 
-  ictx->ref = luaL_ref(L, LUA_REGISTRYINDEX);
+  // init the coroutine
+  crt = &ictx->cinfo.routine;
+  crt->mctx = hci->routine.mctx;
+  crt->lua = l;
+  crt->ref = luaL_ref(L, LUA_REGISTRYINDEX);
 
-  ictx->mctx = http_ctx->mctx;
-  ictx->hctx = http_ctx;
+  // Todo: replace the global, context table for crt->lua
 
-  ts_lua_set_http_intercept_ctx(ictx->lua, ictx);
+  // replicate the param
+  for (i = 0; i < n; i++) {
+    lua_pushvalue(L, i + 1);
+  }
+
+  lua_xmove(L, l, n); // move the intercept function and params to the new lua_thread
+
+  ts_lua_set_http_intercept_ctx(l, ictx);
 
   return ictx;
 }
@@ -470,240 +508,276 @@ ts_lua_create_http_intercept_ctx(ts_lua_http_ctx *http_ctx)
 void
 ts_lua_destroy_http_intercept_ctx(ts_lua_http_intercept_ctx *ictx)
 {
-  ts_lua_main_ctx *main_ctx;
-  struct ict_item *node, *snode;
+  ts_lua_cont_info *ci;
 
-  main_ctx = ictx->mctx;
+  ci = &ictx->cinfo;
 
-  if (ictx->net_vc)
+  if (ictx->net_vc) {
     TSVConnClose(ictx->net_vc);
+  }
 
   TS_LUA_RELEASE_IO_HANDLE((&ictx->input));
   TS_LUA_RELEASE_IO_HANDLE((&ictx->output));
 
-  node = ictx->ict_chain;
+  ts_lua_release_cont_info(ci);
+  TSfree(ictx);
+}
 
-  while (node) {
-    if (node->cleanup)
-      node->cleanup(node);
+void
+ts_lua_set_http_transform_ctx(lua_State *L, ts_lua_http_transform_ctx *tctx)
+{
+  lua_pushliteral(L, "__ts_http_transform_ctx");
+  lua_pushlightuserdata(L, tctx);
+  lua_rawset(L, LUA_GLOBALSINDEX);
+}
 
-    snode = node;
-    node = node->next;
+ts_lua_http_transform_ctx *
+ts_lua_get_http_transform_ctx(lua_State *L)
+{
+  ts_lua_http_transform_ctx *tctx;
 
-    TSfree(snode);
-  }
+  lua_pushliteral(L, "__ts_http_transform_ctx");
+  lua_rawget(L, LUA_GLOBALSINDEX);
+  tctx = lua_touserdata(L, -1);
 
-  luaL_unref(main_ctx->lua, LUA_REGISTRYINDEX, ictx->ref);
-  TSfree(ictx);
-  return;
+  lua_pop(L, 1); // pop the ictx out
+
+  return tctx;
+}
+
+ts_lua_http_transform_ctx *
+ts_lua_create_http_transform_ctx(ts_lua_http_ctx *http_ctx, TSVConn connp)
+{
+  lua_State *L;
+  ts_lua_cont_info *hci;
+  ts_lua_cont_info *ci;
+  ts_lua_coroutine *crt;
+  ts_lua_http_transform_ctx *transform_ctx;
+
+  hci = &http_ctx->cinfo;
+  L = hci->routine.lua;
+
+  transform_ctx = (ts_lua_http_transform_ctx *)TSmalloc(sizeof(ts_lua_http_transform_ctx));
+  memset(transform_ctx, 0, sizeof(ts_lua_http_transform_ctx));
+
+  transform_ctx->hctx = http_ctx;
+  TSContDataSet(connp, transform_ctx);
+
+  ci = &transform_ctx->cinfo;
+  ci->contp = connp;
+  ci->mutex = TSContMutexGet((TSCont)http_ctx->txnp);
+
+  crt = &ci->routine;
+  crt->mctx = hci->routine.mctx;
+  crt->lua = lua_newthread(L);
+  crt->ref = luaL_ref(L, LUA_REGISTRYINDEX);
+  ts_lua_set_http_transform_ctx(crt->lua, transform_ctx);
+
+  lua_pushlightuserdata(L, transform_ctx);
+  lua_pushvalue(L, 2);
+  lua_rawset(L, LUA_GLOBALSINDEX); // L[GLOBAL][transform_ctx] = transform handler
+
+  return transform_ctx;
 }
 
 void
-ts_lua_destroy_transform_ctx(ts_lua_transform_ctx *transform_ctx)
+ts_lua_destroy_http_transform_ctx(ts_lua_http_transform_ctx *transform_ctx)
 {
+  ts_lua_cont_info *ci;
+
   if (!transform_ctx)
     return;
 
-  if (transform_ctx->output_reader)
-    TSIOBufferReaderFree(transform_ctx->output_reader);
+  ci = &transform_ctx->cinfo;
+
+  TS_LUA_RELEASE_IO_HANDLE((&transform_ctx->output));
+  TS_LUA_RELEASE_IO_HANDLE((&transform_ctx->reserved));
 
-  if (transform_ctx->output_buffer)
-    TSIOBufferDestroy(transform_ctx->output_buffer);
+  ts_lua_release_cont_info(ci);
 
   TSfree(transform_ctx);
 }
 
 int
-ts_lua_http_cont_handler(TSCont contp, TSEvent event, void *edata)
+ts_lua_http_cont_handler(TSCont contp, TSEvent ev, void *edata)
 {
-  TSHttpTxn txnp = (TSHttpTxn)edata;
-  int ret;
-  lua_State *l;
+  TSHttpTxn txnp;
+  int event, ret, rc, n, t;
+  lua_State *L;
   ts_lua_http_ctx *http_ctx;
   ts_lua_main_ctx *main_ctx;
+  ts_lua_cont_info *ci;
+  ts_lua_coroutine *crt;
 
+  event = (int)ev;
   http_ctx = (ts_lua_http_ctx *)TSContDataGet(contp);
-  main_ctx = http_ctx->mctx;
+  ci = &http_ctx->cinfo;
+  crt = &ci->routine;
+
+  main_ctx = crt->mctx;
+  L = crt->lua;
+
+  txnp = http_ctx->txnp;
 
-  ret = 0;
-  l = http_ctx->lua;
+  rc = ret = 0;
 
   TSMutexLock(main_ctx->mutexp);
+  ts_lua_set_cont_info(L, ci);
 
   switch (event) {
   case TS_EVENT_HTTP_POST_REMAP:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_POST_REMAP);
+    lua_getglobal(L, TS_LUA_FUNCTION_POST_REMAP);
 
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_CACHE_LOOKUP_COMPLETE:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_CACHE_LOOKUP_COMPLETE);
+    lua_getglobal(L, TS_LUA_FUNCTION_CACHE_LOOKUP_COMPLETE);
 
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_SEND_REQUEST_HDR:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_SEND_REQUEST);
+    lua_getglobal(L, TS_LUA_FUNCTION_SEND_REQUEST);
 
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_READ_RESPONSE_HDR:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_READ_RESPONSE);
+    lua_getglobal(L, TS_LUA_FUNCTION_READ_RESPONSE);
 
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_SEND_RESPONSE_HDR:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_SEND_RESPONSE);
+    lua_getglobal(L, TS_LUA_FUNCTION_SEND_RESPONSE);
 
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_READ_REQUEST_HDR:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_READ_REQUEST);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    lua_getglobal(L, TS_LUA_FUNCTION_READ_REQUEST);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_TXN_START:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_TXN_START);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    lua_getglobal(L, TS_LUA_FUNCTION_TXN_START);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_PRE_REMAP:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_PRE_REMAP);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    lua_getglobal(L, TS_LUA_FUNCTION_PRE_REMAP);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_OS_DNS:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_OS_DNS);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    lua_getglobal(L, TS_LUA_FUNCTION_OS_DNS);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_SELECT_ALT:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_SELECT_ALT);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    lua_getglobal(L, TS_LUA_FUNCTION_SELECT_ALT);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_READ_CACHE_HDR:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_READ_CACHE);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
-      }
-
-      ret = lua_tointeger(l, -1);
+    lua_getglobal(L, TS_LUA_FUNCTION_READ_CACHE);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      ret = lua_resume(L, 0);
     }
 
     break;
 
   case TS_EVENT_HTTP_TXN_CLOSE:
-    lua_getglobal(l, TS_LUA_FUNCTION_TXN_CLOSE);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        TSError("lua_pcall failed: %s", lua_tostring(l, -1));
+    lua_getglobal(L, TS_LUA_FUNCTION_TXN_CLOSE);
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+      if (lua_pcall(L, 0, 1, 0)) {
+        TSError("lua_pcall failed: %s", lua_tostring(L, -1));
       }
-
-      ret = lua_tointeger(l, -1);
     }
 
     ts_lua_destroy_http_ctx(http_ctx);
-    TSContDestroy(contp);
     break;
 
+  case TS_LUA_EVENT_COROUTINE_CONT:
+    n = (intptr_t)edata;
+    ret = lua_resume(L, n);
+
   default:
     break;
   }
 
+  switch (ret) {
+  case 0: // coroutine succeed
+    t = lua_gettop(L);
+    if (t > 0) {
+      rc = lua_tointeger(L, -1);
+      lua_pop(L, 1);
+    }
+    break;
+
+  case LUA_YIELD: // coroutine yield
+    rc = 1;
+    break;
+
+  default: // coroutine failed
+    ee("lua_resume failed: %s", lua_tostring(L, -1));
+    rc = -1;
+    lua_pop(L, 1);
+    break;
+  }
+
   TSMutexUnlock(main_ctx->mutexp);
 
-  if (ret) {
+  if (rc == 0) {
+    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
+
+  } else if (rc < 0) {
     TSHttpTxnReenable(txnp, TS_EVENT_HTTP_ERROR);
 
   } else {
-    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
+    // wait for async
   }
 
   return 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_util.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_util.h b/plugins/experimental/ts_lua/ts_lua_util.h
index c2b259c..d8a003b 100644
--- a/plugins/experimental/ts_lua/ts_lua_util.h
+++ b/plugins/experimental/ts_lua/ts_lua_util.h
@@ -36,15 +36,21 @@ int ts_lua_del_instance(ts_lua_instance_conf *conf);
 void ts_lua_set_instance_conf(lua_State *L, ts_lua_instance_conf *conf);
 ts_lua_instance_conf *ts_lua_get_instance_conf(lua_State *L);
 
+void ts_lua_set_cont_info(lua_State *L, ts_lua_cont_info *ci);
+ts_lua_cont_info *ts_lua_get_cont_info(lua_State *L);
+
 void ts_lua_set_http_ctx(lua_State *L, ts_lua_http_ctx *ctx);
 ts_lua_http_ctx *ts_lua_get_http_ctx(lua_State *L);
 
 ts_lua_http_ctx *ts_lua_create_http_ctx(ts_lua_main_ctx *mctx, ts_lua_instance_conf *conf);
 void ts_lua_destroy_http_ctx(ts_lua_http_ctx *http_ctx);
 
-void ts_lua_destroy_transform_ctx(ts_lua_transform_ctx *transform_ctx);
+ts_lua_http_transform_ctx *ts_lua_create_http_transform_ctx(ts_lua_http_ctx *http_ctx, TSVConn connp);
+void ts_lua_destroy_http_transform_ctx(ts_lua_http_transform_ctx *transform_ctx);
+void ts_lua_set_http_transform_ctx(lua_State *L, ts_lua_http_transform_ctx *tctx);
+ts_lua_http_transform_ctx *ts_lua_get_http_transform_ctx(lua_State *L);
 
-ts_lua_http_intercept_ctx *ts_lua_create_http_intercept_ctx(ts_lua_http_ctx *http_ctx);
+ts_lua_http_intercept_ctx *ts_lua_create_http_intercept_ctx(lua_State *L, ts_lua_http_ctx *http_ctx, int n);
 ts_lua_http_intercept_ctx *ts_lua_get_http_intercept_ctx(lua_State *L);
 void ts_lua_destroy_http_intercept_ctx(ts_lua_http_intercept_ctx *ictx);
 


[2/2] trafficserver git commit: TS-1611: Async request support in lua plugin

Posted by ki...@apache.org.
TS-1611: Async request support in lua plugin


Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/68b4d853
Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/68b4d853
Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/68b4d853

Branch: refs/heads/master
Commit: 68b4d8536570167ae1bb8eb812b26aa55c620d08
Parents: bdf8438
Author: Gang Li <po...@gmail.com>
Authored: Tue Mar 31 01:07:21 2015 -0700
Committer: Kit Chan <ki...@apache.org>
Committed: Tue Mar 31 01:07:21 2015 -0700

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 doc/reference/plugins/ts_lua.en.rst             | 322 +++++++---
 plugins/experimental/ts_lua/Makefile.am         |   6 +-
 .../experimental/ts_lua/example/test_fetch.lua  |  52 ++
 .../ts_lua/example/test_fetch_multi.lua         |  64 ++
 .../ts_lua/example/test_global_hook.lua         |  10 +-
 .../experimental/ts_lua/example/test_sleep.lua  |  51 ++
 .../ts_lua/example/test_transform.lua           |  38 +-
 plugins/experimental/ts_lua/ts_lua.c            |  64 +-
 plugins/experimental/ts_lua/ts_lua_common.h     |  78 +--
 plugins/experimental/ts_lua/ts_lua_constant.c   |  41 ++
 plugins/experimental/ts_lua/ts_lua_constant.h   |  25 +
 plugins/experimental/ts_lua/ts_lua_coroutine.c  |  98 +++
 plugins/experimental/ts_lua/ts_lua_coroutine.h  |  71 +++
 plugins/experimental/ts_lua/ts_lua_fetch.c      | 603 +++++++++++++++++++
 plugins/experimental/ts_lua/ts_lua_fetch.h      |  48 ++
 plugins/experimental/ts_lua/ts_lua_hook.c       |  36 +-
 plugins/experimental/ts_lua/ts_lua_http.c       |  46 ++
 .../experimental/ts_lua/ts_lua_http_intercept.c | 241 ++++++--
 .../experimental/ts_lua/ts_lua_http_intercept.h |   1 +
 plugins/experimental/ts_lua/ts_lua_io.c         |  31 +
 plugins/experimental/ts_lua/ts_lua_io.h         |  10 +
 plugins/experimental/ts_lua/ts_lua_misc.c       | 147 +----
 plugins/experimental/ts_lua/ts_lua_transform.c  | 181 ++++--
 plugins/experimental/ts_lua/ts_lua_util.c       | 360 ++++++-----
 plugins/experimental/ts_lua/ts_lua_util.h       |  10 +-
 26 files changed, 2043 insertions(+), 593 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index de36cb8..24a6811 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 6.0.0
 
+  *) [TS-1611] Async request suport in lua plugin.
+
   *) [TS-3456] SSL blind tunnel sometimes not created.
 
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/doc/reference/plugins/ts_lua.en.rst
----------------------------------------------------------------------
diff --git a/doc/reference/plugins/ts_lua.en.rst b/doc/reference/plugins/ts_lua.en.rst
index 5e01c36..a3e9a11 100644
--- a/doc/reference/plugins/ts_lua.en.rst
+++ b/doc/reference/plugins/ts_lua.en.rst
@@ -1480,6 +1480,19 @@ We will get the response like this:
 
 `TOP <#ts-lua-plugin>`_
 
+Number constants
+----------------------
+**context:** global
+
+::
+
+    TS_LUA_INT64_MAX (9223372036854775808)
+    TS_LUA_INT64_MIN (-9223372036854775808L)
+
+These constants are usually used in transform handler.
+
+`TOP <#ts-lua-plugin>`_
+
 ts.http.resp_cache_transformed
 ------------------------------
 **syntax:** *ts.http.resp_cache_transformed(BOOL)*
@@ -1493,11 +1506,7 @@ Here is an example:
 ::
 
     function upper_transform(data, eos)
-        if eos == 1 then
-            return string.upper(data)..'S.H.E.\n', eos
-        else
-            return string.upper(data), eos
-        end
+        return string.upper(data), eos
     end
 
     function do_remap()
@@ -1524,20 +1533,71 @@ Here is an example:
 ::
 
     function upper_transform(data, eos)
+        return string.upper(data), eos
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_RESPONSE_TRANSFORM, upper_transform)
+        ts.http.resp_cache_untransformed(1)
+        return 0
+    end
+
+This function is usually called after we hook TS_LUA_RESPONSE_TRANSFORM.
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.http.resp_transform.get_upstream_bytes
+-----------------------------------------
+**syntax:** *ts.http.resp_transform.get_upstream_bytes()*
+
+**context:** transform handler
+
+**description**: This function can be used to retrive the total bytes to be received from the upstream. If we got
+chunked response body from origin server, TS_LUA_INT64_MAX will be returned.
+
+Here is an example:
+
+::
+
+    local APPEND_DATA = 'TAIL\n'
+
+    function append_transform(data, eos)
+        if ts.ctx['len_set'] == nil then
+            local sz = ts.http.resp_transform.get_upstream_bytes()
+            if sz ~= TS_LUA_INT64_MAX then
+                ts.http.resp_transform.set_downstream_bytes(sz + string.len(APPEND_DATA))
+            end
+
+            ts.ctx['len_set'] = true
+        end
+
         if eos == 1 then
-            return string.upper(data)..'S.H.E.\n', eos
+            return data .. APPEND_DATA, eos
         else
-            return string.upper(data), eos
+            return data, eos
         end
     end
 
     function do_remap()
-        ts.hook(TS_LUA_RESPONSE_TRANSFORM, upper_transform)
+        ts.hook(TS_LUA_RESPONSE_TRANSFORM, append_transform)
+        ts.http.resp_cache_transformed(0)
         ts.http.resp_cache_untransformed(1)
         return 0
     end
 
-This function is usually called after we hook TS_LUA_RESPONSE_TRANSFORM.
+`TOP <#ts-lua-plugin>`_
+
+ts.http.resp_transform.set_downstream_bytes
+-----------------------------------------
+**syntax:** *ts.http.resp_transform.set_downstream_bytes(NUMBER)*
+
+**context:** transform handler
+
+**description**: This function can be used to set the total bytes to be sent to the downstream.
+
+Sometimes we want to set Content-Length header in client_response, and this function should be called before any real
+data is returned from the transform handler.
 
 
 `TOP <#ts-lua-plugin>`_
@@ -1566,9 +1626,9 @@ ts.http.is_internal_request
 ---------------------------
 **syntax:** *ts.http.is_internal_request()*
 
-** context:** do_remap or do_global_* or later
+**context:** do_remap or do_global_* or later
 
-** description**: This function can be used to tell is a request is internal or not
+**description:** This function can be used to tell is a request is internal or not
 
 Here is an example:
 
@@ -1792,24 +1852,111 @@ Here is an example:
 
 `TOP <#ts-lua-plugin>`_
 
-ts.intercept
+ts.fetch
+-----------
+**syntax:** *res = ts.fetch(url, table?)*
+
+**context:** after do_remap
+
+**description:** Issues a synchronous but still non-block http request with the ``url`` and the optional ``table``.
+
+Returns a Lua table with serveral slots (res.status, res.header, res.body, and res.truncated).
+
+``res.status`` holds the response status code.
+
+``res.header`` holds the response header table.
+
+``res.body`` holds the response body which may be truncated, you need to check res.truncated to see if the data is
+truncated. 
+
+Here is a basic example:
+
+::
+
+    function post_remap()
+        local url = string.format('http://%s/foo.txt', ts.ctx['host'])
+        local res = ts.fetch(url)
+        if res.status == 200 then
+            print(res.body)
+        end
+    end
+
+    function do_remap()
+        local inner = ts.http.is_internal_request()
+        if inner ~= 0 then
+            return 0
+        end
+        local host = ts.client_request.header['Host']
+        ts.ctx['host'] = host
+        ts.hook(TS_LUA_HOOK_POST_REMAP, post_remap)
+    end
+
+We can set the optional table with serveral members:
+
+``header`` holds the request header table.
+
+``method`` holds the request method. The default method is 'GET'.
+
+``cliaddr`` holds the request client address in ip:port form. The default cliaddr is '127.0.0.1:33333'
+
+Issuing a post request:
+
+::
+
+    res = ts.fetch('http://xx.com/foo', {method = 'POST', body = 'hello world'})
+
+`TOP <#ts-lua-plugin>`_
+
+ts.fetch_multi
 ------------
-**syntax:** *ts.intercept(FUNCTION)*
+**syntax:** *vec = ts.fetch_multi({{url, table?}, {url, table?}, ...})*
+
+**context:** after do_remap
+
+Just like `ts.fetch`, but supports multiple http requests running in parallel.
+
+This function will fetch all the urls specified by the input table and return a table which contain all the results in
+the same order.
+
+Here is an example:
+
+::
+
+    local vec = ts.fetch_multi({
+                    {'http://xx.com/slayer'},
+                    {'http://xx.com/am', {cliaddr = '192.168.1.19:35423'}},
+                    {'http://xx.com/naga', {method = 'POST', body = 'hello world'}},
+                })
+
+    for i = 1, #(vec) do
+        print(vec[i].status)
+    end
+
+
+`TOP <#ts-lua-plugin>`_
+
+
+ts.http.intercept
+------------
+**syntax:** *ts.http.intercept(FUNCTION, param1?, param2?, ...)*
 
 **context:** do_remap or do_global_*
 
-**description:** Intercepts the client request and processes it in FUNCTION.
+**description:** Intercepts the client request and processes it in FUNCTION with optional params.
 
 We should construct the response for the client request, and the request will not be processed by other modules, like
 hostdb, cache, origin server...
 
+Intercept FUNCTION will be executed in a new lua_thread, so we can delivery optional params from old lua_thread to new
+lua_thread if needed.
+
 Here is an example:
 
 ::
 
     require 'os'
 
-    function send_data()
+    function send_data(dstr)
         local nt = os.time()..' Zheng.\n'
         local resp =  'HTTP/1.0 200 OK\r\n' ..
                       'Server: ATS/3.2.0\r\n' ..
@@ -1820,11 +1967,12 @@ Here is an example:
                       'Cache-Control: max-age=7200\r\n' ..
                       'Accept-Ranges: bytes\r\n\r\n' ..
                       nt
+        print(dstr)
         ts.say(resp)
     end
 
     function do_remap()
-        ts.http.intercept(send_data)
+        ts.http.intercept(send_data, 'hello world')
         return 0
     end
 
@@ -1848,6 +1996,72 @@ Then we will get the response like this:
 
 `TOP <#ts-lua-plugin>`_
 
+ts.http.server_intercept
+------------
+**syntax:** *ts.http.server_intercept(FUNCTION, param1?, param2?, ...)*
+
+**context:** do_remap or do_global_*
+
+**description:** Intercepts the server request and acts as the origin server.
+
+Just like ts.http.intercept, but this function will intercept the server request, and we can acts as the origin server
+in `FUNCTION`.
+
+Here is an example:
+
+::
+
+    require 'os'
+
+    function process_combo(host)
+        local url1 = string.format('http://%s/css/1.css', host)
+        local url2 = string.format('http://%s/css/2.css', host)
+        local url3 = string.format('http://%s/css/3.css', host)
+
+        local hdr = {
+            ['Host'] = host,
+            ['User-Agent'] = 'blur blur',
+        }
+
+        local ct = {
+            header = hdr,
+            method = 'GET'
+        }
+
+        local arr = ts.fetch_multi(
+                {
+                    {url1, ct},
+                    {url2, ct},
+                    {url3, ct},
+                })
+
+        local ctype = arr[1].header['Content-Type']
+        local body = arr[1].body .. arr[2].body .. arr[3].body
+
+        local resp =  'HTTP/1.1 200 OK\r\n' ..
+                      'Server: ATS/5.2.0\r\n' ..
+                      'Last-Modified: ' .. os.date("%a, %d %b %Y %H:%M:%S GMT", os.time()) .. '\r\n' ..
+                      'Cache-Control: max-age=7200\r\n' ..
+                      'Accept-Ranges: bytes\r\n' ..
+                      'Content-Type: ' .. ctype .. '\r\n' ..
+                      'Content-Length: ' .. string.format('%d', string.len(body)) .. '\r\n\r\n' ..
+                      body
+
+        ts.say(resp)
+    end
+
+    function do_remap()
+        local inner =  ts.http.is_internal_request()
+        if inner ~= 0 then
+            return 0
+        end
+
+        local h = ts.client_request.header['Host']
+        ts.http.server_intercept(process_combo, h)
+    end
+
+`TOP <#ts-lua-plugin>`_
+
 ts.say
 ------
 **syntax:** *ts.say(data)*
@@ -1926,7 +2140,7 @@ ts.sleep
 --------
 **syntax:** *ts.sleep(sec)*
 
-**context:** *intercept or server_intercept*
+**context:** *after do_remap*
 
 **description:** Sleeps for the specified seconds without blocking.
 
@@ -1936,80 +2150,19 @@ Here is an example:
 
 ::
 
-    require 'os'
-
-    function send_data()
-        local nt = os.time()..' Zheng.\n'
-        local resp =  'HTTP/1.0 200 OK\r\n' ..
-                      'Server: ATS/3.2.0\r\n' ..
-                      'Content-Type: text/plain\r\n' ..
-                      'Content-Length: ' .. string.format('%d', string.len(nt)) .. '\r\n\r\n' ..
-                      nt
+    function send_response()
         ts.sleep(3)
-        ts.say(resp)
     end
 
-    function do_remap()
-        ts.http.intercept(send_data)
-        return 0
-    end
-
-`TOP <#ts-lua-plugin>`_
-
-ts.server_intercept
--------------------
-**syntax:** *ts.server_intercept(FUNCTION)*
-
-**context:** do_remap or do_global_*
-
-**description:** Intercepts the server request and acts as the origin server.
-
-We should construct the response for the server request, so the request will be processed within FUNCTION in case of
-miss for the cache lookup.
-
-Here is an example:
-
-::
-
-    require 'os'
-
-    function send_data()
-        local nt = os.time()..' Zheng.\n'
-        local resp =  'HTTP/1.0 200 OK\r\n' ..
-                      'Server: ATS/3.2.0\r\n' ..
-                      'Content-Type: text/plain\r\n' ..
-                      'Content-Length: ' .. string.format('%d', string.len(nt)) .. '\r\n' ..
-                      'Last-Modified: ' .. os.date("%a, %d %b %Y %H:%M:%S GMT", os.time()) .. '\r\n' ..
-                      'Connection: keep-alive\r\n' ..
-                      'Cache-Control: max-age=7200\r\n' ..
-                      'Accept-Ranges: bytes\r\n\r\n' ..
-                      nt
-        ts.say(resp)
+    function read_response()
+        ts.sleep(3)
     end
 
     function do_remap()
-        ts.http.server_intercept(send_data)
-        return 0
+        ts.hook(TS_LUA_HOOK_READ_RESPONSE_HDR, read_response)
+        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
     end
 
-Then we will get the response like this:
-
-::
-
-    HTTP/1.1 200 OK
-    Server: ATS/5.0.0
-    Content-Type: text/plain
-    Content-Length: 18
-    Last-Modified: Tue, 18 Mar 2014 08:23:12 GMT
-    Cache-Control: max-age=7200
-    Accept-Ranges: bytes
-    Date: Tue, 18 Mar 2014 12:23:12 GMT
-    Age: 1890
-    Connection: keep-alive
-
-    1395145392 Zheng.
-
-
 `TOP <#ts-lua-plugin>`_
 
 ts.http.config_int_get
@@ -2464,9 +2617,8 @@ be returned with 4 functions to increment, decrement, get and set the value. Tha
 
 Todo
 ====
-* ts.fetch
 * ts.cache_xxx
-* `support lua-5.2 <https://github.com/portl4t/ts-lua/wiki/support-Lua-5.2>`_
+* protocol
 
 Currently when we use ts_lua as a global plugin, each global hook is using a separate lua state for the same
 transaction. This can be wasteful. Also the state cannot be reused for the same transaction across the global hooks. The

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/Makefile.am
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/Makefile.am b/plugins/experimental/ts_lua/Makefile.am
index 4d2a711..3e09ed1 100644
--- a/plugins/experimental/ts_lua/Makefile.am
+++ b/plugins/experimental/ts_lua/Makefile.am
@@ -47,6 +47,10 @@ tslua_la_SOURCES = \
   ts_lua_package.c \
   ts_lua_string.c \
   ts_lua_crypto.c \
-  ts_lua_stat.c
+  ts_lua_stat.c \
+  ts_lua_io.c \
+  ts_lua_coroutine.c \
+  ts_lua_fetch.c \
+  ts_lua_constant.c
 
 tslua_la_LDFLAGS = $(TS_PLUGIN_LDFLAGS)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/example/test_fetch.lua
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/example/test_fetch.lua b/plugins/experimental/ts_lua/example/test_fetch.lua
new file mode 100644
index 0000000..758810a
--- /dev/null
+++ b/plugins/experimental/ts_lua/example/test_fetch.lua
@@ -0,0 +1,52 @@
+--  Licensed to the Apache Software Foundation (ASF) under one
+--  or more contributor license agreements.  See the NOTICE file
+--  distributed with this work for additional information
+--  regarding copyright ownership.  The ASF licenses this file
+--  to you under the Apache License, Version 2.0 (the
+--  "License"); you may not use this file except in compliance
+--  with the License.  You may obtain a copy of the License at
+--
+--  http://www.apache.org/licenses/LICENSE-2.0
+--
+--  Unless required by applicable law or agreed to in writing, software
+--  distributed under the License is distributed on an "AS IS" BASIS,
+--  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+--  See the License for the specific language governing permissions and
+--  limitations under the License.
+
+
+function send_response()
+    if ts.ctx['flen'] ~= nil then
+        ts.client_response.header['Flen'] = ts.ctx['flen']
+    end
+end
+
+function post_remap()
+    local url = string.format('http://%s/foo.txt', ts.ctx['host'])
+    local hdr = {
+        ['Host'] = ts.ctx['host'],
+        ['User-Agent'] = 'dummy',
+    }
+
+    local res = ts.fetch(url, {method = 'GET', header=hdr})
+
+    if res.status == 200 then
+        ts.ctx['flen'] = string.len(res.body)
+        print(res.body)
+    end
+end
+
+
+function do_remap()
+    local inner = ts.http.is_internal_request()
+    if inner ~= 0 then
+        return 0
+    end
+
+    local host = ts.client_request.header['Host']
+    ts.ctx['host'] = host
+
+    ts.hook(TS_LUA_HOOK_POST_REMAP, post_remap)
+    ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+end
+

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/example/test_fetch_multi.lua
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/example/test_fetch_multi.lua b/plugins/experimental/ts_lua/example/test_fetch_multi.lua
new file mode 100644
index 0000000..928ce27
--- /dev/null
+++ b/plugins/experimental/ts_lua/example/test_fetch_multi.lua
@@ -0,0 +1,64 @@
+--  Licensed to the Apache Software Foundation (ASF) under one
+--  or more contributor license agreements.  See the NOTICE file
+--  distributed with this work for additional information
+--  regarding copyright ownership.  The ASF licenses this file
+--  to you under the Apache License, Version 2.0 (the
+--  "License"); you may not use this file except in compliance
+--  with the License.  You may obtain a copy of the License at
+--
+--  http://www.apache.org/licenses/LICENSE-2.0
+--
+--  Unless required by applicable law or agreed to in writing, software
+--  distributed under the License is distributed on an "AS IS" BASIS,
+--  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+--  See the License for the specific language governing permissions and
+--  limitations under the License.
+
+require 'os'
+
+function process_combo(host)
+    local url1 = string.format('http://%s/css/1.css', host)
+    local url2 = string.format('http://%s/css/2.css', host)
+    local url3 = string.format('http://%s/css/3.css', host)
+
+    local hdr = {
+        ['Host'] = host,
+        ['User-Agent'] = 'blur blur',
+    }
+
+    local ct = {
+        header = hdr,
+        method = 'GET'
+    }
+
+    local arr = ts.fetch_multi(
+            {
+                {url1, ct},
+                {url2, ct},
+                {url3, ct},
+            })
+
+    local ctype = arr[1].header['Content-Type']
+    local body = arr[1].body .. arr[2].body .. arr[3].body
+
+    local resp =  'HTTP/1.1 200 OK\r\n' ..
+                  'Server: ATS/5.2.0\r\n' ..
+                  'Last-Modified: ' .. os.date("%a, %d %b %Y %H:%M:%S GMT", os.time()) .. '\r\n' ..
+                  'Cache-Control: max-age=7200\r\n' ..
+                  'Accept-Ranges: bytes\r\n' ..
+                  'Content-Type: ' .. ctype .. '\r\n' ..
+                  'Content-Length: ' .. string.format('%d', string.len(body)) .. '\r\n\r\n' ..
+                  body
+
+    ts.say(resp)
+end
+
+function do_remap()
+    local inner =  ts.http.is_internal_request()
+    if inner ~= 0 then
+        return 0
+    end
+
+    local h = ts.client_request.header['Host']
+    ts.http.intercept(process_combo, h)
+end

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/example/test_global_hook.lua
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/example/test_global_hook.lua b/plugins/experimental/ts_lua/example/test_global_hook.lua
index bf71f0c..fcdb56b 100644
--- a/plugins/experimental/ts_lua/example/test_global_hook.lua
+++ b/plugins/experimental/ts_lua/example/test_global_hook.lua
@@ -68,11 +68,11 @@ function do_global_cache_lookup_complete()
     return 0
 end
 
-function do_global_select_alt()
-    ts.debug('select_alt')
-
-    return 0
-end
+--function do_global_select_alt()
+--    ts.debug('select_alt')
+--
+--    return 0
+--end
 
 function do_global_read_cache()
     ts.debug('read_cache')

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/example/test_sleep.lua
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/example/test_sleep.lua b/plugins/experimental/ts_lua/example/test_sleep.lua
new file mode 100644
index 0000000..c3e778e
--- /dev/null
+++ b/plugins/experimental/ts_lua/example/test_sleep.lua
@@ -0,0 +1,51 @@
+--  Licensed to the Apache Software Foundation (ASF) under one
+--  or more contributor license agreements.  See the NOTICE file
+--  distributed with this work for additional information
+--  regarding copyright ownership.  The ASF licenses this file
+--  to you under the Apache License, Version 2.0 (the
+--  "License"); you may not use this file except in compliance
+--  with the License.  You may obtain a copy of the License at
+--
+--  http://www.apache.org/licenses/LICENSE-2.0
+--
+--  Unless required by applicable law or agreed to in writing, software
+--  distributed under the License is distributed on an "AS IS" BASIS,
+--  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+--  See the License for the specific language governing permissions and
+--  limitations under the License.
+
+
+function send_response()
+    ts.client_response.header['Rhost'] = ts.ctx['rhost']
+    ts.sleep(1)
+
+    return 0
+end
+
+function read_response()
+    local rs = ts.server_response.header['Server']
+    if rs ~= nil then
+        ts.server_response.header['Rserver'] = string.reverse(rs)
+    end
+
+    ts.sleep(1)
+
+    return 0
+end
+
+
+function do_remap()
+    local req_host = ts.client_request.header.Host
+
+    if req_host == nil then
+        return 0
+    end
+
+    ts.ctx['rhost'] = string.reverse(req_host)
+
+    ts.hook(TS_LUA_HOOK_READ_RESPONSE_HDR, read_response)
+    ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+
+    return 0
+end
+

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/example/test_transform.lua
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/example/test_transform.lua b/plugins/experimental/ts_lua/example/test_transform.lua
index 599ab16..14ba816 100644
--- a/plugins/experimental/ts_lua/example/test_transform.lua
+++ b/plugins/experimental/ts_lua/example/test_transform.lua
@@ -15,35 +15,31 @@
 --  limitations under the License.
 
 
-function upper_transform(data, eos)
-    if eos == 1 then
-        return string.upper(data)..'S.H.E.\n', eos
-    else
-        return string.upper(data), eos
-    end
-end
+local APPEND_DATA = 'TAIL\n'
 
-function send_response()
-    ts.client_response.header['SHE'] = ts.ctx['tb']['she']
-    return 0
-end
+function append_transform(data, eos)
+    if ts.ctx['len_set'] == nil then
+        local sz = ts.http.resp_transform.get_upstream_bytes()
 
+        if sz ~= TS_LUA_INT64_MAX then
+            ts.http.resp_transform.set_downstream_bytes(sz + string.len(APPEND_DATA))
+        end
 
-function do_remap()
-    local req_host = ts.client_request.header.Host
-
-    if req_host == nil then
-        return 0
+        ts.ctx['len_set'] = true
     end
 
-    ts.ctx['tb'] = {}
-    ts.ctx['tb']['she'] = 'wo ai yu ye hua'
+    if eos == 1 then
+        return data .. APPEND_DATA, eos
 
-    ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
-    ts.hook(TS_LUA_RESPONSE_TRANSFORM, upper_transform)
+    else
+        return data, eos
+    end
+end
 
+
+function do_remap()
+    ts.hook(TS_LUA_RESPONSE_TRANSFORM, append_transform)
     ts.http.resp_cache_transformed(0)
     ts.http.resp_cache_untransformed(1)
     return 0
 end
-

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua.c b/plugins/experimental/ts_lua/ts_lua.c
index e1676fb..4df5fc0 100644
--- a/plugins/experimental/ts_lua/ts_lua.c
+++ b/plugins/experimental/ts_lua/ts_lua.c
@@ -23,7 +23,7 @@
 
 #include "ts_lua_util.h"
 
-#define TS_LUA_MAX_STATE_COUNT 512
+#define TS_LUA_MAX_STATE_COUNT 256
 
 static uint64_t ts_lua_http_next_id = 0;
 static uint64_t ts_lua_g_http_next_id = 0;
@@ -124,17 +124,17 @@ TSRemapDoRemap(void *ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
   uint64_t req_id;
 
   TSCont contp;
-  lua_State *l;
+  lua_State *L;
 
   ts_lua_main_ctx *main_ctx;
   ts_lua_http_ctx *http_ctx;
+  ts_lua_cont_info *ci;
 
   ts_lua_instance_conf *instance_conf;
 
   instance_conf = (ts_lua_instance_conf *)ih;
   req_id = __sync_fetch_and_add(&ts_lua_http_next_id, 1);
 
-
   main_ctx = &ts_lua_main_ctx_array[req_id % TS_LUA_MAX_STATE_COUNT];
 
   TSMutexLock(main_ctx->mutexp);
@@ -148,24 +148,31 @@ TSRemapDoRemap(void *ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
   http_ctx->remap = 1;
   http_ctx->has_hook = 0;
 
+  ci = &http_ctx->cinfo;
+  L = ci->routine.lua;
+
   contp = TSContCreate(ts_lua_http_cont_handler, NULL);
   TSContDataSet(contp, http_ctx);
-  http_ctx->main_contp = contp;
 
-  l = http_ctx->lua;
+  ci->contp = contp;
+  ci->mutex = TSContMutexGet((TSCont)rh);
 
-  lua_getglobal(l, TS_LUA_FUNCTION_REMAP);
-  if (lua_type(l, -1) != LUA_TFUNCTION) {
+  lua_getglobal(L, TS_LUA_FUNCTION_REMAP);
+  if (lua_type(L, -1) != LUA_TFUNCTION) {
     TSMutexUnlock(main_ctx->mutexp);
     return TSREMAP_NO_REMAP;
   }
 
-  if (lua_pcall(l, 0, 1, 0) != 0) {
-    TSError("lua_pcall failed: %s", lua_tostring(l, -1));
+  ts_lua_set_cont_info(L, NULL);
+  if (lua_pcall(L, 0, 1, 0) != 0) {
+    ee("lua_pcall failed: %s", lua_tostring(L, -1));
+    ret = TSREMAP_NO_REMAP;
+
+  } else {
+    ret = lua_tointeger(L, -1);
   }
 
-  ret = lua_tointeger(l, -1);
-  lua_pop(l, 1);
+  lua_pop(L, 1);
 
   if (http_ctx->has_hook) {
     TSDebug(TS_LUA_DEBUG_TAG, "[%s] has txn hook -> adding txn close hook handler to release resources", __FUNCTION__);
@@ -173,7 +180,6 @@ TSRemapDoRemap(void *ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
   } else {
     TSDebug(TS_LUA_DEBUG_TAG, "[%s] no txn hook -> release resources now", __FUNCTION__);
     ts_lua_destroy_http_ctx(http_ctx);
-    TSContDestroy(contp);
   }
 
   TSMutexUnlock(main_ctx->mutexp);
@@ -198,6 +204,7 @@ globalHookHandler(TSCont contp, TSEvent event ATS_UNUSED, void *edata)
 
   ts_lua_main_ctx *main_ctx;
   ts_lua_http_ctx *http_ctx;
+  ts_lua_cont_info *ci;
 
   ts_lua_instance_conf *conf = (ts_lua_instance_conf *)TSContDataGet(contp);
 
@@ -225,15 +232,21 @@ globalHookHandler(TSCont contp, TSEvent event ATS_UNUSED, void *edata)
   }
 
   if (!http_ctx->client_request_hdrp) {
+    ts_lua_destroy_http_ctx(http_ctx);
+    TSMutexUnlock(main_ctx->mutexp);
+
     TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
     return 0;
   }
 
   txn_contp = TSContCreate(ts_lua_http_cont_handler, NULL);
   TSContDataSet(txn_contp, http_ctx);
-  http_ctx->main_contp = txn_contp;
 
-  l = http_ctx->lua;
+  ci = &http_ctx->cinfo;
+  ci->contp = txn_contp;
+  ci->mutex = TSContMutexGet((TSCont)txnp);
+
+  l = ci->routine.lua;
 
   switch (event) {
   case TS_EVENT_HTTP_READ_REQUEST_HDR:
@@ -285,17 +298,23 @@ globalHookHandler(TSCont contp, TSEvent event ATS_UNUSED, void *edata)
     break;
 
   default:
+    ts_lua_destroy_http_ctx(http_ctx);
+    TSMutexUnlock(main_ctx->mutexp);
     TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
     return 0;
-    break;
   }
 
   if (lua_type(l, -1) != LUA_TFUNCTION) {
-    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
     lua_pop(l, 1);
+    ts_lua_destroy_http_ctx(http_ctx);
+    TSMutexUnlock(main_ctx->mutexp);
+
+    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
     return 0;
   }
 
+  ts_lua_set_cont_info(l, NULL);
+
   if (lua_pcall(l, 0, 1, 0) != 0) {
     TSError("lua_pcall failed: %s", lua_tostring(l, -1));
   }
@@ -310,7 +329,6 @@ globalHookHandler(TSCont contp, TSEvent event ATS_UNUSED, void *edata)
   } else {
     TSDebug(TS_LUA_DEBUG_TAG, "[%s] no txn hook -> release resources now", __FUNCTION__);
     ts_lua_destroy_http_ctx(http_ctx);
-    TSContDestroy(txn_contp);
   }
 
   TSMutexUnlock(main_ctx->mutexp);
@@ -327,6 +345,16 @@ globalHookHandler(TSCont contp, TSEvent event ATS_UNUSED, void *edata)
 void
 TSPluginInit(int argc, const char *argv[])
 {
+  TSPluginRegistrationInfo info;
+
+  info.plugin_name = "ts_lua";
+  info.vendor_name = "Apache Software Foundation";
+  info.support_email = "dev@trafficserver.apache.org";
+
+  if (TSPluginRegister(TS_SDK_VERSION_3_0, &info) != TS_SUCCESS) {
+    TSError("Plugin registration failed. \n");
+  }
+
   int ret = 0;
   ts_lua_g_main_ctx_array = TSmalloc(sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
   memset(ts_lua_g_main_ctx_array, 0, sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
@@ -381,7 +409,7 @@ TSPluginInit(int argc, const char *argv[])
   // adding hook based on whether the lua global function exists.
   ts_lua_main_ctx *main_ctx = &ts_lua_g_main_ctx_array[0];
   ts_lua_http_ctx *http_ctx = ts_lua_create_http_ctx(main_ctx, conf);
-  lua_State *l = http_ctx->lua;
+  lua_State *l = http_ctx->cinfo.routine.lua;
 
   lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_REQUEST);
   if (lua_type(l, -1) == LUA_TFUNCTION) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_common.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_common.h b/plugins/experimental/ts_lua/ts_lua_common.h
index 2651c4c..bd1f0f2 100644
--- a/plugins/experimental/ts_lua/ts_lua_common.h
+++ b/plugins/experimental/ts_lua/ts_lua_common.h
@@ -32,6 +32,7 @@
 #include <ts/experimental.h>
 #include <ts/remap.h>
 #include "ink_defs.h"
+#include "ts_lua_coroutine.h"
 
 #define TS_LUA_FUNCTION_REMAP "do_remap"
 #define TS_LUA_FUNCTION_CACHE_LOOKUP_COMPLETE "do_cache_lookup_complete"
@@ -63,6 +64,8 @@
 
 #define TS_LUA_DEBUG_TAG "ts_lua"
 
+#define TS_LUA_EVENT_COROUTINE_CONT 20000
+
 #define TS_LUA_MAX_SCRIPT_FNAME_LENGTH 1024
 #define TS_LUA_MAX_CONFIG_VARS_COUNT 256
 #define TS_LUA_MAX_SHARED_DICT_NAME_LENGTH 128
@@ -81,6 +84,11 @@
     X, #X                       \
   }
 
+#define ee(...)                              \
+  fprintf(stderr, "Lua *** %s: ", __func__); \
+  fprintf(stderr, __VA_ARGS__);              \
+  fprintf(stderr, " @%s:%d\n", __FILE__, __LINE__)
+
 /* for http config or cntl var */
 typedef struct {
   int nvar;
@@ -97,19 +105,11 @@ typedef struct {
 } ts_lua_instance_conf;
 
 
-/* global lua state struct */
-typedef struct {
-  lua_State *lua;
-  TSMutex mutexp;
-  int gref;
-} ts_lua_main_ctx;
-
 /* lua state for http request */
 typedef struct {
-  lua_State *lua;
-  TSHttpTxn txnp;
-  TSCont main_contp;
+  ts_lua_cont_info cinfo;
 
+  TSHttpTxn txnp;
   TSMBuffer client_request_bufp;
   TSMLoc client_request_hdrp;
   TSMLoc client_request_url;
@@ -118,7 +118,6 @@ typedef struct {
   TSMLoc server_request_hdrp;
   TSMLoc server_request_url;
 
-
   TSMBuffer server_response_bufp;
   TSMLoc server_response_hdrp;
 
@@ -128,13 +127,8 @@ typedef struct {
   TSMBuffer cached_response_bufp;
   TSMLoc cached_response_hdrp;
 
-  ts_lua_main_ctx *mctx;
-
   ts_lua_instance_conf *instance_conf;
 
-  int intercept_type;
-  int ref;
-
   int remap;
   int has_hook;
 
@@ -148,46 +142,29 @@ typedef struct {
 } ts_lua_io_handle;
 
 typedef struct {
-  TSVIO output_vio;
-  TSIOBuffer output_buffer;
-  TSIOBufferReader output_reader;
-
-  int64_t total;
-  ts_lua_http_ctx *hctx;
-  int eos;
+  ts_lua_cont_info cinfo;
 
-} ts_lua_transform_ctx;
+  ts_lua_io_handle output;
+  ts_lua_io_handle reserved;
 
-/* for intercept */
-struct ict_item;
-struct ict_ctx;
-typedef int (*ict_clean)(struct ict_item *item);
+  ts_lua_http_ctx *hctx;
+  int64_t upstream_bytes;
+  int64_t downstream_bytes;
+  int64_t total;
 
-typedef struct ict_item {
-  struct ict_item *next;
-  struct ict_ctx *ictx;
+} ts_lua_http_transform_ctx;
 
-  TSCont contp;
-  ict_clean cleanup;
-  void *data;
-  int deleted : 1;
-} ts_lua_http_intercept_item;
+typedef struct {
+  ts_lua_cont_info cinfo;
 
-typedef struct ict_ctx {
-  lua_State *lua;
-  TSCont contp;
   ts_lua_io_handle input;
   ts_lua_io_handle output;
-  TSVConn net_vc;
 
-  ts_lua_main_ctx *mctx;
+  TSVConn net_vc;
   ts_lua_http_ctx *hctx;
 
-  struct ict_item *ict_chain;
-
   int64_t to_flush;
-  int ref;
-
+  int reuse : 1;
   int recv_complete : 1;
   int send_complete : 1;
   int all_ready : 1;
@@ -205,15 +182,4 @@ typedef struct ict_ctx {
     }                                   \
   } while (0)
 
-#define TS_LUA_ADD_INTERCEPT_ITEM(ictx, item, contp, func, d) \
-  {                                                           \
-    item->cleanup = func;                                     \
-    item->data = d;                                           \
-    item->ictx = ictx;                                        \
-    item->contp = contp;                                      \
-    item->deleted = 0;                                        \
-    item->next = ictx->ict_chain;                             \
-    ictx->ict_chain = item;                                   \
-  }
-
 #endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_constant.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_constant.c b/plugins/experimental/ts_lua/ts_lua_constant.c
new file mode 100644
index 0000000..8a981ea
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_constant.c
@@ -0,0 +1,41 @@
+/*
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+*/
+
+
+#include <stdint.h>
+#include <stdio.h>
+#include <lua.h>
+
+static void ts_lua_inject_number_variables(lua_State *L);
+
+
+void
+ts_lua_inject_constant_api(lua_State *L)
+{
+  ts_lua_inject_number_variables(L);
+}
+
+static void
+ts_lua_inject_number_variables(lua_State *L)
+{
+  lua_pushinteger(L, INT64_MAX);
+  lua_setglobal(L, "TS_LUA_INT64_MAX");
+
+  lua_pushinteger(L, INT64_MIN);
+  lua_setglobal(L, "TS_LUA_INT64_MIN");
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_constant.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_constant.h b/plugins/experimental/ts_lua/ts_lua_constant.h
new file mode 100644
index 0000000..a345f84
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_constant.h
@@ -0,0 +1,25 @@
+/*
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+*/
+
+
+#ifndef _TS_LUA_CONSTANT_H
+#define _TS_LUA_CONSTANT_H
+
+void ts_lua_inject_constant_api(lua_State *L);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_coroutine.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_coroutine.c b/plugins/experimental/ts_lua/ts_lua_coroutine.c
new file mode 100644
index 0000000..dbe4187
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_coroutine.c
@@ -0,0 +1,98 @@
+/*
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+*/
+
+#include "ts_lua_coroutine.h"
+
+static inline void ts_lua_async_push_item(ts_lua_async_item **head, ts_lua_async_item *node);
+static inline void ts_lua_async_destroy_item(ts_lua_async_item *node);
+static inline void ts_lua_async_destroy_chain(ts_lua_async_item **head);
+
+
+inline ts_lua_async_item *
+ts_lua_async_create_item(TSCont cont, async_clean func, void *d, ts_lua_cont_info *ci)
+{
+  ts_lua_async_item *ai;
+
+  ai = (ts_lua_async_item *)TSmalloc(sizeof(ts_lua_async_item));
+  if (ai == NULL)
+    return NULL;
+
+  ai->cinfo = ci;
+
+  ai->cleanup = func;
+  ai->data = d;
+  ai->contp = cont;
+  ai->deleted = 0;
+
+  ts_lua_async_push_item(&ci->async_chain, ai);
+
+  return ai;
+}
+
+static inline void
+ts_lua_async_push_item(ts_lua_async_item **head, ts_lua_async_item *node)
+{
+  node->next = *head;
+  *head = node;
+}
+
+inline void
+ts_lua_async_destroy_item(ts_lua_async_item *node)
+{
+  if (node->cleanup && !node->deleted) {
+    node->cleanup(node);
+  }
+
+  TSfree(node);
+}
+
+inline void
+ts_lua_async_destroy_chain(ts_lua_async_item **head)
+{
+  ts_lua_async_item *node, *next;
+
+  node = *head;
+
+  while (node) {
+    next = node->next;
+    ts_lua_async_destroy_item(node);
+    node = next;
+  }
+}
+
+inline void
+ts_lua_release_cont_info(ts_lua_cont_info *ci)
+{
+  ts_lua_main_ctx *mctx;
+  ts_lua_coroutine *crt;
+
+  crt = &ci->routine;
+  mctx = crt->mctx;
+
+  ts_lua_async_destroy_chain(&ci->async_chain);
+
+  if (ci->contp) {
+    TSContDestroy(ci->contp);
+  }
+
+  if (crt->lua) {
+    TSMutexLock(mctx->mutexp);
+    luaL_unref(crt->lua, LUA_REGISTRYINDEX, crt->ref);
+    TSMutexUnlock(mctx->mutexp);
+  }
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_coroutine.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_coroutine.h b/plugins/experimental/ts_lua/ts_lua_coroutine.h
new file mode 100644
index 0000000..78a657c
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_coroutine.h
@@ -0,0 +1,71 @@
+/*
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+*/
+
+#ifndef _TS_LUA_COROUTINE_H
+#define _TS_LUA_COROUTINE_H
+
+#include <stdio.h>
+#include <lua.h>
+#include <lualib.h>
+#include <lauxlib.h>
+#include <ts/ts.h>
+
+
+struct async_item;
+typedef int (*async_clean)(struct async_item *item);
+
+/* main context*/
+typedef struct {
+  lua_State *lua; // basic lua vm, injected
+  TSMutex mutexp; // mutex for lua vm
+  int gref;       // reference for lua vm self, in reg table
+} ts_lua_main_ctx;
+
+/* coroutine */
+typedef struct {
+  ts_lua_main_ctx *mctx;
+  lua_State *lua; // derived lua_thread
+  int ref;        // reference for lua_thread, in REG Table
+} ts_lua_coroutine;
+
+/* continuation info */
+typedef struct {
+  ts_lua_coroutine routine;
+  TSCont contp;                   // continuation for the routine
+  TSMutex mutex;                  // mutex for continuation
+  struct async_item *async_chain; // async_item list
+} ts_lua_cont_info;
+
+
+/* asynchronous item */
+typedef struct async_item {
+  struct async_item *next;
+  ts_lua_cont_info *cinfo;
+
+  TSCont contp; // continuation for the async operation
+  void *data;   // private data
+
+  async_clean cleanup; // cleanup function
+  int deleted : 1;
+} ts_lua_async_item;
+
+
+ts_lua_async_item *ts_lua_async_create_item(TSCont cont, async_clean func, void *d, ts_lua_cont_info *ci);
+void ts_lua_release_cont_info(ts_lua_cont_info *ci);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_fetch.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_fetch.c b/plugins/experimental/ts_lua/ts_lua_fetch.c
new file mode 100644
index 0000000..39cf66a
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_fetch.c
@@ -0,0 +1,603 @@
+/*
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+*/
+
+#include <sys/socket.h>
+#include <netinet/in.h>
+#include <arpa/inet.h>
+
+#include "ts_lua_util.h"
+#include "ts_lua_io.h"
+#include "ts_lua_fetch.h"
+
+#define TS_LUA_EVENT_FETCH_OVER 20010
+#define TS_LUA_FETCH_CLIENT_ADDRESS "127.0.0.1"
+#define TS_LUA_FETCH_CLIENT_PORT 33333
+#define TS_LUA_FETCH_USER_AGENT "TS Fetcher/1.0"
+
+static int ts_lua_fetch(lua_State *L);
+static int ts_lua_fetch_multi(lua_State *L);
+static int ts_lua_fetch_handler(TSCont contp, TSEvent event, void *edata);
+static int ts_lua_fetch_multi_cleanup(ts_lua_async_item *ai);
+static int ts_lua_fetch_multi_handler(TSCont contp, TSEvent event, void *edata);
+static int ts_lua_fetch_one_item(lua_State *L, const char *url, size_t url_len, ts_lua_fetch_info *fi);
+static inline void ts_lua_destroy_fetch_multi_info(ts_lua_fetch_multi_info *fmi);
+
+
+void
+ts_lua_inject_fetch_api(lua_State *L)
+{
+  /* ts.fetch() */
+  lua_pushcfunction(L, ts_lua_fetch);
+  lua_setfield(L, -2, "fetch");
+
+  /* ts.fetch_multi() */
+  lua_pushcfunction(L, ts_lua_fetch_multi);
+  lua_setfield(L, -2, "fetch_multi");
+}
+
+static int
+ts_lua_fetch(lua_State *L)
+{
+  int sz;
+  size_t n;
+  const char *url;
+  size_t url_len;
+  TSCont contp;
+  ts_lua_cont_info *ci;
+  ts_lua_async_item *ai;
+  ts_lua_fetch_info *fi;
+  ts_lua_fetch_multi_info *fmi;
+
+  ci = ts_lua_get_cont_info(L);
+  if (ci == NULL)
+    return 0;
+
+  n = lua_gettop(L);
+  if (n < 1) {
+    return luaL_error(L, "'ts.fetch' requires parameter");
+  }
+
+  /* url */
+  if (!lua_isstring(L, 1)) {
+    return luaL_error(L, "'ts.fetch' first param is not string");
+  }
+
+  url = luaL_checklstring(L, 1, &url_len);
+
+  /* replicate misc table */
+  if (n >= 2) {
+    lua_pushvalue(L, 2);
+
+  } else {
+    lua_pushnil(L);
+  }
+
+  contp = TSContCreate(ts_lua_fetch_multi_handler, ci->mutex);
+
+  sz = sizeof(ts_lua_fetch_multi_info) + 1 * sizeof(ts_lua_fetch_info);
+  fmi = (ts_lua_fetch_multi_info *)TSmalloc(sz);
+
+  memset(fmi, 0, sz);
+  fmi->total = 1;
+  fmi->contp = contp;
+
+  fi = &fmi->fiv[0];
+  fi->fmi = fmi;
+  fi->buffer = TSIOBufferCreate();
+  fi->reader = TSIOBufferReaderAlloc(fi->buffer);
+
+  ts_lua_fetch_one_item(L, url, url_len, fi);
+
+  // pop the replicated misc table
+  lua_pop(L, 1);
+
+  ai = ts_lua_async_create_item(contp, ts_lua_fetch_multi_cleanup, fmi, ci);
+  TSContDataSet(contp, ai);
+
+  return lua_yield(L, 0);
+  ;
+}
+
+static int
+ts_lua_fetch_multi(lua_State *L)
+{
+  int type, sz;
+  size_t i, n;
+  const char *url;
+  size_t url_len;
+  TSCont contp;
+  ts_lua_cont_info *ci;
+  ts_lua_async_item *ai;
+  ts_lua_fetch_info *fi;
+  ts_lua_fetch_multi_info *fmi;
+
+  ci = ts_lua_get_cont_info(L);
+  if (ci == NULL)
+    return 0;
+
+  if (lua_gettop(L) < 1) {
+    return luaL_error(L, "'ts.fetch_mutli' requires one parameter");
+  }
+
+  type = lua_type(L, 1);
+  if (type != LUA_TTABLE) {
+    return luaL_error(L, "'ts.fetch_mutli' requires table as parameter");
+  }
+
+  // main continuation handler
+  contp = TSContCreate(ts_lua_fetch_multi_handler, ci->mutex);
+
+  // Iterate the table
+  n = lua_objlen(L, 1);
+
+  sz = sizeof(ts_lua_fetch_multi_info) + n * sizeof(ts_lua_fetch_info);
+  fmi = (ts_lua_fetch_multi_info *)TSmalloc(sz);
+
+  memset(fmi, 0, sz);
+  fmi->total = n;
+  fmi->contp = contp;
+  fmi->multi = 1;
+
+  for (i = 0; i < n; i++) {
+    /* push fetch item */
+    lua_pushinteger(L, i + 1);
+    lua_gettable(L, -2);
+
+    if (lua_objlen(L, -1) < 1) {
+      ts_lua_destroy_fetch_multi_info(fmi);
+      TSContDestroy(contp);
+
+      return luaL_error(L, "'ts.fetch_mutli' got empty table item");
+    }
+
+    /* push url */
+    lua_pushnumber(L, 1);
+    lua_gettable(L, -2);
+
+    if (!lua_isstring(L, -1)) {
+      ts_lua_destroy_fetch_multi_info(fmi);
+      TSContDestroy(contp);
+
+      return luaL_error(L, "'ts.fetch_mutli' got invalid table item: url illegal");
+    }
+
+    url = luaL_checklstring(L, -1, &url_len);
+
+    /* push misc table */
+    lua_pushinteger(L, 2);
+    lua_gettable(L, -3);
+
+    fi = &fmi->fiv[i];
+    fi->fmi = fmi;
+    fi->buffer = TSIOBufferCreate();
+    fi->reader = TSIOBufferReaderAlloc(fi->buffer);
+
+    ts_lua_fetch_one_item(L, url, url_len, fi);
+    lua_pop(L, 3); // misc table, url, fetch item
+  }
+
+  ai = ts_lua_async_create_item(contp, ts_lua_fetch_multi_cleanup, (void *)fmi, ci);
+  TSContDataSet(contp, ai);
+
+  return lua_yield(L, 0);
+  ;
+}
+
+static int
+ts_lua_fetch_one_item(lua_State *L, const char *url, size_t url_len, ts_lua_fetch_info *fi)
+{
+  TSCont contp;
+  int tb, flags, host_len, rc, port, n;
+  int cl, ht, ua;
+  const char *method, *key, *value, *body, *opt;
+  const char *addr, *ptr, *host;
+  size_t method_len, key_len, value_len, body_len;
+  size_t addr_len, opt_len, i, left;
+  char c;
+  struct sockaddr_in clientaddr;
+  char ipstr[32];
+  char buf[32];
+
+  tb = lua_istable(L, -1);
+
+  /* method */
+  if (tb) {
+    lua_pushlstring(L, "method", sizeof("method") - 1);
+    lua_gettable(L, -2);
+    if (lua_isstring(L, -1)) {
+      method = luaL_checklstring(L, -1, &method_len);
+
+    } else {
+      method = "GET";
+      method_len = sizeof("GET") - 1;
+    }
+
+    lua_pop(L, 1);
+
+  } else {
+    method = "GET";
+    method_len = sizeof("GET") - 1;
+  }
+
+  /* body */
+  body = NULL;
+  body_len = 0;
+
+  if (tb) {
+    lua_pushlstring(L, "body", sizeof("body") - 1);
+    lua_gettable(L, -2);
+
+    if (lua_isstring(L, -1)) {
+      body = luaL_checklstring(L, -1, &body_len);
+    }
+
+    lua_pop(L, 1);
+  }
+
+  /* cliaddr */
+  memset(&clientaddr, 0, sizeof(clientaddr));
+  clientaddr.sin_family = AF_INET;
+  rc = 0;
+
+  if (tb) {
+    lua_pushlstring(L, "cliaddr", sizeof("cliaddr") - 1);
+    lua_gettable(L, -2);
+
+    if (lua_isstring(L, -1)) {
+      addr = luaL_checklstring(L, -1, &addr_len);
+      rc = sscanf(addr, "%15s:%d", ipstr, &port);
+      if (rc == 2) {
+        clientaddr.sin_port = htons(port);
+        inet_aton(ipstr, (struct in_addr *)&clientaddr.sin_addr.s_addr);
+      }
+    }
+
+    lua_pop(L, 1);
+  }
+
+  if (rc != 2) {
+    clientaddr.sin_port = htons(TS_LUA_FETCH_CLIENT_PORT);
+    inet_aton(TS_LUA_FETCH_CLIENT_ADDRESS, (struct in_addr *)&clientaddr.sin_addr.s_addr);
+  }
+
+  /* option */
+  flags = TS_FETCH_FLAGS_DECHUNK; // dechunk the body default
+
+  if (tb) {
+    lua_pushlstring(L, "option", sizeof("option") - 1);
+    lua_gettable(L, -2);
+
+    if (lua_isstring(L, -1)) {
+      opt = luaL_checklstring(L, -1, &opt_len);
+
+      for (i = 0; i < opt_len; i++) {
+        c = opt[i];
+
+        switch (c) {
+        case 'c':
+          flags &= (~TS_FETCH_FLAGS_DECHUNK);
+          break;
+
+        default:
+          break;
+        }
+      }
+    }
+
+    lua_pop(L, 1);
+  }
+
+  contp = TSContCreate(ts_lua_fetch_handler, TSContMutexGet(fi->fmi->contp)); // reuse parent cont's mutex
+  TSContDataSet(contp, fi);
+
+  fi->contp = contp;
+  fi->fch = TSFetchCreate(contp, method, url, "HTTP/1.1", (struct sockaddr *)&clientaddr, flags);
+
+  /* header */
+  cl = ht = ua = 0;
+
+  if (tb) {
+    lua_pushlstring(L, "header", sizeof("header") - 1);
+    lua_gettable(L, -2);
+
+    if (lua_istable(L, -1)) {
+      // iterate the header table
+      lua_pushnil(L);
+
+      while (lua_next(L, -2)) {
+        lua_pushvalue(L, -2);
+
+        key = luaL_checklstring(L, -1, &key_len);
+        value = luaL_checklstring(L, -2, &value_len);
+
+        if (key_len == TS_MIME_LEN_CONTENT_LENGTH && !strncasecmp(TS_MIME_FIELD_CONTENT_LENGTH, key, key_len)) { // Content-Length
+          cl = 1;
+
+        } else if (key_len == TS_MIME_LEN_HOST && !strncasecmp(TS_MIME_FIELD_HOST, key, key_len)) { // Host
+          ht = 1;
+
+        } else if (key_len == TS_MIME_LEN_USER_AGENT && !strncasecmp(TS_MIME_FIELD_USER_AGENT, key, key_len)) { // User-Agent
+          ua = 1;
+        }
+
+        TSFetchHeaderAdd(fi->fch, key, key_len, value, value_len);
+
+        lua_pop(L, 2);
+      }
+    }
+
+    lua_pop(L, 1);
+  }
+
+  /* Host */
+  if (ht == 0) {
+    ptr = memchr(url, ':', url_len);
+
+    if (ptr) {
+      host = ptr + 3;
+      left = url_len - (host - url);
+
+      ptr = memchr(host, '/', left);
+
+      if (ptr) {
+        host_len = ptr - host;
+
+      } else {
+        host_len = left;
+      }
+
+      TSFetchHeaderAdd(fi->fch, TS_MIME_FIELD_HOST, TS_MIME_LEN_HOST, host, host_len);
+    }
+  }
+
+  /* User-Agent */
+  if (ua == 0) {
+    TSFetchHeaderAdd(fi->fch, TS_MIME_FIELD_USER_AGENT, TS_MIME_LEN_USER_AGENT, TS_LUA_FETCH_USER_AGENT,
+                     sizeof(TS_LUA_FETCH_USER_AGENT) - 1);
+  }
+
+  if (body_len > 0 && cl == 0) { // add Content-Length header
+    n = sprintf(buf, "%zu", body_len);
+    TSFetchHeaderAdd(fi->fch, TS_MIME_FIELD_CONTENT_LENGTH, TS_MIME_LEN_CONTENT_LENGTH, buf, n);
+  }
+
+  TSFetchLaunch(fi->fch);
+
+  if (body_len > 0) {
+    TSFetchWriteData(fi->fch, body, body_len);
+  }
+
+  return 0;
+}
+
+static int
+ts_lua_fetch_handler(TSCont contp, TSEvent ev, void *edata ATS_UNUSED)
+{
+  int event;
+  char *from;
+  int64_t n, wavail;
+  TSIOBufferBlock blk;
+
+  ts_lua_fetch_info *fi;
+  ts_lua_fetch_multi_info *fmi;
+
+  event = (int)ev;
+  fi = TSContDataGet(contp);
+  fmi = fi->fmi;
+
+  switch (event) {
+  case TS_FETCH_EVENT_EXT_HEAD_READY:
+  case TS_FETCH_EVENT_EXT_HEAD_DONE:
+    break;
+
+  case TS_FETCH_EVENT_EXT_BODY_READY:
+  case TS_FETCH_EVENT_EXT_BODY_DONE:
+
+    do {
+      blk = TSIOBufferStart(fi->buffer);
+      from = TSIOBufferBlockWriteStart(blk, &wavail);
+      n = TSFetchReadData(fi->fch, from, wavail);
+      TSIOBufferProduce(fi->buffer, n);
+    } while (n == wavail);
+
+    if (event == TS_FETCH_EVENT_EXT_BODY_DONE) { // fetch over
+      fi->over = 1;
+    }
+
+    break;
+
+  default:
+    fi->failed = 1;
+    break;
+  }
+
+  if (fi->over || fi->failed) {
+    TSContCall(fmi->contp, TS_LUA_EVENT_FETCH_OVER, fi); // error exist
+  }
+
+  return 0;
+}
+
+static int
+ts_lua_fill_one_result(lua_State *L, ts_lua_fetch_info *fi)
+{
+  const char *name, *value;
+  int name_len, value_len;
+  char *dst;
+  int64_t ravail;
+  TSMBuffer bufp;
+  TSMLoc hdrp;
+  TSMLoc field_loc, next_field_loc;
+  TSHttpStatus status;
+
+  bufp = TSFetchRespHdrMBufGet(fi->fch);
+  hdrp = TSFetchRespHdrMLocGet(fi->fch);
+
+  // result table
+  lua_newtable(L);
+
+  // status code
+  status = TSHttpHdrStatusGet(bufp, hdrp);
+  lua_pushlstring(L, "status", sizeof("status") - 1);
+  lua_pushnumber(L, status);
+  lua_rawset(L, -3);
+
+  // header
+  lua_pushlstring(L, "header", sizeof("header") - 1);
+  lua_newtable(L);
+
+  field_loc = TSMimeHdrFieldGet(bufp, hdrp, 0);
+  while (field_loc) {
+    name = TSMimeHdrFieldNameGet(bufp, hdrp, field_loc, &name_len);
+    value = TSMimeHdrFieldValueStringGet(bufp, hdrp, field_loc, -1, &value_len);
+
+    lua_pushlstring(L, name, name_len);
+    lua_pushlstring(L, value, value_len);
+    lua_rawset(L, -3);
+
+    next_field_loc = TSMimeHdrFieldNext(bufp, hdrp, field_loc);
+    TSHandleMLocRelease(bufp, hdrp, field_loc);
+    field_loc = next_field_loc;
+  }
+  lua_rawset(L, -3);
+
+  // body
+  ravail = TSIOBufferReaderAvail(fi->reader);
+  if (ravail > 0) {
+    lua_pushlstring(L, "body", sizeof("body") - 1);
+
+    dst = (char *)TSmalloc(ravail);
+    IOBufferReaderCopy(fi->reader, dst, ravail);
+    lua_pushlstring(L, (char *)dst, ravail);
+
+    lua_rawset(L, -3);
+    TSfree(dst);
+  }
+
+  // truncked
+  lua_pushlstring(L, "truncked", sizeof("truncked") - 1);
+  if (fi->failed) {
+    lua_pushboolean(L, 1);
+
+  } else {
+    lua_pushboolean(L, 0);
+  }
+
+  lua_rawset(L, -3);
+
+  return 0;
+}
+
+static int
+ts_lua_fetch_multi_handler(TSCont contp, TSEvent event ATS_UNUSED, void *edata)
+{
+  int i;
+  lua_State *L;
+  TSMutex lmutex;
+
+  ts_lua_async_item *ai;
+  ts_lua_cont_info *ci;
+  ts_lua_fetch_info *fi;
+  ts_lua_fetch_multi_info *fmi;
+
+  ai = TSContDataGet(contp);
+  ci = ai->cinfo;
+
+  fmi = (ts_lua_fetch_multi_info *)ai->data;
+  fi = (ts_lua_fetch_info *)edata;
+
+  L = ai->cinfo->routine.lua;
+  lmutex = ai->cinfo->routine.mctx->mutexp;
+
+  fmi->done++;
+
+  if (fmi->done != fmi->total)
+    return 0;
+
+  // all finish
+  TSMutexLock(lmutex);
+
+  if (fmi->total == 1 && !fmi->multi) {
+    ts_lua_fill_one_result(L, fi);
+    TSContCall(ci->contp, TS_LUA_EVENT_COROUTINE_CONT, (void *)1);
+
+  } else {
+    lua_newtable(L);
+
+    for (i = 1; i <= fmi->total; i++) {
+      ts_lua_fill_one_result(L, &fmi->fiv[i - 1]);
+      lua_rawseti(L, -2, i);
+    }
+
+    TSContCall(ci->contp, TS_LUA_EVENT_COROUTINE_CONT, (void *)1);
+  }
+
+  TSMutexUnlock(lmutex);
+  return 0;
+}
+
+
+static inline void
+ts_lua_destroy_fetch_multi_info(ts_lua_fetch_multi_info *fmi)
+{
+  int i;
+  ts_lua_fetch_info *fi;
+
+  if (fmi == NULL)
+    return;
+
+  for (i = 0; i < fmi->total; i++) {
+    fi = &fmi->fiv[i];
+
+    if (fi->reader) {
+      TSIOBufferReaderFree(fi->reader);
+    }
+
+    if (fi->buffer) {
+      TSIOBufferDestroy(fi->buffer);
+    }
+
+    if (fi->fch) {
+      TSFetchDestroy(fi->fch);
+    }
+
+    if (fi->contp) {
+      TSContDestroy(fi->contp);
+    }
+  }
+
+  TSfree(fmi);
+}
+
+static int
+ts_lua_fetch_multi_cleanup(ts_lua_async_item *ai)
+{
+  ts_lua_fetch_multi_info *fmi;
+
+  if (ai->data) {
+    fmi = (ts_lua_fetch_multi_info *)ai->data;
+    ts_lua_destroy_fetch_multi_info(fmi);
+
+    ai->data = NULL;
+  }
+
+  TSContDestroy(ai->contp);
+  ai->deleted = 1;
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_fetch.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_fetch.h b/plugins/experimental/ts_lua/ts_lua_fetch.h
new file mode 100644
index 0000000..4c6394a
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_fetch.h
@@ -0,0 +1,48 @@
+/*
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+*/
+
+#ifndef _TS_LUA_FETCH_H
+#define _TS_LUA_FETCH_H
+
+#include "ts_lua_common.h"
+
+struct fetch_multi_info;
+
+typedef struct {
+  TSCont contp;
+  struct fetch_multi_info *fmi;
+
+  TSIOBuffer buffer;
+  TSIOBufferReader reader;
+  TSFetchSM fch;
+
+  int over : 1;
+  int failed : 1;
+} ts_lua_fetch_info;
+
+typedef struct fetch_multi_info {
+  TSCont contp; // should be destroyed only in cleanup
+  int multi;    // invoke from ts.fetch_multi
+  int total;
+  int done;
+  ts_lua_fetch_info fiv[0];
+} ts_lua_fetch_multi_info;
+
+void ts_lua_inject_fetch_api(lua_State *L);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_hook.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_hook.c b/plugins/experimental/ts_lua/ts_lua_hook.c
index 0a9ece3..30c3262 100644
--- a/plugins/experimental/ts_lua/ts_lua_hook.c
+++ b/plugins/experimental/ts_lua/ts_lua_hook.c
@@ -80,10 +80,11 @@ ts_lua_add_hook(lua_State *L)
   int entry;
 
   TSVConn connp;
+  TSCont main_contp;
   ts_lua_http_ctx *http_ctx;
-  ts_lua_transform_ctx *transform_ctx;
 
   http_ctx = ts_lua_get_http_ctx(L);
+  main_contp = http_ctx->cinfo.contp;
 
   entry = lua_tointeger(L, 1); // get hook id
 
@@ -94,7 +95,7 @@ ts_lua_add_hook(lua_State *L)
   switch (entry) {
   case TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_CACHE_LOOKUP_COMPLETE);
@@ -106,7 +107,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_SEND_REQUEST_HDR:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_SEND_REQUEST_HDR_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_SEND_REQUEST_HDR_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_SEND_REQUEST);
@@ -118,7 +119,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_READ_RESPONSE_HDR:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_READ_RESPONSE_HDR_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_READ_RESPONSE_HDR_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_READ_RESPONSE);
@@ -130,7 +131,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_SEND_RESPONSE_HDR:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_SEND_RESPONSE_HDR_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_SEND_RESPONSE_HDR_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_SEND_RESPONSE);
@@ -142,7 +143,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_READ_REQUEST_HDR:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_READ_REQUEST_HDR_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_READ_REQUEST_HDR_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_READ_REQUEST);
@@ -154,7 +155,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_TXN_START:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_TXN_START_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_TXN_START_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_TXN_START);
@@ -166,7 +167,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_PRE_REMAP:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_PRE_REMAP_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_PRE_REMAP_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_PRE_REMAP);
@@ -178,7 +179,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_POST_REMAP:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_POST_REMAP_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_POST_REMAP_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_POST_REMAP);
@@ -190,7 +191,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_OS_DNS:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_OS_DNS_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_OS_DNS_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_OS_DNS);
@@ -202,7 +203,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_SELECT_ALT:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_SELECT_ALT_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_SELECT_ALT_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_SELECT_ALT);
@@ -214,7 +215,7 @@ ts_lua_add_hook(lua_State *L)
 
   case TS_LUA_HOOK_READ_CACHE_HDR:
     if (http_ctx) {
-      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_READ_CACHE_HDR_HOOK, http_ctx->main_contp);
+      TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_READ_CACHE_HDR_HOOK, main_contp);
       http_ctx->has_hook = 1;
       lua_pushvalue(L, 2);
       lua_setglobal(L, TS_LUA_FUNCTION_READ_CACHE);
@@ -238,23 +239,14 @@ ts_lua_add_hook(lua_State *L)
   case TS_LUA_REQUEST_TRANSFORM:
   case TS_LUA_RESPONSE_TRANSFORM:
     if (http_ctx) {
-      http_ctx->has_hook = 1;
-      transform_ctx = (ts_lua_transform_ctx *)TSmalloc(sizeof(ts_lua_transform_ctx));
-      memset(transform_ctx, 0, sizeof(ts_lua_transform_ctx));
-      transform_ctx->hctx = http_ctx;
-
       connp = TSTransformCreate(ts_lua_transform_entry, http_ctx->txnp);
-      TSContDataSet(connp, transform_ctx);
+      ts_lua_create_http_transform_ctx(http_ctx, connp);
 
       if (entry == TS_LUA_REQUEST_TRANSFORM) {
         TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_REQUEST_TRANSFORM_HOOK, connp);
       } else {
         TSHttpTxnHookAdd(http_ctx->txnp, TS_HTTP_RESPONSE_TRANSFORM_HOOK, connp);
       }
-
-      lua_pushlightuserdata(L, transform_ctx);
-      lua_pushvalue(L, 2);
-      lua_rawset(L, LUA_GLOBALSINDEX);
     }
     break;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_http.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_http.c b/plugins/experimental/ts_lua/ts_lua_http.c
index e285af4..618deff 100644
--- a/plugins/experimental/ts_lua/ts_lua_http.c
+++ b/plugins/experimental/ts_lua/ts_lua_http.c
@@ -54,6 +54,10 @@ static int ts_lua_http_resp_cache_untransformed(lua_State *L);
 static int ts_lua_http_is_internal_request(lua_State *L);
 static int ts_lua_http_skip_remapping_set(lua_State *L);
 
+static void ts_lua_inject_http_resp_transform_api(lua_State *L);
+static int ts_lua_http_resp_transform_get_upstream_bytes(lua_State *L);
+static int ts_lua_http_resp_transform_set_downstream_bytes(lua_State *L);
+
 void
 ts_lua_inject_http_api(lua_State *L)
 {
@@ -107,6 +111,21 @@ ts_lua_inject_http_transform_api(lua_State *L)
 
   lua_pushcfunction(L, ts_lua_http_resp_cache_untransformed);
   lua_setfield(L, -2, "resp_cache_untransformed");
+
+  /*  ts.http.resp_transform api */
+  lua_newtable(L);
+  ts_lua_inject_http_resp_transform_api(L);
+  lua_setfield(L, -2, "resp_transform");
+}
+
+static void
+ts_lua_inject_http_resp_transform_api(lua_State *L)
+{
+  lua_pushcfunction(L, ts_lua_http_resp_transform_get_upstream_bytes);
+  lua_setfield(L, -2, "get_upstream_bytes");
+
+  lua_pushcfunction(L, ts_lua_http_resp_transform_set_downstream_bytes);
+  lua_setfield(L, -2, "set_downstream_bytes");
 }
 
 static void
@@ -297,3 +316,30 @@ ts_lua_http_skip_remapping_set(lua_State *L)
 
   return 0;
 }
+
+static int
+ts_lua_http_resp_transform_get_upstream_bytes(lua_State *L)
+{
+  ts_lua_http_transform_ctx *transform_ctx;
+
+  transform_ctx = ts_lua_get_http_transform_ctx(L);
+
+  lua_pushnumber(L, transform_ctx->upstream_bytes);
+
+  return 1;
+}
+
+static int
+ts_lua_http_resp_transform_set_downstream_bytes(lua_State *L)
+{
+  int64_t n;
+  ts_lua_http_transform_ctx *transform_ctx;
+
+  transform_ctx = ts_lua_get_http_transform_ctx(L);
+
+  n = luaL_checkinteger(L, 1);
+
+  transform_ctx->downstream_bytes = n;
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_http_intercept.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_http_intercept.c b/plugins/experimental/ts_lua/ts_lua_http_intercept.c
index 3ba64ce..766e9f4 100644
--- a/plugins/experimental/ts_lua/ts_lua_http_intercept.c
+++ b/plugins/experimental/ts_lua/ts_lua_http_intercept.c
@@ -18,19 +18,12 @@
 
 
 #include "ts_lua_util.h"
-
-#define TS_LUA_FUNCTION_HTTP_INTERCEPT "do_intercept"
-#define TS_LUA_FUNCTION_HTTP_SERVER_INTERCEPT "do_server_intercept"
-
-typedef enum {
-  TS_LUA_TYPE_HTTP_INTERCEPT = 0,
-  TS_LUA_TYPE_HTTP_SERVER_INTERCEPT = 1,
-} TSInterceptType;
+#include "ts_lua_http_intercept.h"
 
 static int ts_lua_http_intercept(lua_State *L);
 static int ts_lua_http_server_intercept(lua_State *L);
 static int ts_lua_http_intercept_entry(TSCont contp, TSEvent event, void *edata);
-static void ts_lua_http_intercept_process(ts_lua_http_ctx *http_ctx, TSVConn conn);
+static void ts_lua_http_intercept_process(ts_lua_http_intercept_ctx *ictx, TSVConn conn);
 static void ts_lua_http_intercept_setup_read(ts_lua_http_intercept_ctx *ictx);
 static void ts_lua_http_intercept_setup_write(ts_lua_http_intercept_ctx *ictx);
 static int ts_lua_http_intercept_handler(TSCont contp, TSEvent event, void *edata);
@@ -38,44 +31,65 @@ static int ts_lua_http_intercept_run_coroutine(ts_lua_http_intercept_ctx *ictx,
 static int ts_lua_http_intercept_process_read(TSEvent event, ts_lua_http_intercept_ctx *ictx);
 static int ts_lua_http_intercept_process_write(TSEvent event, ts_lua_http_intercept_ctx *ictx);
 
-extern int ts_lua_flush_launch(ts_lua_http_intercept_ctx *ictx);
+static int ts_lua_say(lua_State *L);
+static int ts_lua_flush(lua_State *L);
+static int ts_lua_flush_wakeup(ts_lua_http_intercept_ctx *ictx);
+static int ts_lua_flush_wakeup_handler(TSCont contp, TSEvent event, void *edata);
+static int ts_lua_flush_cleanup(ts_lua_async_item *ai);
 
 
 void
 ts_lua_inject_http_intercept_api(lua_State *L)
 {
+  /* ts.intercept */
   lua_pushcfunction(L, ts_lua_http_intercept);
   lua_setfield(L, -2, "intercept");
 
+  /* ts.server_intercept */
   lua_pushcfunction(L, ts_lua_http_server_intercept);
   lua_setfield(L, -2, "server_intercept");
 }
 
+void
+ts_lua_inject_intercept_api(lua_State *L)
+{
+  /*  ts.say(...) */
+  lua_pushcfunction(L, ts_lua_say);
+  lua_setfield(L, -2, "say");
+
+  /*  ts.flush(...) */
+  lua_pushcfunction(L, ts_lua_flush);
+  lua_setfield(L, -2, "flush");
+}
+
 static int
 ts_lua_http_intercept(lua_State *L)
 {
   TSCont contp;
-  int type;
+  int type, n;
   ts_lua_http_ctx *http_ctx;
+  ts_lua_http_intercept_ctx *ictx;
 
   http_ctx = ts_lua_get_http_ctx(L);
-  http_ctx->has_hook = 1;
+  n = lua_gettop(L);
 
-  type = lua_type(L, 1);
+  if (n < 1) {
+    ee("ts.http.intercept need at least one param");
+    return 0;
+  }
 
+  type = lua_type(L, 1);
   if (type != LUA_TFUNCTION) {
-    TSError("[%s] param in ts.http.intercept should be a function", __FUNCTION__);
+    ee("ts.http.intercept should use function as param, but there is %s", lua_typename(L, type));
     return 0;
   }
 
-  lua_pushvalue(L, 1);
-  lua_setglobal(L, TS_LUA_FUNCTION_HTTP_INTERCEPT);
-
-  http_ctx->intercept_type = TS_LUA_TYPE_HTTP_INTERCEPT;
-
+  ictx = ts_lua_create_http_intercept_ctx(L, http_ctx, n);
   contp = TSContCreate(ts_lua_http_intercept_entry, TSMutexCreate());
-  TSContDataSet(contp, http_ctx);
+  TSContDataSet(contp, ictx);
+
   TSHttpTxnIntercept(contp, http_ctx->txnp);
+  http_ctx->has_hook = 1;
 
   return 0;
 }
@@ -84,43 +98,51 @@ static int
 ts_lua_http_server_intercept(lua_State *L)
 {
   TSCont contp;
-  int type;
+  int type, n;
   ts_lua_http_ctx *http_ctx;
+  ts_lua_http_intercept_ctx *ictx;
 
   http_ctx = ts_lua_get_http_ctx(L);
-  http_ctx->has_hook = 1;
+  n = lua_gettop(L);
 
-  type = lua_type(L, 1);
+  if (n < 1) {
+    ee("ts.http.server_intercept need at least one param");
+    return 0;
+  }
 
+  type = lua_type(L, 1);
   if (type != LUA_TFUNCTION) {
-    TSError("[%s] param in ts.http.server_intercept should be a function", __FUNCTION__);
+    ee("ts.http.server_intercept should use function as param, but there is %s", lua_typename(L, type));
     return 0;
   }
 
-  lua_pushvalue(L, 1);
-  lua_setglobal(L, TS_LUA_FUNCTION_HTTP_SERVER_INTERCEPT);
-
-  http_ctx->intercept_type = TS_LUA_TYPE_HTTP_SERVER_INTERCEPT;
-
+  ictx = ts_lua_create_http_intercept_ctx(L, http_ctx, n);
   contp = TSContCreate(ts_lua_http_intercept_entry, TSMutexCreate());
-  TSContDataSet(contp, http_ctx);
+  TSContDataSet(contp, ictx);
+
   TSHttpTxnServerIntercept(contp, http_ctx->txnp);
+  http_ctx->has_hook = 1;
 
   return 0;
 }
 
-
 static int
 ts_lua_http_intercept_entry(TSCont contp, TSEvent event, void *edata)
 {
+  ts_lua_http_intercept_ctx *ictx;
+
+  ictx = (ts_lua_http_intercept_ctx *)TSContDataGet(contp);
+
   switch (event) {
   case TS_EVENT_NET_ACCEPT_FAILED:
     if (edata)
       TSVConnClose((TSVConn)edata);
+
+    ts_lua_destroy_http_intercept_ctx(ictx);
     break;
 
   case TS_EVENT_NET_ACCEPT:
-    ts_lua_http_intercept_process((ts_lua_http_ctx *)TSContDataGet(contp), (TSVConn)edata);
+    ts_lua_http_intercept_process(ictx, (TSVConn)edata);
     break;
 
   default:
@@ -132,25 +154,24 @@ ts_lua_http_intercept_entry(TSCont contp, TSEvent event, void *edata)
 }
 
 static void
-ts_lua_http_intercept_process(ts_lua_http_ctx *http_ctx, TSVConn conn)
+ts_lua_http_intercept_process(ts_lua_http_intercept_ctx *ictx, TSVConn conn)
 {
+  int n;
   TSCont contp;
-  lua_State *l;
+  lua_State *L;
   TSMutex mtxp;
-  ts_lua_http_intercept_ctx *ictx;
+  ts_lua_cont_info *ci;
 
-  mtxp = http_ctx->mctx->mutexp;
-  TSMutexLock(mtxp);
-
-  ictx = ts_lua_create_http_intercept_ctx(http_ctx);
+  ci = &ictx->cinfo;
+  mtxp = ictx->cinfo.routine.mctx->mutexp;
 
   contp = TSContCreate(ts_lua_http_intercept_handler, TSMutexCreate());
   TSContDataSet(contp, ictx);
 
-  ictx->contp = contp;
-  ictx->net_vc = conn;
+  ci->contp = contp;
+  ci->mutex = TSContMutexGet(contp);
 
-  l = ictx->lua;
+  ictx->net_vc = conn;
 
   // set up read.
   ts_lua_http_intercept_setup_read(ictx);
@@ -159,13 +180,13 @@ ts_lua_http_intercept_process(ts_lua_http_ctx *http_ctx, TSVConn conn)
   ts_lua_http_intercept_setup_write(ictx);
 
   // invoke function here
-  if (http_ctx->intercept_type == TS_LUA_TYPE_HTTP_INTERCEPT) {
-    lua_getglobal(l, TS_LUA_FUNCTION_HTTP_INTERCEPT);
-  } else {
-    lua_getglobal(l, TS_LUA_FUNCTION_HTTP_SERVER_INTERCEPT);
-  }
+  L = ci->routine.lua;
+
+  TSMutexLock(mtxp);
 
-  ts_lua_http_intercept_run_coroutine(ictx, 0);
+  n = lua_gettop(L);
+
+  ts_lua_http_intercept_run_coroutine(ictx, n - 1);
 
   TSMutexUnlock(mtxp);
 }
@@ -175,7 +196,7 @@ ts_lua_http_intercept_setup_read(ts_lua_http_intercept_ctx *ictx)
 {
   ictx->input.buffer = TSIOBufferCreate();
   ictx->input.reader = TSIOBufferReaderAlloc(ictx->input.buffer);
-  ictx->input.vio = TSVConnRead(ictx->net_vc, ictx->contp, ictx->input.buffer, INT64_MAX);
+  ictx->input.vio = TSVConnRead(ictx->net_vc, ictx->cinfo.contp, ictx->input.buffer, INT64_MAX);
 }
 
 static void
@@ -183,7 +204,7 @@ ts_lua_http_intercept_setup_write(ts_lua_http_intercept_ctx *ictx)
 {
   ictx->output.buffer = TSIOBufferCreate();
   ictx->output.reader = TSIOBufferReaderAlloc(ictx->output.buffer);
-  ictx->output.vio = TSVConnWrite(ictx->net_vc, ictx->contp, ictx->output.reader, INT64_MAX);
+  ictx->output.vio = TSVConnWrite(ictx->net_vc, ictx->cinfo.contp, ictx->output.reader, INT64_MAX);
 }
 
 static int
@@ -203,26 +224,18 @@ ts_lua_http_intercept_handler(TSCont contp, TSEvent event, void *edata)
     ret = ts_lua_http_intercept_process_write(event, ictx);
 
   } else {
-    mtxp = ictx->mctx->mutexp;
-    n = (int)((intptr_t)edata & 0xFFFF);
+    mtxp = ictx->cinfo.routine.mctx->mutexp;
+    n = (intptr_t)edata;
+
     TSMutexLock(mtxp);
     ret = ts_lua_http_intercept_run_coroutine(ictx, n);
+    TSMutexUnlock(mtxp);
   }
 
   if (ret || (ictx->send_complete && ictx->recv_complete)) {
-    TSContDestroy(contp);
-
-    if (!mtxp) {
-      mtxp = ictx->mctx->mutexp;
-      TSMutexLock(mtxp);
-    }
-
     ts_lua_destroy_http_intercept_ctx(ictx);
   }
 
-  if (mtxp)
-    TSMutexUnlock(mtxp);
-
   return 0;
 }
 
@@ -232,10 +245,13 @@ ts_lua_http_intercept_run_coroutine(ts_lua_http_intercept_ctx *ictx, int n)
   int ret;
   int64_t avail;
   int64_t done;
+  ts_lua_cont_info *ci;
   lua_State *L;
 
-  L = ictx->lua;
+  ci = &ictx->cinfo;
+  L = ci->routine.lua;
 
+  ts_lua_set_cont_info(L, ci);
   ret = lua_resume(L, n);
 
   switch (ret) {
@@ -247,17 +263,18 @@ ts_lua_http_intercept_run_coroutine(ts_lua_http_intercept_ctx *ictx, int n)
 
     if (avail) {
       TSVIOReenable(ictx->output.vio);
-
     } else {
       ictx->send_complete = 1;
     }
+
     break;
 
   case 1: // yield
     break;
 
   default: // error
-    TSError("lua_resume failed: %s", lua_tostring(L, -1));
+    ee("lua_resume failed: %s", lua_tostring(L, -1));
+    lua_pop(L, 1);
     return -1;
   }
 
@@ -308,12 +325,13 @@ ts_lua_http_intercept_process_write(TSEvent event, ts_lua_http_intercept_ctx *ic
 
       } else { // we had flush all the data we want
         ictx->to_flush = 0;
-        ts_lua_flush_launch(ictx); // wake up
+        ts_lua_flush_wakeup(ictx); // wake up
       }
 
     } else if (avail > 0) {
       TSVIOReenable(ictx->output.vio);
     }
+
     break;
 
   case TS_EVENT_VCONN_WRITE_COMPLETE:
@@ -327,3 +345,94 @@ ts_lua_http_intercept_process_write(TSEvent event, ts_lua_http_intercept_ctx *ic
 
   return 0;
 }
+
+static int
+ts_lua_say(lua_State *L)
+{
+  const char *data;
+  size_t len;
+  ts_lua_http_intercept_ctx *ictx;
+
+  ictx = ts_lua_get_http_intercept_ctx(L);
+  data = luaL_checklstring(L, 1, &len);
+
+  if (len > 0) {
+    TSIOBufferWrite(ictx->output.buffer, data, len);
+    TSVIOReenable(ictx->output.vio);
+  }
+
+  return 0;
+}
+
+static int
+ts_lua_flush(lua_State *L)
+{
+  int64_t avail;
+  ts_lua_http_intercept_ctx *ictx;
+
+  ictx = ts_lua_get_http_intercept_ctx(L);
+  avail = TSIOBufferReaderAvail(ictx->output.reader);
+
+  if (avail > 0) {
+    ictx->to_flush = TSVIONDoneGet(ictx->output.vio) + TSIOBufferReaderAvail(ictx->output.reader);
+    TSVIOReenable(ictx->output.vio);
+
+    return lua_yield(L, 0);
+  }
+
+  return 0;
+}
+
+static int
+ts_lua_flush_wakeup(ts_lua_http_intercept_ctx *ictx)
+{
+  ts_lua_async_item *ai;
+  ts_lua_cont_info *ci;
+  TSAction action;
+  TSCont contp;
+
+  ci = &ictx->cinfo;
+
+  contp = TSContCreate(ts_lua_flush_wakeup_handler, ci->mutex);
+  action = TSContSchedule(contp, 0, TS_THREAD_POOL_DEFAULT);
+
+  ai = ts_lua_async_create_item(contp, ts_lua_flush_cleanup, (void *)action, ci);
+  TSContDataSet(contp, ai);
+
+  return 0;
+}
+
+static int
+ts_lua_flush_wakeup_handler(TSCont contp, TSEvent event ATS_UNUSED, void *edata ATS_UNUSED)
+{
+  ts_lua_async_item *ai;
+  ts_lua_cont_info *ci;
+
+  ai = TSContDataGet(contp);
+  ci = ai->cinfo;
+
+  ai->data = NULL;
+
+  ts_lua_flush_cleanup(ai);
+
+  TSContCall(ci->contp, TS_LUA_EVENT_COROUTINE_CONT, 0);
+
+  return 0;
+}
+
+static int
+ts_lua_flush_cleanup(ts_lua_async_item *ai)
+{
+  if (ai->deleted)
+    return 0;
+
+  if (ai->data) {
+    TSActionCancel((TSAction)ai->data);
+    ai->data = NULL;
+  }
+
+  TSContDestroy(ai->contp);
+  ai->deleted = 1;
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_http_intercept.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_http_intercept.h b/plugins/experimental/ts_lua/ts_lua_http_intercept.h
index 0f67127..27804f0 100644
--- a/plugins/experimental/ts_lua/ts_lua_http_intercept.h
+++ b/plugins/experimental/ts_lua/ts_lua_http_intercept.h
@@ -23,5 +23,6 @@
 #include "ts_lua_common.h"
 
 void ts_lua_inject_http_intercept_api(lua_State *L);
+void ts_lua_inject_intercept_api(lua_State *L);
 
 #endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_io.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_io.c b/plugins/experimental/ts_lua/ts_lua_io.c
new file mode 100644
index 0000000..39a0c01
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_io.c
@@ -0,0 +1,31 @@
+
+#include "ts_lua_io.h"
+
+int64_t
+IOBufferReaderCopy(TSIOBufferReader readerp, void *buf, int64_t length)
+{
+  int64_t avail, need, n;
+  const char *start;
+  TSIOBufferBlock blk;
+
+  n = 0;
+  blk = TSIOBufferReaderStart(readerp);
+
+  while (blk) {
+    start = TSIOBufferBlockReadStart(blk, readerp, &avail);
+    need = length < avail ? length : avail;
+
+    if (need > 0) {
+      memcpy((char *)buf + n, start, need);
+      length -= need;
+      n += need;
+    }
+
+    if (length == 0)
+      break;
+
+    blk = TSIOBufferBlockNext(blk);
+  }
+
+  return n;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_io.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_io.h b/plugins/experimental/ts_lua/ts_lua_io.h
new file mode 100644
index 0000000..ef65a60
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_io.h
@@ -0,0 +1,10 @@
+
+#ifndef _TS_LUA_IO_H
+#define _TS_LUA_IO_H
+
+#include <ts/ts.h>
+#include <string.h>
+
+int64_t IOBufferReaderCopy(TSIOBufferReader readerp, void *buf, int64_t length);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/68b4d853/plugins/experimental/ts_lua/ts_lua_misc.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_misc.c b/plugins/experimental/ts_lua/ts_lua_misc.c
index 5de238d..5840bd2 100644
--- a/plugins/experimental/ts_lua/ts_lua_misc.c
+++ b/plugins/experimental/ts_lua/ts_lua_misc.c
@@ -24,17 +24,10 @@ static int ts_lua_get_now_time(lua_State *L);
 static int ts_lua_debug(lua_State *L);
 static int ts_lua_error(lua_State *L);
 static int ts_lua_sleep(lua_State *L);
-static int ts_lua_say(lua_State *L);
-static int ts_lua_flush(lua_State *L);
 
-static int ts_lua_sleep_cleanup(struct ict_item *item);
+static int ts_lua_sleep_cleanup(ts_lua_async_item *ai);
 static int ts_lua_sleep_handler(TSCont contp, TSEvent event, void *edata);
 
-int ts_lua_flush_launch(ts_lua_http_intercept_ctx *ictx);
-static int ts_lua_flush_cleanup(struct ict_item *item);
-static int ts_lua_flush_handler(TSCont contp, TSEvent event, void *edata);
-
-
 void
 ts_lua_inject_misc_api(lua_State *L)
 {
@@ -53,14 +46,6 @@ ts_lua_inject_misc_api(lua_State *L)
   /* ts.sleep(...) */
   lua_pushcfunction(L, ts_lua_sleep);
   lua_setfield(L, -2, "sleep");
-
-  /* ts.say(...) */
-  lua_pushcfunction(L, ts_lua_say);
-  lua_setfield(L, -2, "say");
-
-  /* ts.flush(...) */
-  lua_pushcfunction(L, ts_lua_flush);
-  lua_setfield(L, -2, "flush");
 }
 
 static int
@@ -99,132 +84,54 @@ ts_lua_sleep(lua_State *L)
   int sec;
   TSAction action;
   TSCont contp;
-  ts_lua_http_intercept_item *node;
-  ts_lua_http_intercept_ctx *ictx;
+  ts_lua_async_item *ai;
+  ts_lua_cont_info *ci;
+
+  ci = ts_lua_get_cont_info(L);
+  if (ci == NULL)
+    return 0;
 
-  ictx = ts_lua_get_http_intercept_ctx(L);
   sec = luaL_checknumber(L, 1);
+  if (sec < 1) {
+    sec = 1;
+  }
 
-  contp = TSContCreate(ts_lua_sleep_handler, TSContMutexGet(ictx->contp));
+  contp = TSContCreate(ts_lua_sleep_handler, ci->mutex);
   action = TSContSchedule(contp, sec * 1000, TS_THREAD_POOL_DEFAULT);
 
-  node = (ts_lua_http_intercept_item *)TSmalloc(sizeof(ts_lua_http_intercept_item));
-  TS_LUA_ADD_INTERCEPT_ITEM(ictx, node, contp, ts_lua_sleep_cleanup, action);
-  TSContDataSet(contp, node);
+  ai = ts_lua_async_create_item(contp, ts_lua_sleep_cleanup, (void *)action, ci);
+  TSContDataSet(contp, ai);
 
   return lua_yield(L, 0);
 }
 
 static int
-ts_lua_say(lua_State *L)
-{
-  const char *data;
-  size_t len;
-
-  ts_lua_http_intercept_ctx *ictx;
-
-  ictx = ts_lua_get_http_intercept_ctx(L);
-
-  data = luaL_checklstring(L, 1, &len);
-
-  if (len > 0) {
-    TSIOBufferWrite(ictx->output.buffer, data, len);
-    TSVIOReenable(ictx->output.vio);
-  }
-
-  return 0;
-}
-
-static int
-ts_lua_flush(lua_State *L)
-{
-  int64_t avail;
-  ts_lua_http_intercept_ctx *ictx;
-
-  ictx = ts_lua_get_http_intercept_ctx(L);
-  avail = TSIOBufferReaderAvail(ictx->output.reader);
-
-  if (avail > 0) {
-    ictx->to_flush = TSVIONDoneGet(ictx->output.vio) + TSIOBufferReaderAvail(ictx->output.reader);
-    TSVIOReenable(ictx->output.vio);
-
-    return lua_yield(L, 0);
-  }
-
-  return 0;
-}
-
-static int
-ts_lua_sleep_handler(TSCont contp, TSEvent event, void *edata ATS_UNUSED)
+ts_lua_sleep_handler(TSCont contp, TSEvent event ATS_UNUSED, void *edata ATS_UNUSED)
 {
-  ts_lua_http_intercept_item *item = TSContDataGet(contp);
+  ts_lua_async_item *ai;
+  ts_lua_cont_info *ci;
 
-  ts_lua_sleep_cleanup(item);
+  ai = TSContDataGet(contp);
+  ci = ai->cinfo;
 
-  TSContCall(item->ictx->contp, event, 0);
+  ai->data = NULL;
+  ts_lua_sleep_cleanup(ai);
 
-  return 0;
-}
-
-static int
-ts_lua_sleep_cleanup(struct ict_item *item)
-{
-  if (item->deleted)
-    return 0;
-
-  if (item->data) {
-    TSActionCancel((TSAction)item->data);
-    item->data = NULL;
-  }
-
-  TSContDestroy(item->contp);
-  item->deleted = 1;
-
-  return 0;
-}
-
-int
-ts_lua_flush_launch(ts_lua_http_intercept_ctx *ictx)
-{
-  TSAction action;
-  TSCont contp;
-  ts_lua_http_intercept_item *node;
-
-  contp = TSContCreate(ts_lua_flush_handler, TSContMutexGet(ictx->contp));
-  action = TSContSchedule(contp, 0, TS_THREAD_POOL_DEFAULT);
-
-  node = (ts_lua_http_intercept_item *)TSmalloc(sizeof(ts_lua_http_intercept_item));
-  TS_LUA_ADD_INTERCEPT_ITEM(ictx, node, contp, ts_lua_flush_cleanup, action);
-  TSContDataSet(contp, node);
+  TSContCall(ci->contp, TS_LUA_EVENT_COROUTINE_CONT, 0);
 
   return 0;
 }
 
 static int
-ts_lua_flush_cleanup(struct ict_item *item)
+ts_lua_sleep_cleanup(ts_lua_async_item *ai)
 {
-  if (item->deleted)
-    return 0;
-
-  if (item->data) {
-    TSActionCancel((TSAction)item->data);
-    item->data = NULL;
+  if (ai->data) {
+    TSActionCancel((TSAction)ai->data);
+    ai->data = NULL;
   }
 
-  TSContDestroy(item->contp);
-  item->deleted = 1;
-
-  return 0;
-}
-
-static int
-ts_lua_flush_handler(TSCont contp, TSEvent event, void *edata ATS_UNUSED)
-{
-  ts_lua_http_intercept_item *item = TSContDataGet(contp);
-
-  ts_lua_flush_cleanup(item);
-
-  TSContCall(item->ictx->contp, event, 0);
+  TSContDestroy(ai->contp);
+  ai->deleted = 1;
 
   return 0;
 }