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 2014/05/22 22:39:32 UTC

[1/3] TS-2723: add new features to ts_lua plugin

Repository: trafficserver
Updated Branches:
  refs/heads/master d9f04c7ae -> 83f16762b


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_server_response.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_server_response.c b/plugins/experimental/ts_lua/ts_lua_server_response.c
index ff6b41a..55a2a63 100644
--- a/plugins/experimental/ts_lua/ts_lua_server_response.c
+++ b/plugins/experimental/ts_lua/ts_lua_server_response.c
@@ -19,13 +19,32 @@
 
 #include "ts_lua_util.h"
 
+#define TS_LUA_CHECK_SERVER_RESPONSE_HDR(http_ctx)     \
+do {        \
+    if (!http_ctx->server_response_hdrp) {           \
+        if (TSHttpTxnServerRespGet(http_ctx->txnp,   \
+                    &http_ctx->server_response_bufp, \
+                    &http_ctx->server_response_hdrp) != TS_SUCCESS) {    \
+            return 0;   \
+        }   \
+    }   \
+} while(0)
+
+
 static void ts_lua_inject_server_response_header_api(lua_State * L);
-static void ts_lua_inject_server_response_header_misc_api(lua_State * L);
+static void ts_lua_inject_server_response_headers_api(lua_State * L);
+static void ts_lua_inject_server_response_misc_api(lua_State * L);
 
 static int ts_lua_server_response_header_get(lua_State * L);
 static int ts_lua_server_response_header_set(lua_State * L);
 
-static int ts_lua_server_response_header_get_status(lua_State * L);
+static int ts_lua_server_response_get_headers(lua_State * L);
+
+static int ts_lua_server_response_get_status(lua_State * L);
+static int ts_lua_server_response_set_status(lua_State * L);
+
+static int ts_lua_server_response_get_version(lua_State * L);
+static int ts_lua_server_response_set_version(lua_State * L);
 
 
 void
@@ -34,6 +53,8 @@ ts_lua_inject_server_response_api(lua_State * L)
   lua_newtable(L);
 
   ts_lua_inject_server_response_header_api(L);
+  ts_lua_inject_server_response_headers_api(L);
+  ts_lua_inject_server_response_misc_api(L);
 
   lua_setfield(L, -2, "server_response");
 }
@@ -44,8 +65,6 @@ ts_lua_inject_server_response_header_api(lua_State * L)
 {
   lua_newtable(L);              /* .header */
 
-  ts_lua_inject_server_response_header_misc_api(L);
-
   lua_createtable(L, 0, 2);     /* metatable for .header */
 
   lua_pushcfunction(L, ts_lua_server_response_header_get);
@@ -59,10 +78,65 @@ ts_lua_inject_server_response_header_api(lua_State * L)
 }
 
 static void
-ts_lua_inject_server_response_header_misc_api(lua_State * L)
+ts_lua_inject_server_response_headers_api(lua_State * L)
 {
-  lua_pushcfunction(L, ts_lua_server_response_header_get_status);
+  lua_pushcfunction(L, ts_lua_server_response_get_headers);
+  lua_setfield(L, -2, "get_headers");
+}
+
+static int
+ts_lua_server_response_get_headers(lua_State * L)
+{
+  const char *name;
+  const char *value;
+  int name_len;
+  int value_len;
+  TSMLoc field_loc;
+  TSMLoc next_field_loc;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_RESPONSE_HDR(http_ctx);
+
+  lua_newtable(L);
+
+  field_loc = TSMimeHdrFieldGet(http_ctx->server_response_bufp, http_ctx->server_response_hdrp, 0);
+
+  while (field_loc) {
+
+    name = TSMimeHdrFieldNameGet(http_ctx->server_response_bufp, http_ctx->server_response_hdrp, field_loc, &name_len);
+    if (name && name_len) {
+
+      value =
+        TSMimeHdrFieldValueStringGet(http_ctx->server_response_bufp, http_ctx->server_response_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(http_ctx->server_response_bufp, http_ctx->server_response_hdrp, field_loc);
+    TSHandleMLocRelease(http_ctx->server_response_bufp, http_ctx->server_response_hdrp, field_loc);
+    field_loc = next_field_loc;
+  }
+
+  return 1;
+}
+
+static void
+ts_lua_inject_server_response_misc_api(lua_State * L)
+{
+  lua_pushcfunction(L, ts_lua_server_response_get_status);
   lua_setfield(L, -2, "get_status");
+  lua_pushcfunction(L, ts_lua_server_response_set_status);
+  lua_setfield(L, -2, "set_status");
+
+  lua_pushcfunction(L, ts_lua_server_response_get_version);
+  lua_setfield(L, -2, "get_version");
+  lua_pushcfunction(L, ts_lua_server_response_set_version);
+  lua_setfield(L, -2, "set_version");
 }
 
 static int
@@ -81,14 +155,7 @@ ts_lua_server_response_header_get(lua_State * L)
   /*  we skip the first argument that is the table */
   key = luaL_checklstring(L, 2, &key_len);
 
-  if (!http_ctx->server_response_hdrp) {
-    if (TSHttpTxnServerRespGet(http_ctx->txnp,
-                               &http_ctx->server_response_bufp, &http_ctx->server_response_hdrp) != TS_SUCCESS) {
-
-      lua_pushnil(L);
-      return 1;
-    }
-  }
+  TS_LUA_CHECK_SERVER_RESPONSE_HDR(http_ctx);
 
   if (key && key_len) {
 
@@ -137,12 +204,7 @@ ts_lua_server_response_header_set(lua_State * L)
     val = luaL_checklstring(L, 3, &val_len);
   }
 
-  if (!http_ctx->server_response_hdrp) {
-    if (TSHttpTxnServerRespGet(http_ctx->txnp,
-                               &http_ctx->server_response_bufp, &http_ctx->server_response_hdrp) != TS_SUCCESS) {
-      return 0;
-    }
-  }
+  TS_LUA_CHECK_SERVER_RESPONSE_HDR(http_ctx);
 
   field_loc = TSMimeHdrFieldFind(http_ctx->server_response_bufp, http_ctx->server_response_hdrp, key, key_len);
 
@@ -173,21 +235,14 @@ ts_lua_server_response_header_set(lua_State * L)
 }
 
 static int
-ts_lua_server_response_header_get_status(lua_State * L)
+ts_lua_server_response_get_status(lua_State * L)
 {
   int status;
   ts_lua_http_ctx *http_ctx;
 
   http_ctx = ts_lua_get_http_ctx(L);
 
-  if (!http_ctx->server_response_hdrp) {
-    if (TSHttpTxnServerRespGet(http_ctx->txnp,
-                               &http_ctx->server_response_bufp, &http_ctx->server_response_hdrp) != TS_SUCCESS) {
-
-      lua_pushnil(L);
-      return 1;
-    }
-  }
+  TS_LUA_CHECK_SERVER_RESPONSE_HDR(http_ctx);
 
   status = TSHttpHdrStatusGet(http_ctx->server_response_bufp, http_ctx->server_response_hdrp);
 
@@ -195,3 +250,70 @@ ts_lua_server_response_header_get_status(lua_State * L)
 
   return 1;
 }
+
+static int
+ts_lua_server_response_set_status(lua_State * L)
+{
+  int status;
+  const char *reason;
+  int reason_len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_RESPONSE_HDR(http_ctx);
+
+  status = luaL_checkint(L, 1);
+
+  reason = TSHttpHdrReasonLookup(status);
+  reason_len = strlen(reason);
+
+  TSHttpHdrStatusSet(http_ctx->server_response_bufp, http_ctx->server_response_hdrp, status);
+  TSHttpHdrReasonSet(http_ctx->server_response_bufp, http_ctx->server_response_hdrp, reason, reason_len);
+
+  return 0;
+}
+
+static int
+ts_lua_server_response_get_version(lua_State * L)
+{
+  int version;
+  char buf[32];
+  int n;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_RESPONSE_HDR(http_ctx);
+
+  version = TSHttpHdrVersionGet(http_ctx->server_response_bufp, http_ctx->server_response_hdrp);
+
+  n = snprintf(buf, sizeof(buf) - 1, "%d.%d", TS_HTTP_MAJOR(version), TS_HTTP_MINOR(version));
+  lua_pushlstring(L, buf, n);
+
+  return 1;
+}
+
+static int
+ts_lua_server_response_set_version(lua_State * L)
+{
+  const char *version;
+  size_t len;
+  int major, minor;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_RESPONSE_HDR(http_ctx);
+
+  version = luaL_checklstring(L, 1, &len);
+
+  sscanf(version, "%2u.%2u", &major, &minor);
+
+  TSHttpHdrVersionSet(http_ctx->server_response_bufp, http_ctx->server_response_hdrp, TS_HTTP_VERSION(major, minor));
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_string.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_string.c b/plugins/experimental/ts_lua/ts_lua_string.c
new file mode 100644
index 0000000..40511c5
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_string.c
@@ -0,0 +1,32 @@
+/*
+  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_string.h"
+
+u_char *
+ts_lua_hex_dump(u_char * dst, u_char * src, size_t len)
+{
+  static u_char hex[] = "0123456789abcdef";
+
+  while (len--) {
+    *dst++ = hex[*src >> 4];
+    *dst++ = hex[*src++ & 0xf];
+  }
+
+  return dst;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_string.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_string.h b/plugins/experimental/ts_lua/ts_lua_string.h
new file mode 100644
index 0000000..2d67468
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_string.h
@@ -0,0 +1,30 @@
+/*
+  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_STRING_H
+#define _TS_LUA_STRING_H
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <sys/types.h>
+#include <inttypes.h>
+
+u_char *ts_lua_hex_dump(u_char * dst, u_char * src, size_t len);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 010a31d..0030fa5 100644
--- a/plugins/experimental/ts_lua/ts_lua_util.c
+++ b/plugins/experimental/ts_lua/ts_lua_util.c
@@ -29,6 +29,9 @@
 #include "ts_lua_http.h"
 #include "ts_lua_misc.h"
 #include "ts_lua_log.h"
+#include "ts_lua_crypto.h"
+#include "ts_lua_mgmt.h"
+#include "ts_lua_package.h"
 
 static lua_State *ts_lua_new_state();
 static void ts_lua_init_registry(lua_State * L);
@@ -51,7 +54,7 @@ ts_lua_create_vm(ts_lua_main_ctx * arr, int n)
 
     lua_pushvalue(L, LUA_GLOBALSINDEX);
 
-    arr[i].gref = luaL_ref(L, LUA_REGISTRYINDEX);
+    arr[i].gref = luaL_ref(L, LUA_REGISTRYINDEX);       /* L[REG][gref] = L[GLOBAL] */
     arr[i].lua = L;
     arr[i].mutexp = TSMutexCreate();
   }
@@ -103,27 +106,43 @@ ts_lua_add_module(ts_lua_instance_conf * conf, ts_lua_main_ctx * arr, int n, int
   lua_State *L;
 
   for (i = 0; i < n; i++) {
+    conf->_first = (i == 0) ? 1 : 0;
+    conf->_last = (i == n - 1) ? 1 : 0;
+
+    TSMutexLock(arr[i].mutexp);
 
     L = arr[i].lua;
 
-    lua_newtable(L);            // create this module's global table
-    lua_pushvalue(L, -1);
-    lua_setfield(L, -2, "_G");
-    lua_newtable(L);
-    lua_rawgeti(L, LUA_REGISTRYINDEX, arr[i].gref);
-    lua_setfield(L, -2, "__index");
-    lua_setmetatable(L, -2);
-    lua_replace(L, LUA_GLOBALSINDEX);
+    lua_newtable(L);            /* new TB1 */
+    lua_pushvalue(L, -1);       /* new TB2 */
+    lua_setfield(L, -2, "_G");  /* TB1[_G] = TB2 empty table, we can change _G to xx */
+    lua_newtable(L);            /* new TB3 */
+    lua_rawgeti(L, LUA_REGISTRYINDEX, arr[i].gref);     /* push L[GLOBAL] */
+    lua_setfield(L, -2, "__index");     /* TB3[__index] = L[GLOBAL] which has ts.xxx api */
+    lua_setmetatable(L, -2);    /* TB1[META]  = TB3 */
+    lua_replace(L, LUA_GLOBALSINDEX);   /* L[GLOBAL] = TB1 */
+
+    ts_lua_set_instance_conf(L, conf);
+
+    if (conf->content) {
+      if (luaL_loadstring(L, conf->content)) {
+        fprintf(stderr, "[%s] luaL_loadstring %s failed: %s\n", __FUNCTION__, conf->script, lua_tostring(L, -1));
+        lua_pop(L, 1);
+        TSMutexUnlock(arr[i].mutexp);
+        return -1;
+      }
 
-    if (luaL_loadfile(L, conf->script)) {
+    } else if (luaL_loadfile(L, conf->script)) {
       fprintf(stderr, "[%s] luaL_loadfile %s failed: %s\n", __FUNCTION__, conf->script, lua_tostring(L, -1));
       lua_pop(L, 1);
+      TSMutexUnlock(arr[i].mutexp);
       return -1;
     }
 
     if (lua_pcall(L, 0, 0, 0)) {
       fprintf(stderr, "[%s] lua_pcall %s failed: %s\n", __FUNCTION__, conf->script, lua_tostring(L, -1));
       lua_pop(L, 1);
+      TSMutexUnlock(arr[i].mutexp);
       return -1;
     }
 
@@ -143,14 +162,17 @@ ts_lua_add_module(ts_lua_instance_conf * conf, ts_lua_main_ctx * arr, int n, int
       if (lua_pcall(L, 1, 1, 0)) {
         fprintf(stderr, "[%s] lua_pcall %s failed: %s\n", __FUNCTION__, conf->script, lua_tostring(L, -1));
         lua_pop(L, 1);
+        TSMutexUnlock(arr[i].mutexp);
         return -1;
       }
 
       ret = lua_tonumber(L, -1);
       lua_pop(L, 1);
 
-      if (ret)
+      if (ret) {
+        TSMutexUnlock(arr[i].mutexp);
         return -1;              /* script parse error */
+      }
 
     } else {
       lua_pop(L, 1);            /* pop nil */
@@ -159,16 +181,70 @@ ts_lua_add_module(ts_lua_instance_conf * conf, ts_lua_main_ctx * arr, int n, int
 
     lua_pushlightuserdata(L, conf);
     lua_pushvalue(L, LUA_GLOBALSINDEX);
-    lua_rawset(L, LUA_REGISTRYINDEX);
+    lua_rawset(L, LUA_REGISTRYINDEX);   /* L[REG][conf] = L[GLOBAL] */
 
     lua_newtable(L);
-    lua_replace(L, LUA_GLOBALSINDEX);   // set empty table to global
+    lua_replace(L, LUA_GLOBALSINDEX);   /* L[GLOBAL] = EMPTY */
+
+    TSMutexUnlock(arr[i].mutexp);
   }
 
+  return 0;
+}
+
+int
+ts_lua_del_module(ts_lua_instance_conf * conf, ts_lua_main_ctx * arr, int n)
+{
+  int i;
+  lua_State *L;
+
+  for (i = 0; i < n; i++) {
+
+    TSMutexLock(arr[i].mutexp);
+
+    L = arr[i].lua;
+
+    /* call "__clean__", to clean resources */
+    lua_pushlightuserdata(L, conf);
+    lua_rawget(L, LUA_REGISTRYINDEX);
+    lua_replace(L, LUA_GLOBALSINDEX);   /* L[GLOBAL] = L[REG][conf] */
+
+    lua_getglobal(L, "__clean__");      /* get __clean__ function */
+
+    if (lua_type(L, -1) == LUA_TFUNCTION) {
+
+      if (lua_pcall(L, 0, 0, 0)) {
+        fprintf(stderr, "[%s] lua_pcall %s failed: %s\n", __FUNCTION__, conf->script, lua_tostring(L, -1));
+      }
+
+    } else {
+      lua_pop(L, 1);            /* pop nil */
+    }
+
+    lua_pushlightuserdata(L, conf);
+    lua_pushnil(L);
+    lua_rawset(L, LUA_REGISTRYINDEX);   /* L[REG][conf] = nil */
+
+    lua_newtable(L);
+    lua_replace(L, LUA_GLOBALSINDEX);   /* L[GLOBAL] = EMPTY  */
+
+    TSMutexUnlock(arr[i].mutexp);
+  }
 
   return 0;
 }
 
+int
+ts_lua_init_instance(ts_lua_instance_conf * conf)
+{
+  return 0;
+}
+
+int
+ts_lua_del_instance(ts_lua_instance_conf * conf)
+{
+  return 0;
+}
 
 static void
 ts_lua_init_registry(lua_State * L ATS_UNUSED)
@@ -201,6 +277,9 @@ ts_lua_inject_ts_api(lua_State * L)
 
   ts_lua_inject_http_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);
 
   lua_getglobal(L, "package");
   lua_getfield(L, -1, "loaded");
@@ -212,6 +291,28 @@ ts_lua_inject_ts_api(lua_State * L)
 }
 
 void
+ts_lua_set_instance_conf(lua_State * L, ts_lua_instance_conf * conf)
+{
+  lua_pushliteral(L, "__ts_instance_conf");
+  lua_pushlightuserdata(L, conf);
+  lua_rawset(L, LUA_GLOBALSINDEX);
+}
+
+ts_lua_instance_conf *
+ts_lua_get_instance_conf(lua_State * L)
+{
+  ts_lua_instance_conf *conf;
+
+  lua_pushliteral(L, "__ts_instance_conf");
+  lua_rawget(L, LUA_GLOBALSINDEX);
+  conf = lua_touserdata(L, -1);
+
+  lua_pop(L, 1);                // pop the conf out
+
+  return conf;
+}
+
+void
 ts_lua_set_http_ctx(lua_State * L, ts_lua_http_ctx * ctx)
 {
   lua_pushliteral(L, "__ts_http_ctx");
@@ -236,14 +337,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)
 {
+  int i, size;
   ts_lua_http_ctx *http_ctx;
   lua_State *L;
   lua_State *l;
 
   L = main_ctx->lua;
 
-  http_ctx = TSmalloc(sizeof(ts_lua_http_ctx));
-  memset(http_ctx, 0, sizeof(ts_lua_http_ctx));
+  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->lua = lua_newthread(L);
   l = http_ctx->lua;
@@ -265,6 +371,7 @@ ts_lua_create_http_ctx(ts_lua_main_ctx * main_ctx, ts_lua_instance_conf * conf)
   http_ctx->ref = luaL_ref(L, LUA_REGISTRYINDEX);
 
   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);
@@ -286,6 +393,10 @@ ts_lua_destroy_http_ctx(ts_lua_http_ctx * http_ctx)
     }
   }
 
+  if (http_ctx->server_request_url) {
+    TSHandleMLocRelease(http_ctx->server_request_bufp, http_ctx->server_request_hdrp, http_ctx->server_request_url);
+  }
+
   if (http_ctx->server_request_bufp) {
     TSHandleMLocRelease(http_ctx->server_request_bufp, TS_NULL_MLOC, http_ctx->server_request_hdrp);
   }
@@ -299,7 +410,9 @@ ts_lua_destroy_http_ctx(ts_lua_http_ctx * http_ctx)
   }
 
   if (http_ctx->cached_response_bufp) {
+    TSMimeHdrDestroy(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp);
     TSHandleMLocRelease(http_ctx->cached_response_bufp, TS_NULL_MLOC, http_ctx->cached_response_hdrp);
+    TSMBufferDestroy(http_ctx->cached_response_bufp);
   }
 
   luaL_unref(main_ctx->lua, LUA_REGISTRYINDEX, http_ctx->ref);
@@ -331,18 +444,24 @@ 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)
 {
+  int i, size;
   lua_State *L;
   ts_lua_http_intercept_ctx *ictx;
 
   L = http_ctx->lua;
 
-  ictx = TSmalloc(sizeof(ts_lua_http_intercept_ctx));
-  memset(ictx, 0, sizeof(ts_lua_http_intercept_ctx));
+  size = TS_LUA_MEM_ALIGN(sizeof(ts_lua_http_intercept_ctx));
+  ictx = TSmalloc(size);
+
+  for (i = 0; i < TS_LUA_ALIGN_COUNT(size); i++) {
+    ((void **) ictx)[i] = 0;
+  }
 
   ictx->lua = lua_newthread(L);
 
   ictx->ref = luaL_ref(L, LUA_REGISTRYINDEX);
 
+  ictx->mctx = http_ctx->mctx;
   ictx->hctx = http_ctx;
 
   ts_lua_set_http_intercept_ctx(ictx->lua, ictx);
@@ -353,9 +472,10 @@ 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_http_ctx *http_ctx;
+  ts_lua_main_ctx *main_ctx;
+  struct ict_item *node, *snode;
 
-  http_ctx = ictx->hctx;
+  main_ctx = ictx->mctx;
 
   if (ictx->net_vc)
     TSVConnClose(ictx->net_vc);
@@ -363,7 +483,20 @@ ts_lua_destroy_http_intercept_ctx(ts_lua_http_intercept_ctx * ictx)
   TS_LUA_RELEASE_IO_HANDLE((&ictx->input));
   TS_LUA_RELEASE_IO_HANDLE((&ictx->output));
 
-  luaL_unref(http_ctx->lua, LUA_REGISTRYINDEX, ictx->ref);
+  node = ictx->ict_chain;
+
+  while (node) {
+
+    if (node->cleanup)
+      node->cleanup(node);
+
+    snode = node;
+    node = node->next;
+
+    TSfree(snode);
+  }
+
+  luaL_unref(main_ctx->lua, LUA_REGISTRYINDEX, ictx->ref);
   TSfree(ictx);
   return;
 }
@@ -387,6 +520,7 @@ int
 ts_lua_http_cont_handler(TSCont contp, TSEvent event, void *edata)
 {
   TSHttpTxn txnp = (TSHttpTxn) edata;
+  int ret;
   lua_State *l;
   ts_lua_http_ctx *http_ctx;
   ts_lua_main_ctx *main_ctx;
@@ -394,141 +528,84 @@ ts_lua_http_cont_handler(TSCont contp, TSEvent event, void *edata)
   http_ctx = (ts_lua_http_ctx *) TSContDataGet(contp);
   main_ctx = http_ctx->mctx;
 
+  ret = 0;
   l = http_ctx->lua;
 
   TSMutexLock(main_ctx->mutexp);
 
   switch (event) {
 
-  case TS_EVENT_HTTP_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)) {
-        fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
-      }
-    }
-
-    break;
+  case TS_EVENT_HTTP_POST_REMAP:
 
-  case TS_EVENT_HTTP_SEND_REQUEST_HDR:
+    lua_getglobal(l, TS_LUA_FUNCTION_POST_REMAP);
 
-    lua_getglobal(l, TS_LUA_FUNCTION_SEND_REQUEST);
     if (lua_type(l, -1) == LUA_TFUNCTION) {
       if (lua_pcall(l, 0, 1, 0)) {
         fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
       }
-    }
-
-    break;
-
-  case TS_EVENT_HTTP_READ_RESPONSE_HDR:
 
-    lua_getglobal(l, TS_LUA_FUNCTION_READ_RESPONSE);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
-      }
+      ret = lua_tointeger(l, -1);
     }
 
     break;
 
-  case TS_EVENT_HTTP_SEND_RESPONSE_HDR:
-
-    lua_getglobal(l, TS_LUA_FUNCTION_SEND_RESPONSE);
-    if (lua_type(l, -1) == LUA_TFUNCTION) {
-      if (lua_pcall(l, 0, 1, 0)) {
-        fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
-      }
-    }
-
-    break;
+  case TS_EVENT_HTTP_CACHE_LOOKUP_COMPLETE:
 
-  case TS_EVENT_HTTP_READ_REQUEST_HDR:
+    lua_getglobal(l, TS_LUA_FUNCTION_CACHE_LOOKUP_COMPLETE);
 
-    lua_getglobal(l, TS_LUA_FUNCTION_READ_REQUEST);
     if (lua_type(l, -1) == LUA_TFUNCTION) {
       if (lua_pcall(l, 0, 1, 0)) {
         fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
       }
-    }
-
-    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)) {
-        fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
-      }
+      ret = lua_tointeger(l, -1);
     }
 
     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)) {
-        fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
-      }
-    }
-
-    break;
+  case TS_EVENT_HTTP_SEND_REQUEST_HDR:
 
-  case TS_EVENT_HTTP_POST_REMAP:
+    lua_getglobal(l, TS_LUA_FUNCTION_SEND_REQUEST);
 
-    lua_getglobal(l, TS_LUA_FUNCTION_POST_REMAP);
     if (lua_type(l, -1) == LUA_TFUNCTION) {
       if (lua_pcall(l, 0, 1, 0)) {
         fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
       }
-    }
-
-    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)) {
-        fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
-      }
+      ret = lua_tointeger(l, -1);
     }
 
     break;
 
-  case TS_EVENT_HTTP_SELECT_ALT:
+  case TS_EVENT_HTTP_READ_RESPONSE_HDR:
+
+    lua_getglobal(l, TS_LUA_FUNCTION_READ_RESPONSE);
 
-    lua_getglobal(l, TS_LUA_FUNCTION_SELECT_ALT);
     if (lua_type(l, -1) == LUA_TFUNCTION) {
       if (lua_pcall(l, 0, 1, 0)) {
         fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
       }
+
+      ret = lua_tointeger(l, -1);
     }
 
     break;
 
-  case TS_EVENT_HTTP_READ_CACHE_HDR:
+  case TS_EVENT_HTTP_SEND_RESPONSE_HDR:
+
+    lua_getglobal(l, TS_LUA_FUNCTION_SEND_RESPONSE);
 
-    lua_getglobal(l, TS_LUA_FUNCTION_READ_CACHE);
     if (lua_type(l, -1) == LUA_TFUNCTION) {
       if (lua_pcall(l, 0, 1, 0)) {
         fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
       }
+
+      ret = lua_tointeger(l, -1);
     }
 
     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)) {
-        fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
-      }
-    }
-
     ts_lua_destroy_http_ctx(http_ctx);
     TSContDestroy(contp);
     break;
@@ -538,6 +615,13 @@ ts_lua_http_cont_handler(TSCont contp, TSEvent event, void *edata)
   }
 
   TSMutexUnlock(main_ctx->mutexp);
-  TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
+
+  if (ret) {
+    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_ERROR);
+
+  } else {
+    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
+  }
+
   return 0;
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 c30a249..efd1b3a 100644
--- a/plugins/experimental/ts_lua/ts_lua_util.h
+++ b/plugins/experimental/ts_lua/ts_lua_util.h
@@ -27,6 +27,14 @@ void ts_lua_destroy_vm(ts_lua_main_ctx * arr, int n);
 
 int ts_lua_add_module(ts_lua_instance_conf * conf, ts_lua_main_ctx * arr, int n, int argc, char *argv[]);
 
+int ts_lua_del_module(ts_lua_instance_conf * conf, ts_lua_main_ctx * arr, int n);
+
+int ts_lua_init_instance(ts_lua_instance_conf * conf);
+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_http_ctx(lua_State * L, ts_lua_http_ctx * ctx);
 ts_lua_http_ctx *ts_lua_get_http_ctx(lua_State * L);
 


[3/3] git commit: TS-2723: add new features to ts_lua plugin

Posted by ki...@apache.org.
TS-2723: add new features to ts_lua plugin


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

Branch: refs/heads/master
Commit: 83f16762be04abc075e5ef5466aa921e51e294eb
Parents: d9f04c7
Author: Gang Li <qu...@taobao.com>
Authored: Thu May 22 13:38:41 2014 -0700
Committer: Kit Chan <ki...@apache.org>
Committed: Thu May 22 13:38:41 2014 -0700

----------------------------------------------------------------------
 CHANGES                                         |    3 +
 doc/reference/plugins/ts_lua.en.rst             | 2129 +++++++++++++++---
 plugins/experimental/ts_lua/Makefile.am         |    8 +-
 plugins/experimental/ts_lua/ts_lua.c            |   58 +-
 plugins/experimental/ts_lua/ts_lua_atomic.c     |   96 -
 plugins/experimental/ts_lua/ts_lua_atomic.h     |   33 -
 .../ts_lua/ts_lua_cached_response.c             |  176 +-
 .../experimental/ts_lua/ts_lua_client_request.c |  226 ++
 .../ts_lua/ts_lua_client_response.c             |  229 +-
 plugins/experimental/ts_lua/ts_lua_common.h     |   76 +-
 plugins/experimental/ts_lua/ts_lua_crypto.c     |  177 ++
 plugins/experimental/ts_lua/ts_lua_crypto.h     |   27 +
 plugins/experimental/ts_lua/ts_lua_hook.c       |    2 +-
 plugins/experimental/ts_lua/ts_lua_http.c       |   39 +-
 plugins/experimental/ts_lua/ts_lua_http_cntl.c  |  101 +
 plugins/experimental/ts_lua/ts_lua_http_cntl.h  |   27 +
 .../experimental/ts_lua/ts_lua_http_config.c    |  304 +++
 .../experimental/ts_lua/ts_lua_http_config.h    |   27 +
 .../experimental/ts_lua/ts_lua_http_intercept.c |   64 +-
 plugins/experimental/ts_lua/ts_lua_log.c        |    2 +-
 plugins/experimental/ts_lua/ts_lua_mgmt.c       |  106 +
 plugins/experimental/ts_lua/ts_lua_mgmt.h       |   27 +
 plugins/experimental/ts_lua/ts_lua_misc.c       |  146 +-
 plugins/experimental/ts_lua/ts_lua_package.c    |  297 +++
 plugins/experimental/ts_lua/ts_lua_package.h    |   27 +
 plugins/experimental/ts_lua/ts_lua_remap.c      |   36 +-
 .../experimental/ts_lua/ts_lua_server_request.c |  194 ++
 .../ts_lua/ts_lua_server_response.c             |  180 +-
 plugins/experimental/ts_lua/ts_lua_string.c     |   32 +
 plugins/experimental/ts_lua/ts_lua_string.h     |   30 +
 plugins/experimental/ts_lua/ts_lua_util.c       |  280 ++-
 plugins/experimental/ts_lua/ts_lua_util.h       |    8 +
 32 files changed, 4432 insertions(+), 735 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index ecf8804..319a627 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,9 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.0.0
 
+  *) [TS-2723] add new features to ts_lua plugin.
+   Author: Quehan <qu...@taobao.com>
+
   *) [TS-2816] Bump the minor cache version for ATS 5.0 release
 
   *) [TS-2764] Remove when_to_add_no_cache_to_msie_requests configuration.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 8049a53..8e8c329 100644
--- a/doc/reference/plugins/ts_lua.en.rst
+++ b/doc/reference/plugins/ts_lua.en.rst
@@ -21,38 +21,1334 @@ ts-lua Plugin
   under the License.
 
 
-Embed the Power of Lua into TrafficServer.
+This module embeds Lua, via the standard Lua 5.1 interpreter, into Apache Traffic Server. With this module, we can
+implement ATS plugin by writing lua script instead of c code. Lua code executed using this module can be 100%
+non-blocking because the powerful Lua coroutines had been integrated in to ATS event model.
 
 Synopsis
 ========
+**test_hdr.lua**
+
+::
+
+    function send_response()
+        ts.client_response.header['Rhost'] = ts.ctx['rhost']
+        return 0
+    end
+
+    function do_remap()
+        local req_host = ts.client_request.header.Host
+        ts.ctx['rhost'] = string.reverse(req_host)
+        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+        return 0
+    end
+
+**sethost.lua**
+
+::
+
+    local HOSTNAME = ''
+    function __init__(argtb)
+        if (#argtb) < 1 then
+            print(argtb[0], 'hostname parameter required!!')
+            return -1
+        end
+        HOSTNAME = argtb[1]
+    end
+
+    function do_remap()
+        ts.client_request.header['Host'] = HOSTNAME
+        return 0
+    end
+
+
+Installation
+============
+
+This plugin is only built if the configure option
+
+::
+
+    --enable-experimental-plugins
+
+is given at build time.
+
+Configuration
+=============
+
+This module acts as remap plugin of Traffic Server, so we should realize 'do_remap' function in each lua script. We can
+write this in remap.config:
+
+::
+
+    map http://a.tbcdn.cn/ http://inner.tbcdn.cn/ @plugin=/XXX/libtslua.so @pparam=/XXX/test_hdr.lua
+
+Sometimes we want to receive parameters and process them in the script, we should realize '__init__' function in the lua
+script, and we can write this in remap.config:
+
+::
+
+    map http://a.x.cn/ http://b.x.cn/ @plugin=/X/libtslua.so @pparam=/X/sethost.lua @pparam=a.st.cn
+
+This module can also act as a global plugin of Traffic Server. In this case we should provide one of these functions in
+each lua script:
+
+- **'do_global_txn_start'**
+- **'do_global_txn_close'**
+- **'do_global_os_dns'**
+- **'do_global_pre_remap'**
+- **'do_global_post_remap'**
+- **'do_global_read_request'**
+- **'do_global_send_request'**
+- **'do_global_read_response'**
+- **'do_global_send_response'**
+- **'do_global_cache_lookup_complete'**
+- **'do_global_read_cache'**
+- **'do_global_select_alt'**
+
+We can write this in plugin.config:
+
+::
+  
+    tslua.so /etc/trafficserver/script/test_global_hdr.lua
+
+
+TS API for Lua
+==============
+
+Introduction
+------------
+The API is exposed to Lua in the form of one standard packages ``ts``. This package is in the default global scope and
+is always available within lua script. This package can be introduced into Lua like this:
+
+::
+
+    ts.say('Hello World')
+    ts.sleep(10)
+
+`TOP <#ts-lua-plugin>`_
+
+ts.now
+------
+**syntax:** *val = ts.now()*
+
+**context:** global
+
+**description:** This function returns the time since the Epoch (00:00:00 UTC, January 1, 1970), measured in seconds.
+
+Here is an example:
+
+::
+
+    local nt = ts.now()  -- 1395221053
+
+`TOP <#ts-lua-plugin>`_
+
+ts.debug
+--------
+**syntax:** *ts.debug(MESSAGE)*
+
+**context:** global
+
+**description**: Log the MESSAGE to traffic.out if debug is enabled.
+
+Here is an example:
+
+::
+
+       ts.debug('I am in do_remap now.')
+
+The debug tag is **ts_lua** and we should write this in records.config:
+
+``CONFIG proxy.config.diags.debug.tags STRING ts_lua``
+
+`TOP <#ts-lua-plugin>`_
+
+Remap status constants
+----------------------
+**context:** do_remap
+
+::
+
+    TS_LUA_REMAP_NO_REMAP (0)
+    TS_LUA_REMAP_DID_REMAP (1)
+    TS_LUA_REMAP_NO_REMAP_STOP (2)
+    TS_LUA_REMAP_DID_REMAP_STOP (3)
+    TS_LUA_REMAP_ERROR (-1)
+
+These constants are usually used as return value of do_remap function.
+
+`TOP <#ts-lua-plugin>`_
+
+ts.hook
+-------
+**syntax:** *ts.hook(HOOK_POINT, FUNCTION)*
+
+**context:** do_remap or do_global_* or later
+
+**description**: Hooks are points in http transaction processing where we can step in and do some work. FUNCTION will be
+called called when the http transaction steps in to HOOK_POINT.
+
+Here is an example
+
+::
+
+    function send_response()
+        s.client_response.header['SHE'] = 'belief'
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+    end
+
+Then the client will get the response like this:
+
+::
+
+    HTTP/1.1 200 OK
+    Content-Type: text/html
+    Server: ATS/3.2.0
+    SHE: belief
+    Connection: Keep-Alive
+    ...
+
+`TOP <#ts-lua-plugin>`_
+
+Hook point constants
+--------------------
+**context:** do_remap or later
+
+::
+
+    TS_LUA_HOOK_OS_DNS
+    TS_LUA_HOOK_PRE_REMAP
+    TS_LUA_HOOK_READ_CACHE_HDR
+    TS_LUA_HOOK_SELECT_ALT
+    TS_LUA_HOOK_TXN_CLOSE
+    TS_LUA_HOOK_POST_REMAP
+    TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE
+    TS_LUA_HOOK_READ_REQUEST_HDR
+    TS_LUA_HOOK_SEND_REQUEST_HDR
+    TS_LUA_HOOK_READ_RESPONSE_HDR
+    TS_LUA_HOOK_SEND_RESPONSE_HDR
+    TS_LUA_REQUEST_TRANSFORM
+    TS_LUA_RESPONSE_TRANSFORM
+
+These constants are usually used in [ts.hook](#tshook) method call.
+
+`TOP <#ts-lua-plugin>`_
+
+ts.ctx
+------
+**syntax:** *ts.ctx[KEY] = VALUE*
+
+**syntax:** *VALUE = ts.ctx[KEY]*
+
+**context:** do_remap or do_global_* or later
+
+**description:** This table can be used to store per-request Lua context data and has a life time identical to the
+current request.
+
+Here is an example:
+
+::
+
+    function send_response()
+        ts.client_response.header['F-Header'] = ts.ctx['hdr']
+        return 0
+    end
+
+    function do_remap()
+        ts.ctx['hdr'] = 'foo'
+        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+        return 0
+    end
+
+Then the client will get the response like this:
+
+::
+
+    HTTP/1.1 200 OK
+    Content-Type: text/html
+    Server: ATS/3.2.0
+    F-Header: foo
+    Connection: Keep-Alive
+    ...
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.get_method
+----------------------------
+**syntax:** *ts.client_request.get_method()*
+
+**context:** do_remap or do_global_* or later
+
+**description:** This function can be used to retrieve the current client request's method name. String like "GET" or
+"POST" is returned.
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.set_method
+----------------------------
+**syntax:** *ts.client_request.set_method()*
+
+**context:** do_remap or do_global_*
+
+**description:** This function can be used to override the current client request's method with METHOD_NAME.
+
+ts.client_request.get_version
+-----------------------------
+**syntax:** *ver = ts.client_request.get_version()*
+
+**context:** do_remap or do_global_* or later
+
+**description:** Return the http version string of the client request.
+
+Current possible values are 1.0, 1.1, and 0.9.
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.set_version
+-----------------------------
+**syntax:** *ts.client_request.set_version(VERSION_STR)*
+
+**context:** do_remap or do_global_* or later
+
+**description:** Set the http version of the client request with the VERSION_STR
+
+::
+
+    ts.client_request.set_version('1.0')
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.get_uri
+-------------------------
+**syntax:** *ts.client_request.get_uri()*
+
+**context:** do_remap or later
+
+**description:** This function can be used to retrieve the client request's path.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        local uri = ts.client_request.get_uri()
+        print(uri)
+    end
+
+Then ``GET /st?a=1`` will yield the output:
+
+``/st``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.set_uri
+-------------------------
+**syntax:** *ts.client_request.set_uri(PATH)*
+
+**context:** do_remap or do_global_* 
+
+**description:** This function can be used to override the client request's path.
+
+The PATH argument must be a Lua string and starts with ``/``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.get_uri_args
+------------------------------
+**syntax:** *ts.client_request.get_uri_args()*
+
+**context:** do_remap or do_global_* or later
+
+**description:** This function can be used to retrieve the client request's query string.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        local query = ts.client_request.get_uri_args()
+        print(query)
+    end
+
+Then ``GET /st?a=1&b=2`` will yield the output:
+
+``a=1&b=2``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.set_uri_args
+------------------------------
+**syntax:** *ts.client_request.set_uri_args(QUERY_STRING)*
+
+**context:** do_remap or do_global_* 
+
+**description:** This function can be used to override the client request's query string.
+
+::
+
+    ts.client_request.set_uri_args('n=6&p=7')
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.get_url
+-------------------------
+**syntax:** *ts.client_request.get_url()*
+
+**context:** do_remap or do_global_* or later
+
+**description:** This function can be used to retrieve the whole client request's url.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        local url = ts.client_request.get_url()
+        print(url)
+    end
+
+Then ``GET /st?a=1&b=2 HTTP/1.1\r\nHost: a.tbcdn.cn\r\n...`` will yield the output:
+
+``http://a.tbcdn.cn/st?a=1&b=2``
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.header.HEADER
+-------------------------------
+**syntax:** *ts.client_request.header.HEADER = VALUE*
+
+**syntax:** *ts.client_request.header[HEADER] = VALUE*
+
+**syntax:** *VALUE = ts.client_request.header.HEADER*
+
+**context:** do_remap or do_global_* or later
+
+**description:** Set, add to, clear or get the current client request's HEADER.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        local ua = ts.client_request.header['User-Agent']
+        print(ua)
+        ts.client_request.header['Host'] = 'a.tbcdn.cn'
+    end
+
+Then ``GET /st HTTP/1.1\r\nHost: b.tb.cn\r\nUser-Agent: Mozilla/5.0\r\n...`` will yield the output:
+
+``Mozilla/5.0``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.get_headers
+-----------------------------
+**syntax:** *ts.client_request.get_headers()*
+
+**context:** do_remap or do_global_* or later
+
+**description:** Returns a Lua table holding all the headers for the current client request.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        hdrs = ts.client_request.get_headers()
+        for k, v in pairs(hdrs) do
+            print(k..': '..v)
+        end
+    end
+
+Then ``GET /st HTTP/1.1\r\nHost: b.tb.cn\r\nUser-Aget: Mozilla/5.0\r\nAccept: */*`` will yield the output ::
+
+    Host: b.tb.cn
+    User-Agent: Mozilla/5.0
+    Accept: */*
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.client_addr.get_addr
+--------------------------------------
+**syntax:** *ts.client_request.client_addr.get_addr()*
+
+**context:** do_remap or do_global_* or later
+
+**description**: This function can be used to get socket address of the client.
+
+The ts.client_request.client_addr.get_addr function returns three values, ip is a string, port and family is number. 
+
+Here is an example:
+
+::
+
+    function do_remap()
+        ip, port, family = ts.client_request.client_addr.get_addr()
+        print(ip)               -- 192.168.231.17
+        print(port)             -- 17786
+        print(family)           -- 2(AF_INET)
+        return 0
+    end
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.get_url_host
+------------------------------
+**syntax:** *host = ts.client_request.get_url_host()*
+
+**context:** do_remap or do_global_* or later
+
+**description:** Return the ``host`` field of the request url.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        local url_host = ts.client_request.get_url_host()
+        print(url_host)
+    end
+
+Then ``GET /liuyurou.txt HTTP/1.1\r\nHost: 192.168.231.129:8080\r\n...`` will yield the output:
+
+``192.168.231.129``
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.set_url_host
+------------------------------
+**syntax:** *ts.client_request.set_url_host(str)*
+
+**context:** do_remap or do_global_* 
+
+**description:** Set ``host`` field of the request url with ``str``. This function is used to change the address of the
+origin server, and we should return TS_LUA_REMAP_DID_REMAP(_STOP) in do_remap.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        ts.client_request.set_url_host('192.168.231.130')
+        ts.client_request.set_url_port(80)
+        ts.client_request.set_url_scheme('http')
+        return TS_LUA_REMAP_DID_REMAP
+    end
+
+remap.config like this:
+
+::
+
+    map http://192.168.231.129:8080/ http://192.168.231.129:9999/
+
+Then server request will connect to ``192.168.231.130:80``
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.get_url_port
+------------------------------
+**syntax:** *port = ts.client_request.get_url_port()*
+
+**context:** do_remap or do_global_* or later
+
+**description:** Returns the ``port`` field of the request url as a Lua number.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        local url_port = ts.client_request.get_url_port()
+        print(url_port)
+    end
+
+Then Then ``GET /liuyurou.txt HTTP/1.1\r\nHost: 192.168.231.129:8080\r\n...`` will yield the output:
+
+``8080``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.set_url_port
+------------------------------
+**syntax:** *ts.client_request.set_url_port(NUMBER)*
+
+**context:** do_remap or do_global_*
+
+**description:** Set ``port`` field of the request url with ``NUMBER``. This function is used to change the address of
+the origin server, and we should return TS_LUA_REMAP_DID_REMAP(_STOP) in do_remap.
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.get_url_scheme
+--------------------------------
+**syntax:** *scheme = ts.client_request.get_url_scheme()*
+
+**context:** do_remap or do_global_* or later
+
+**description:** Return the ``scheme`` field of the request url.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        local url_scheme = ts.client_request.get_url_scheme()
+        print(url_scheme)
+    end
+
+Then ``GET /liuyurou.txt HTTP/1.1\r\nHost: 192.168.231.129:8080\r\n...`` will yield the output:
+
+``http``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_request.set_url_scheme
+--------------------------------
+**syntax:** *ts.client_request.set_url_scheme(str)*
+
+**context:** do_remap or do_global_* 
+
+**description:** Set ``scheme`` field of the request url with ``str``. This function is used to change the scheme of the
+server request, and we should return TS_LUA_REMAP_DID_REMAP(_STOP) in do_remap.
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.http.set_cache_url
+---------------------
+**syntax:** *ts.http.set_cache_url(KEY_URL)*
+
+**context:** do_remap or do_global_* 
+
+**description:** This function can be used to modify the cache key for the client request.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        ts.http.set_cache_url('http://a.tbcdn.cn/foo')
+        return 0
+    end
+
+`TOP <#ts-lua-plugin>`_
+
+ts.http.set_resp
+----------------
+**syntax:** *ts.http.set_resp(CODE, BODY)*
+
+**context:** do_remap or do_global_*
+
+**description:** This function can be used to set response for the client with the CODE status and BODY string.
+
+Here is an example:
+
+::
+
+    function do_remap()
+        ts.http.set_resp(403, "Document access failed :)\n")
+        return 0
+    end
+
+We will get the response like this:
+
+::
+
+    HTTP/1.1 403 Forbidden
+    Date: Thu, 20 Mar 2014 06:12:43 GMT
+    Connection: close
+    Server: ATS/5.0.0
+    Cache-Control: no-store
+    Content-Type: text/html
+    Content-Language: en
+    Content-Length: 27
+
+    Document access failed :)
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.http.get_cache_lookup_status
+-------------------------------
+**syntax:** *ts.http.get_cache_lookup_status()*
+
+**context:** function @ TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE hook point
+
+**description:** This function can be used to get cache lookup status.
+
+Here is an example:
+
+::
+
+    function cache_lookup()
+        local cache_status = ts.http.get_cache_lookup_status()
+        if cache_status == TS_LUA_CACHE_LOOKUP_HIT_FRESH then
+            print('hit')
+        else
+            print('not hit')
+        end
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE, cache_lookup)
+        return 0
+    end
+
+
+`TOP <#ts-lua-plugin>`_
+
+Http cache lookup status constants
+----------------------------------
+**context:** global
+
+::
+
+    TS_LUA_CACHE_LOOKUP_MISS (0)
+    TS_LUA_CACHE_LOOKUP_HIT_STALE (1)
+    TS_LUA_CACHE_LOOKUP_HIT_FRESH (2)
+    TS_LUA_CACHE_LOOKUP_SKIPPED (3)
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.cached_response.get_status
+-----------------------------
+**syntax:** *status = ts.cached_response.get_status()*
+
+**context:** function @ TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE hook point or later
+
+**description:** This function can be used to retrieve the status code of the cached response. A Lua number will be
+returned.
+
+Here is an example:
+
+::
+
+    function cache_lookup()
+        local cache_status = ts.http.get_cache_lookup_status()
+        if cache_status == TS_LUA_CACHE_LOOKUP_HIT_FRESH then
+            code = ts.cached_response.get_status()
+            print(code)         -- 200
+        end
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE, cache_lookup)
+        return 0
+    end
+
+`TOP <#ts-lua-plugin>`_
+
+ts.cached_response.get_version
+------------------------------
+**syntax:** *ver = ts.cached_response.get_version()*
+
+**context:** function @ TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE hook point or later
+
+**description:** Return the http version string of the cached response.
+
+Current possible values are 1.0, 1.1, and 0.9.
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.cached_response.header.HEADER
+--------------------------------
+**syntax:** *VALUE = ts.cached_response.header.HEADER*
+
+**syntax:** *VALUE = ts.cached_response.header[HEADER]*
+
+**context:** function @ TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE hook point or later
+
+**description:** get the current cached response's HEADER.
+
+Here is an example:
+
+::
+
+    function cache_lookup()
+        local status = ts.http.get_cache_lookup_status()
+        if status == TS_LUA_CACHE_LOOKUP_HIT_FRESH then
+            local ct = ts.cached_response.header['Content-Type']
+            print(ct)         -- text/plain
+        end
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE, cache_lookup)
+        return 0
+    end
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.cached_response.get_headers
+------------------------------
+**syntax:** *ts.cached_response.get_headers()*
+
+**context:** function @ TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE hook point or later
+
+**description:** Returns a Lua table holding all the headers for the current cached response.
+
+Here is an example:
+
+::
+
+    function cache_lookup()
+        local status = ts.http.get_cache_lookup_status()
+        if status == TS_LUA_CACHE_LOOKUP_HIT_FRESH then
+            hdrs = ts.cached_response.get_headers()
+            for k, v in pairs(hdrs) do
+                print(k..': '..v)
+            end
+        end
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE, cache_lookup)
+        return 0
+    end
+
+We will get the output:
+
+::
+
+    Connection: keep-alive
+    Content-Type: text/plain
+    Date: Thu, 20 Mar 2014 06:12:20 GMT
+    Cache-Control: max-age=864000
+    Last-Modified: Sun, 19 May 2013 13:22:01 GMT
+    Accept-Ranges: bytes
+    Content-Length: 15
+    Server: ATS/5.0.0
+
+
+`TOP <#ts-lua-plugin>`_
+
+
+ts.server_request.get_uri
+-------------------------
+**syntax:** *ts.server_request.get_uri()*
+
+**context:** function @ TS_LUA_HOOK_SEND_REQUEST_HDR hook point or later
+
+**description:** This function can be used to retrieve the server request's path.
+
+Here is an example:
+
+::
+
+    function send_request()
+        local uri = ts.server_request.get_uri()
+        print(uri)
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_SEND_REQUEST_HDR, send_request)
+        return 0
+    end
+
+Then ``GET /am.txt?a=1`` will yield the output:
+
+``/am.txt``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.server_request.set_uri
+-------------------------
+**syntax:** *ts.server_request.set_uri(PATH)*
+
+**context:** function @ TS_LUA_HOOK_SEND_REQUEST_HDR hook point
+
+**description:** This function can be used to override the server request's path.
+
+The PATH argument must be a Lua string and starts with ``/``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.server_request.get_uri_args
+------------------------------
+**syntax:** *ts.server_request.get_uri_args()*
+
+**context:** function @ TS_LUA_HOOK_SEND_REQUEST_HDR hook point or later
+
+**description:** This function can be used to retrieve the server request's query string.
+
+Here is an example:
+
+::
+
+    function send_request()
+        local query = ts.server_request.get_uri_args()
+        print(query)
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_SEND_REQUEST_HDR, send_request)
+        return 0
+    end
+
+Then ``GET /st?a=1&b=2`` will yield the output:
+
+``a=1&b=2``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.server_request.set_uri_args
+------------------------------
+**syntax:** *ts.server_request.set_uri_args(QUERY_STRING)*
+
+**context:** function @ TS_LUA_HOOK_SEND_REQUEST_HDR hook point
+
+**description:** This function can be used to override the server request's query string.
+
+::
+
+    ts.server_request.set_uri_args('n=6&p=7')
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.server_request.header.HEADER
+-------------------------------
+**syntax:** *ts.server_request.header.HEADER = VALUE*
+
+**syntax:** *ts.server_request.header[HEADER] = VALUE*
+
+**syntax:** *VALUE = ts.server_request.header.HEADER*
+
+**context:** function @ TS_LUA_HOOK_SEND_REQUEST_HDR hook point or later
+
+**description:** Set, add to, clear or get the current server request's HEADER.
+
+Here is an example:
+
+::
+
+    function send_request()
+        local ua = ts.server_request.header['User-Agent']
+        print(ua)
+        ts.server_request.header['Accept-Encoding'] = 'gzip'
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_SEND_REQUEST_HDR, send_request)
+        return 0
+    end
 
-**test_hdr.lua**
+Then ``GET /st HTTP/1.1\r\nHost: b.tb.cn\r\nUser-Agent: Mozilla/5.0\r\n...`` will yield the output:
+
+``Mozilla/5.0``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.server_request.get_headers
+-----------------------------
+**syntax:** *ts.server_request.get_headers()*
+
+**context:** function @ TS_LUA_HOOK_SEND_REQUEST_HDR hook point or later
+
+**description:** Returns a Lua table holding all the headers for the current server request.
+
+Here is an example:
 
 ::
 
-    function send_response()
-        ts.client_response.header['Rhost'] = ts.ctx['rhost']
+    function send_request()
+        hdrs = ts.cached_response.get_headers()
+        for k, v in pairs(hdrs) do
+            print(k..': '..v)
+        end
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_SEND_REQUEST_HDR, send_request)
         return 0
     end
 
+We will get the output:
+
+::
+
+    Host: b.tb.cn
+    User-Agent: curl/7.19.7
+    Accept: */*
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.server_response.get_status
+-----------------------------
+**syntax:** *status = ts.server_response.get_status()*
+
+**context:** function @ TS_LUA_HOOK_READ_RESPONSE_HDR hook point or later
+
+**description:** This function can be used to retrieve the status code of the origin server's response. A Lua number
+will be returned.
+
+Here is an example:
+
+::
+
+    function read_response()
+        local code = ts.server_response.get_status()
+        print(code)         -- 200
+    end
 
     function do_remap()
-        local req_host = ts.client_request.header.Host
+        ts.hook(TS_LUA_HOOK_READ_RESPONSE_HDR, read_response)
+        return 0
+    end
+
+
+`TOP <#ts-lua-plugin>`_'
+
+ts.server_response.set_status
+-----------------------------
+**syntax:** *ts.server_response.set_status(NUMBER)*
+
+**context:** function @ TS_LUA_HOOK_READ_RESPONSE_HDR hook point
+
+**description:** This function can be used to set the status code of the origin server's response.
+
+Here is an example:
+
+::
+
+    function read_response()
+        ts.server_response.set_status(404)
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_READ_RESPONSE_HDR, read_response)
+        return 0
+    end
+
+
+`TOP <#ts-lua-plugin>`_'
+
+ts.server_response.get_version
+------------------------------
+**syntax:** *ver = ts.server_response.get_version()*
+
+**context:** function @ TS_LUA_HOOK_READ_RESPONSE_HDR hook point or later.
+
+**description:** Return the http version string of the server response.
+
+Current possible values are 1.0, 1.1, and 0.9.
+
+`TOP <#ts-lua-plugin>`_
+
+ts.server_response.set_version
+------------------------------
+**syntax:** *ts.server_response.set_version(VERSION_STR)*
+
+**context:** function @ TS_LUA_HOOK_READ_RESPONSE_HDR hook point
+
+**description:** Set the http version of the server response with the VERSION_STR
+
+::
+
+    ts.server_response.set_version('1.0')
+
+`TOP <#ts-lua-plugin>`_
+
+ts.server_response.header.HEADER
+--------------------------------
+**syntax:** *ts.server_response.header.HEADER = VALUE*
+
+**syntax:** *ts.server_response.header[HEADER] = VALUE*
 
-        if req_host == nil then
-            return 0
+**syntax:** *VALUE = ts.server_response.header.HEADER*
+
+**context:** function @ TS_LUA_HOOK_READ_RESPONSE_HDR hook point or later.
+
+**description:** Set, add to, clear or get the current server response's HEADER.
+
+Here is an example:
+
+::
+
+    function read_response()
+        local ct = ts.server_response.header['Content-Type']
+        print(ct)
+        ts.server_response.header['Cache-Control'] = 'max-age=14400'
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_READ_RESPONSE_HDR, read_response)
+        return 0
+    end
+
+We will get the output:
+
+``text/html``
+
+
+`TOP <#ts-lua-plugin>`_'
+
+ts.server_response.get_headers
+------------------------------
+**syntax:** *ts.server_response.get_headers()*
+
+**context:** function @ TS_LUA_HOOK_READ_RESPONSE_HDR hook point or later
+
+**description:** Returns a Lua table holding all the headers for the current server response.
+
+Here is an example:
+
+::
+
+    function read_response()
+        hdrs = ts.server_response.get_headers()
+        for k, v in pairs(hdrs) do
+            print(k..': '..v)
         end
+    end
 
-        ts.ctx['rhost'] = string.reverse(req_host)
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_READ_RESPONSE_HDR, read_response)
+        return 0
+    end
+
+We will get the output:
+
+::
+
+    Server: nginx/1.5.9
+    Date: Tue, 18 Mar 2014 10:12:25 GMT
+    Content-Type: text/html
+    Content-Length: 555
+    Last-Modified: Mon, 19 Aug 2013 14:25:55 GMT
+    Connection: keep-alive
+    ETag: "52122af3-22b"
+    Cache-Control: max-age=14400
+    Accept-Ranges: bytes
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_response.get_status
+-----------------------------
+**syntax:** *status = ts.client_response.get_status()*
+
+**context:** function @ TS_LUA_HOOK_SEND_RESPONSE_HDR hook point
+
+**description:** This function can be used to retrieve the status code of the response to the client. A Lua number will
+be returned.
+
+Here is an example:
+
+::
+
+    function send_response()
+        local code = ts.client_response.get_status()
+        print(code)         -- 200
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+        return 0
+    end
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_response.set_status
+-----------------------------
+**syntax:** *ts.client_response.set_status(NUMBER)*
+
+**context:** function @ TS_LUA_HOOK_SEND_RESPONSE_HDR hook point
+
+**description:** This function can be used to set the status code of the response to the client.
+
+Here is an example:
+
+::
+
+    function send_response()
+        ts.client_response.set_status(404)
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+        return 0
+    end
 
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_response.get_version
+------------------------------
+**syntax:** *ver = ts.client_response.get_version()*
+
+**context:** function @ TS_LUA_HOOK_SEND_RESPONSE_HDR hook point.
+
+**description:** Return the http version string of the response to the client.
+
+Current possible values are 1.0, 1.1, and 0.9.
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_response.set_version
+------------------------------
+**syntax:** *ts.client_response.set_version(VERSION_STR)*
+
+**context:** function @ TS_LUA_HOOK_SEND_RESPONSE_HDR hook point
+
+**description:** Set the http version of the response to the client with the VERSION_STR
+
+::
+
+    ts.client_response.set_version('1.0')
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_response.header.HEADER
+--------------------------------
+**syntax:** *ts.client_response.header.HEADER = VALUE*
+
+**syntax:** *ts.client_response.header[HEADER] = VALUE*
+
+**syntax:** *VALUE = ts.client_response.header.HEADER*
+
+**context:** function @ TS_LUA_HOOK_SEND_RESPONSE_HDR hook point.
+
+**description:** Set, add to, clear or get the current client response's HEADER.
+
+Here is an example:
+
+::
+
+    function send_response()
+        local ct = ts.client_response.header['Content-Type']
+        print(ct)
+        ts.client_response.header['Cache-Control'] = 'max-age=3600'
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+        return 0
+    end
+
+We will get the output:
+
+``text/html``
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_response.get_headers
+------------------------------
+**syntax:** *ts.client_response.get_headers()*
+
+**context:** function @ TS_LUA_HOOK_SEND_RESPONSE_HDR hook point.
+
+**description:** Returns a Lua table holding all the headers for the current client response.
+
+Here is an example:
+
+::
+
+    function send_response()
+        hdrs = ts.client_response.get_headers()
+        for k, v in pairs(hdrs) do
+            print(k..': '..v)
+        end
+    end
+
+    function do_remap()
         ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+        return 0
+    end
+
+We will get the output:
+
+::
+
+    Server: ATS/5.0.0
+    Date: Tue, 18 Mar 2014 10:12:25 GMT
+    Content-Type: text/html
+    Transfer-Encoding: chunked
+    Last-Modified: Mon, 19 Aug 2013 14:25:55 GMT
+    Connection: keep-alive
+    Cache-Control: max-age=14400
+    Age: 2641
+    Accept-Ranges: bytes
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.client_response.set_error_resp
+---------------------------------
+**syntax:** *ts.client_response.set_error_resp(CODE, BODY)*
+
+**context:** function @ TS_LUA_HOOK_SEND_RESPONSE_HDR hook point.
+
+**description:** This function can be used to set the error response to the client.
+
+With this function we can jump to send error response to the client if exception exists, meanwhile we should return `-1`
+from the function where exception raises.
+
+Here is an example:
+
+::
 
+    function send_response()
+        ts.client_response.set_error_resp(404, 'bad luck :(\n')
+    end
+
+    function cache_lookup()
+        return -1
+    end
+
+    function do_remap()
+        ts.hook(TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE, cache_lookup)
+        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
         return 0
     end
 
+We will get the response like this:
+
+::
+
+    HTTP/1.1 404 Not Found
+    Date: Tue, 18 Mar 2014 11:16:00 GMT
+    Connection: keep-alive
+    Server: ATS/5.0.0
+    Content-Length: 12
+
+    bad luck :(
+
+
+`TOP <#ts-lua-plugin>`_
+
+ts.http.resp_cache_transformed
+------------------------------
+**syntax:** *ts.http.resp_cache_transformed(BOOL)*
 
+**context:** do_remap or do_global_* or later
+
+**description**: This function can be used to tell trafficserver whether to cache the transformed data.
 
-**test_transform.lua**
+Here is an example:
 
 ::
 
@@ -64,569 +1360,656 @@ Synopsis
         end
     end
 
-    function send_response()
-        ts.client_response.header['SHE'] = ts.ctx['tb']['she']
+    function do_remap()
+        ts.hook(TS_LUA_RESPONSE_TRANSFORM, upper_transform)
+        ts.http.resp_cache_transformed(0)
         return 0
     end
 
+This function is usually called after we hook TS_LUA_RESPONSE_TRANSFORM.
 
-    function do_remap()
-        local req_host = ts.client_request.header.Host
 
-        if req_host == nil then
-            return 0
-        end
+`TOP <#ts-lua-plugin>`_
 
-        ts.ctx['tb'] = {}
-        ts.ctx['tb']['she'] = 'wo ai yu ye hua'
+ts.http.resp_cache_untransformed
+--------------------------------
+**syntax:** *ts.http.resp_cache_untransformed(BOOL)*
 
-        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
-        ts.hook(TS_LUA_RESPONSE_TRANSFORM, upper_transform)
+**context:** do_remap or do_global_* or later
 
-        ts.http.resp_cache_transformed(0)
+**description**: This function can be used to tell trafficserver whether to cache the untransformed data.
+
+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
+    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.add_package_path
+-------------------
+**syntax:** *ts.add_package_path(lua-style-path-str)*
+
+**context:** init stage of the lua script
+
+**description:** Adds the Lua module search path used by scripts.
+
+The path string is in standard Lua path form.
+
+Here is an example:
+
+::
+
+    ts.add_package_path('/home/a/test/lua/pac/?.lua')
+    local nt = require("nt")
+    function do_remap()
+        print(nt.t9(7979))
+        return 0
+    end
+
+`TOP <#ts-lua-plugin>`_
 
 
-**test_cache_lookup.lua**
+ts.add_package_cpath
+--------------------
+**syntax:** *ts.add_package_cpath(lua-style-cpath-str)*
 
-::
+**context:** init stage of the lua script
 
-    function send_response()
-        ts.client_response.header['Rhost'] = ts.ctx['rhost']
-        ts.client_response.header['CStatus'] = ts.ctx['cstatus']
-    end
+**description:** Adds the Lua C-module search path used by scripts.
 
+The cpath string is in standard Lua cpath form.
 
-    function cache_lookup()
-        local cache_status = ts.http.get_cache_lookup_status()
-        ts.ctx['cstatus'] = cache_status
-    end
+Here is an example:
 
+::
 
+    ts.add_package_cpath('/home/a/test/c/module/?.so')
+    local ma = require("ma")
     function do_remap()
-        local req_host = ts.client_request.header.Host
+        print(ma.ft())
+        return 0
+    end
 
-        if req_host == nil then
-            return 0
-        end
 
-        ts.ctx['rhost'] = string.reverse(req_host)
+`TOP <#ts-lua-plugin>`_
 
-        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
-        ts.hook(TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE, cache_lookup)
 
-        return 0
-    end
+ts.md5
+------
+**syntax:** *digest = ts.md5(str)*
 
+**context:** global
 
+**description:** Returns the hexadecimal representation of the MD5 digest of the ``str`` argument.
 
-**test_ret_403.lua**
+Here is an example:
 
 ::
 
-    function send_response()
-        ts.client_response.header['Now'] = ts.now()
-        return 0
+    function do_remap()
+        uri = ts.client_request.get_uri()
+        print(uri)              -- /foo
+        print(ts.md5(uri))      -- 1effb2475fcfba4f9e8b8a1dbc8f3caf
     end
 
 
-    function do_remap()
+`TOP <#ts-lua-plugin>`_
 
-        local uri = ts.client_request.get_uri()
+ts.md5_bin
+----------
+**syntax:** *digest = ts.md5_bin(str)*
 
-        pos, len = string.find(uri, '/css/')
-        if pos ~= nil then
-            ts.http.set_resp(403, "Document access failed :)\n")
-            return 0
-        end
+**context:** global
 
-        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+**description:** Returns the binary form of the MD5 digest of the ``str`` argument.
 
-        return 0
+Here is an example:
+
+::
+
+    function do_remap()
+        uri = ts.client_request.get_uri()
+        bin = ts.md5_bin(uri)
     end
 
 
+`TOP <#ts-lua-plugin>`_
 
-**sethost.lua**
+ts.sha1
+-------
+**syntax:** *digest = ts.sha1(str)*
 
-::
+**context:** global
 
-    HOSTNAME = ''
+**description:** Returns the hexadecimal representation of the SHA-1 digest of the ``str`` argument.
 
-    function __init__(argtb)
+Here is an example:
 
-        if (#argtb) < 1 then
-            print(argtb[0], 'hostname parameter required!!')
-            return -1
-        end
+::
 
-        HOSTNAME = argtb[1]
+    function do_remap()
+        uri = ts.client_request.get_uri()
+        print(uri)              -- /foo
+        print(ts.sha1(uri))     -- 6dbd548cc03e44b8b44b6e68e56255ce4273ae49
     end
 
-    function do_remap()
-        local req_host = ts.client_request.header.Host
 
-        if req_host == nil then
-            return 0
-        end
+`TOP <#ts-lua-plugin>`_
 
-        ts.client_request.header['Host'] = HOSTNAME
+ts.sha1_bin
+-----------
+**syntax:** *digest = ts.sha1_bin(str)*
 
-        return 0
-    end
+**context:** global
 
+**description:** Returns the binary form of the SHA-1 digest of the ``str`` argument.
 
-**test_intercept.lua**
+Here is an example:
 
 ::
 
-    require 'os'
+    function do_remap()
+        uri = ts.client_request.get_uri()
+        bin = ts.sha1_bin(uri)
+    end
 
-    function send_data()
-        local nt = os.time()..' Zheng.\n'
-        local resp =  'HTTP/1.1 200 OK\r\n' ..
-            'Server: ATS/3.2.0\r\n' ..
-            'Content-Type: text/plain\r\n' ..
-            'Content-Length: ' .. 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.sleep(1)
-        return resp
-    end
+`TOP <#ts-lua-plugin>`_
 
-    function do_remap()
-        ts.http.intercept(send_data)
-        return 0
-    end
+ts.intercept
+------------
+**syntax:** *ts.intercept(FUNCTION)*
 
+**context:** do_remap or do_global_*
 
-**test_server_intercept.lua**
+**description:** Intercepts the client request and processes it in FUNCTION.
+
+We should construct the response for the client request, and the request will not be processed by other modules, like
+hostdb, cache, origin server...
+
+Here is an example:
 
 ::
 
     require 'os'
 
     function send_data()
-        local nt = os.time()..'\n'
-        local resp =  'HTTP/1.1 200 OK\r\n' ..
-            'Server: ATS/3.2.0\r\n' ..
-            'Content-Type: text/plain\r\n' ..
-            'Content-Length: ' .. 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=30\r\n' ..
-            'Accept-Ranges: bytes\r\n\r\n' ..
-            nt
-        return resp
+        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)
     end
 
     function do_remap()
-        ts.http.server_intercept(send_data)
+        ts.http.intercept(send_data)
         return 0
     end
 
-**test_global_hdr.lua**
+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: 0
+    Connection: keep-alive
 
-    function send_response()
-        ts.client_response.header['Rhost'] = ts.ctx['rhost']
-        return 0
-    end
+    1395145392 Zheng.
 
-    function do_global_read_request()
-        local req_host = ts.client_request.header.Host
 
-        if req_host == nil then
-            return 0
-        end
+`TOP <#ts-lua-plugin>`_
 
-        ts.ctx['rhost'] = string.reverse(req_host)
+ts.say
+------
+**syntax:** *ts.say(data)*
 
-        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+**context:** *intercept or server_intercept*
 
-        return 0
-    end
+**description:** Write response to ATS within intercept or server_intercept.
 
 
-Description
-===========
+`TOP <#ts-lua-plugin>`_
 
-This module embeds Lua, into Apache Traffic Server. This module acts as remap plugin of Traffic Server. In this case we
-should provide **'do_remap'** function in each lua script. We can write this in remap.config::
+ts.flush
+--------
+**syntax:** *ts.flush()*
 
-     map http://a.tbcdn.cn/ http://inner.tbcdn.cn/ @plugin=/usr/lib64/trafficserver/plugins/tslua.so \
-       @pparam=/etc/trafficserver/script/test_hdr.lua
+**context:** *intercept or server_intercept*
 
-Sometimes we want to receive parameters and process them in the script, we should realize **'\__init__'** function in
-the lua script(sethost.lua is a reference), and we can write this in remap.config::
+**description:** Flushes the output to ATS within intercept or server_intercept.
 
-     map http://a.tbcdn.cn/ http://inner.tbcdn.cn/ @plugin=/usr/lib64/trafficserver/plugins/tslua.so \
-       @pparam=/etc/trafficserver/script/sethost.lua @pparam=img03.tbcdn.cn
+In synchronous mode, the function will not return until all output data has been written into the system send buffer.
+Note that using the Lua coroutine mechanism means that this function does not block the ATS event loop even in the
+synchronous mode.
 
-This module can also act as a global plugin of Traffic Server. In this case we should provide one of these functions in
-each lua script:
+Here is an example:
 
-- **'do_global_txn_start'**
-- **'do_global_txn_close'**
-- **'do_global_os_dns'**
-- **'do_global_pre_remap'**
-- **'do_global_post_remap'**
-- **'do_global_read_request'**
-- **'do_global_send_request'**
-- **'do_global_read_response'**
-- **'do_global_send_response'**
-- **'do_global_cache_lookup_complete'**
-- **'do_global_read_cache'**
-- **'do_global_select_alt'**
+::
 
-We can write this in plugin.config::
+    require 'os'
+
+    function send_data()
+        ss = 'wo ai yu ye hua\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', 5*string.len(ss)) .. '\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'
+        ts.say(resp)
+        for i=1, 5 do
+            ts.say(ss)
+            ts.flush()
+        end
+    end
 
-     tslua.so /etc/trafficserver/script/test_global_hdr.lua
+    function do_remap()
+        ts.http.intercept(send_data)
+        return 0
+    end
 
+We will get the response like this:
 
-TS API for Lua
-==============
+::
 
-Introduction
-------------
+    HTTP/1.1 200 OK
+    Server: ATS/5.0.0
+    Content-Type: text/plain
+    Content-Length: 80
+    Last-Modified: Tue, 18 Mar 2014 08:38:29 GMT
+    Cache-Control: max-age=7200
+    Accept-Ranges: bytes
+    Date: Tue, 18 Mar 2014 12:38:29 GMT
+    Age: 0
+    Connection: keep-alive
+
+    wo ai yu ye hua
+    wo ai yu ye hua
+    wo ai yu ye hua
+    wo ai yu ye hua
+    wo ai yu ye hua
+
+`TOP <#ts-lua-plugin>`_
+
+ts.sleep
+--------
+**syntax:** *ts.sleep(sec)*
 
-The API is exposed to Lua in the form of one standard packages ts. This package is in the default global scope and is
-always available within lua script.
+**context:** *intercept or server_intercept*
 
+**description:** Sleeps for the specified seconds without blocking.
 
+Behind the scene, this method makes use of the ATS event model.
 
-ts.now
-------
-**syntax**: *val = ts.now()*
+Here is an example:
 
-**context**: global
+::
 
-**description**: This function returns the time since the Epoch (00:00:00 UTC, January 1, 1970), measured in seconds.
+    require 'os'
 
-Here is an example:::
+    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
+        ts.sleep(3)
+        ts.say(resp)
+    end
 
-    function send_response()
-        ts.client_response.header['Now'] = ts.now()
+    function do_remap()
+        ts.http.intercept(send_data)
         return 0
     end
 
+`TOP <#ts-lua-plugin>`_
 
-ts.debug
---------
-**syntax**: *ts.debug(MESSAGE)*
+ts.server_intercept
+-------------------
+**syntax:** *ts.server_intercept(FUNCTION)*
 
-**context**: global
+**context:** do_remap or do_global_*
 
-**description**: Log the MESSAGE to traffic.out if debug is enabled.
+**description:** Intercepts the server request and acts as the origin server.
 
-Here is an example:::
+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)
+    end
 
     function do_remap()
-       ts.debug('I am in do_remap now.')
-       return 0
+        ts.http.server_intercept(send_data)
+        return 0
     end
-    
-The debug tag is ts_lua and we should write this in records.config:::
-    
-    CONFIG proxy.config.diags.debug.tags STRING ts_lua
-    
 
-ts.hook
--------
-**syntax**: *ts.hook(HOOK_POINT, FUNCTION)*
+Then we will get the response like this:
 
-**context**: do_remap/do_global_*/later
+::
 
-**description**: Hooks are points in http transaction processing where we can step in and do some work.
-FUNCTION will be called when the http transaction steps in to HOOK_POINT.
+    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
 
-Here is an example:::
+    1395145392 Zheng.
 
-    function send_response()
-        s.client_response.header['SHE'] = 'belief'
-    end
-    
-    function do_remap()
-        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
-    end
 
-Hook point constants
---------------------
-**context**: do_remap/do_global_*/later
-
-- TS_LUA_HOOK_OS_DNS
-- TS_LUA_HOOK_PRE_REMAP
-- TS_LUA_HOOK_POST_REMAP
-- TS_LUA_HOOK_READ_CACHE_HDR
-- TS_LUA_HOOK_SELECT_ALT
-- TS_LUA_HOOK_TXN_CLOSE
-- TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE
-- TS_LUA_HOOK_READ_REQUEST_HDR
-- TS_LUA_HOOK_SEND_REQUEST_HDR
-- TS_LUA_HOOK_READ_RESPONSE_HDR
-- TS_LUA_HOOK_SEND_RESPONSE_HDR
-- TS_LUA_REQUEST_TRANSFORM
-- TS_LUA_RESPONSE_TRANSFORM
-    
-These constants are usually used in ts.hook method call.
+`TOP <#ts-lua-plugin>`_
 
+ts.http.config_int_get
+----------------------
+**syntax:** *val = ts.http.config_int_get(CONFIG)*
 
-ts.ctx
-------
-**syntax**: *ts.ctx[KEY]*
+**context:** do_remap or do_global_* or later.
 
-**context**: do_remap/do_global_*/later
+**description:** Configuration option which has a int value can be retrieved with this function.
 
-**description**: This table can be used to store per-request Lua context data and has a life time identical to the current request.
+::
 
-Here is an example:::
+    val = ts.http.config_int_get(TS_LUA_CONFIG_HTTP_CACHE_HTTP)
 
-    function send_response()
-        ts.client_response.header['RR'] = ts.ctx['rhost']
-        return 0
-    end
-    
-    function do_remap()
-        local req_host = ts.client_request.header.Host
-        ts.ctx['rhost'] = string.reverse(req_host)
-        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
-        return 0
-    end
 
+`TOP <#ts-lua-plugin>`_
 
-ts.http.get_cache_lookup_status
--------------------------------
-**syntax**: *ts.http.get_cache_lookup_status()*
+ts.http.config_int_set
+----------------------
+**syntax:** *ts.http.config_int_set(CONFIG, NUMBER)*
 
-**context**: function @ TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE hook point
+**context:** do_remap or do_global_* or later.
 
-**description**: This function can be used to get cache lookup status.
+**description:** This function can be used to overwrite the configuration options.
 
-Here is an example:::
+Here is an example:
+
+::
 
-    function send_response()
-        ts.client_response.header['CStatus'] = ts.ctx['cstatus']
-    end
-    
-    function cache_lookup()
-        local cache_status = ts.http.get_cache_lookup_status()
-        if cache_status == TS_LUA_CACHE_LOOKUP_HIT_FRESH:
-            ts.ctx['cstatus'] = 'hit'
-        else
-            ts.ctx['cstatus'] = 'not hit'
-        end
-    end
-    
     function do_remap()
-        ts.hook(TS_LUA_HOOK_CACHE_LOOKUP_COMPLETE, cache_lookup)
-        ts.hook(TS_LUA_HOOK_SEND_RESPONSE_HDR, send_response)
+        ts.http.config_int_set(TS_LUA_CONFIG_HTTP_CACHE_HTTP, 0)    -- bypass the cache processor
         return 0
     end
 
 
-Http cache lookup status constants
-----------------------------------
-**context**: global
-
-- TS_LUA_CACHE_LOOKUP_MISS (0)
-- TS_LUA_CACHE_LOOKUP_HIT_STALE (1)
-- TS_LUA_CACHE_LOOKUP_HIT_FRESH (2)
-- TS_LUA_CACHE_LOOKUP_SKIPPED (3)
-
+`TOP <#ts-lua-plugin>`_
 
-ts.http.set_cache_url
----------------------
-**syntax**: *ts.http.set_cache_url(KEY_URL)*
+ts.http.config_float_get
+------------------------
+**syntax:** *val = ts.http.config_float_get(CONFIG)*
 
-**context**: do_remap
+**context:** do_remap or do_global_* or later.
 
-**description**: This function can be used to modify the cache key for the request.
+**description:** Configuration option which has a float value can be retrieved with this function.
 
-Here is an example:::
 
-    function do_remap()
-        ts.http.set_cache_url('http://127.0.0.1:8080/abc/')
-        return 0
-    end
+`TOP <#ts-lua-plugin>`_
 
+ts.http.config_float_set
+------------------------
+**syntax:** *ts.http.config_float_set(CONFIG, NUMBER)*
 
-ts.http.resp_cache_transformed
-------------------------------
-**syntax**: *ts.http.resp_cache_transformed(BOOL)*
+**context:** do_remap or do_global_* or later.
 
-**context**: do_remap/do_global_*/later
+**description:** This function can be used to overwrite the configuration options.
 
-**description**: This function can be used to tell trafficserver whether to cache the transformed data.
 
-Here is an example:::
+`TOP <#ts-lua-plugin>`_
 
-    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
-    
-    function do_remap()
-        ts.hook(TS_LUA_RESPONSE_TRANSFORM, upper_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.
+ts.http.config_string_get
+-------------------------
+**syntax:** *val = ts.http.config_string_get(CONFIG)*
 
+**context:** do_remap or do_global_* or later.
 
-ts.http.resp_cache_untransformed
---------------------------------
-**syntax**: *ts.http.resp_cache_untransformed(BOOL)*
+**description:** Configuration option which has a string value can be retrieved with this function.
 
-**context**: do_remap/do_global_*/later
 
-**description**: This function can be used to tell trafficserver whether to cache the untransformed data.
+`TOP <#ts-lua-plugin>`_
 
-Here is an example:::
+ts.http.config_string_set
+-------------------------
+**syntax:** *ts.http.config_string_set(CONFIG, NUMBER)*
 
-    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
-    
-    function do_remap()
-        ts.hook(TS_LUA_RESPONSE_TRANSFORM, upper_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.
+**context:** do_remap or do_global_* or later.
 
+**description:** This function can be used to overwrite the configuration options.
 
-ts.client_request.client_addr.get_addr
---------------------------------------
-**syntax**: *ts.client_request.client_addr.get_addr()*
 
-**context**: do_remap/do_global_*/later
+`TOP <#ts-lua-plugin>`_
 
-**description**: This function can be used to get socket address of the client.
+Http config constants
+---------------------
+**context:** do_remap or do_global_* or later
 
-Here is an example:::
+::
 
-    function do_remap
-        ip, port, family = ts.client_request.client_addr.get_addr()
-        return 0
-    end
+    TS_LUA_CONFIG_URL_REMAP_PRISTINE_HOST_HDR
+    TS_LUA_CONFIG_HTTP_CHUNKING_ENABLED
+    TS_LUA_CONFIG_HTTP_NEGATIVE_CACHING_ENABLED
+    TS_LUA_CONFIG_HTTP_NEGATIVE_CACHING_LIFETIME
+    TS_LUA_CONFIG_HTTP_CACHE_WHEN_TO_REVALIDATE
+    TS_LUA_CONFIG_HTTP_KEEP_ALIVE_ENABLED_IN
+    TS_LUA_CONFIG_HTTP_KEEP_ALIVE_ENABLED_OUT
+    TS_LUA_CONFIG_HTTP_KEEP_ALIVE_POST_OUT
+    TS_LUA_CONFIG_HTTP_SHARE_SERVER_SESSIONS
+    TS_LUA_CONFIG_NET_SOCK_RECV_BUFFER_SIZE_OUT
+    TS_LUA_CONFIG_NET_SOCK_SEND_BUFFER_SIZE_OUT
+    TS_LUA_CONFIG_NET_SOCK_OPTION_FLAG_OUT
+    TS_LUA_CONFIG_HTTP_FORWARD_PROXY_AUTH_TO_PARENT
+    TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_FROM
+    TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_REFERER
+    TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_USER_AGENT
+    TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_COOKIE
+    TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_CLIENT_IP
+    TS_LUA_CONFIG_HTTP_ANONYMIZE_INSERT_CLIENT_IP
+    TS_LUA_CONFIG_HTTP_RESPONSE_SERVER_ENABLED
+    TS_LUA_CONFIG_HTTP_INSERT_SQUID_X_FORWARDED_FOR
+    TS_LUA_CONFIG_HTTP_SERVER_TCP_INIT_CWND
+    TS_LUA_CONFIG_HTTP_SEND_HTTP11_REQUESTS
+    TS_LUA_CONFIG_HTTP_CACHE_HTTP
+    TS_LUA_CONFIG_HTTP_CACHE_CLUSTER_CACHE_LOCAL
+    TS_LUA_CONFIG_HTTP_CACHE_IGNORE_CLIENT_NO_CACHE
+    TS_LUA_CONFIG_HTTP_CACHE_IGNORE_CLIENT_CC_MAX_AGE
+    TS_LUA_CONFIG_HTTP_CACHE_IMS_ON_CLIENT_NO_CACHE
+    TS_LUA_CONFIG_HTTP_CACHE_IGNORE_SERVER_NO_CACHE
+    TS_LUA_CONFIG_HTTP_CACHE_CACHE_RESPONSES_TO_COOKIES
+    TS_LUA_CONFIG_HTTP_CACHE_IGNORE_AUTHENTICATION
+    TS_LUA_CONFIG_HTTP_CACHE_CACHE_URLS_THAT_LOOK_DYNAMIC
+    TS_LUA_CONFIG_HTTP_CACHE_REQUIRED_HEADERS
+    TS_LUA_CONFIG_HTTP_INSERT_REQUEST_VIA_STR
+    TS_LUA_CONFIG_HTTP_INSERT_RESPONSE_VIA_STR
+    TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_MIN_LIFETIME
+    TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_MAX_LIFETIME
+    TS_LUA_CONFIG_HTTP_CACHE_GUARANTEED_MIN_LIFETIME
+    TS_LUA_CONFIG_HTTP_CACHE_GUARANTEED_MAX_LIFETIME
+    TS_LUA_CONFIG_HTTP_CACHE_MAX_STALE_AGE
+    TS_LUA_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_IN
+    TS_LUA_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_OUT
+    TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN
+    TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT
+    TS_LUA_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT
+    TS_LUA_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS
+    TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES
+    TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER
+    TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES
+    TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_TIMEOUT
+    TS_LUA_CONFIG_HTTP_POST_CONNECT_ATTEMPTS_TIMEOUT
+    TS_LUA_CONFIG_HTTP_DOWN_SERVER_CACHE_TIME
+    TS_LUA_CONFIG_HTTP_DOWN_SERVER_ABORT_THRESHOLD
+    TS_LUA_CONFIG_HTTP_CACHE_FUZZ_TIME
+    TS_LUA_CONFIG_HTTP_CACHE_FUZZ_MIN_TIME
+    TS_LUA_CONFIG_HTTP_DOC_IN_CACHE_SKIP_DNS
+    TS_LUA_CONFIG_HTTP_RESPONSE_SERVER_STR
+    TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_LM_FACTOR
+    TS_LUA_CONFIG_HTTP_CACHE_FUZZ_PROBABILITY
+    TS_LUA_CONFIG_NET_SOCK_PACKET_MARK_OUT
+    TS_LUA_CONFIG_NET_SOCK_PACKET_TOS_OUT
+
+`TOP <#ts-lua-plugin>`_
+
+ts.http.cntl_get
+----------------
+**syntax:** *val = ts.http.cntl_get(CNTL_TYPE)*
+
+**context:** do_remap or do_global_* or later.
+
+**description:** This function can be used to retireve the value of control channel.
 
-The ts.client_request.client_addr.get_addr function returns three values, ip is a string, port and family is number.
+::
 
+    val = ts.http.cntl_get(TS_LUA_HTTP_CNTL_GET_LOGGING_MODE)
 
-ts.client_request.get_method
-----------------------------
-**syntax**: *ts.client_request.get_method()*
 
-**context**: do_remap/do_global_*/later
+`TOP <#ts-lua-plugin>`_
 
-**description**: This function can be used to retrieve the current request's request method name. String like "GET" or 
-"POST" is returned.
+ts.http.cntl_set
+----------------
+**syntax:** *ts.http.cntl_set(CNTL_TYPE, BOOL)*
 
+**context:** do_remap or do_global_* or later.
 
-ts.client_request.set_method
-----------------------------
-**syntax**: *ts.client_request.set_method(METHOD_NAME)*
+**description:** This function can be used to set the value of control channel.
 
-**context**: do_remap
+Here is an example:
 
-**description**: This function can be used to override the current request's request method with METHOD_NAME.
+::
 
+    function do_remap()
+        ts.http.cntl_set(TS_LUA_HTTP_CNTL_SET_LOGGING_MODE, 0)      -- do not log the request
+        return 0
+    end
 
-ts.client_request.get_url
--------------------------
-**syntax**: *ts.client_request.get_url()*
 
-**context**: do_remap/do_global_*/later
+`TOP <#ts-lua-plugin>`_
 
-**description**: This function can be used to retrieve the whole request's url.
+Http control channel constants
+------------------------------
+**context:** do_remap or do_global_* or later
 
+::
 
-ts.client_request.get_uri
--------------------------
-**syntax**: *ts.client_request.get_uri()*
+    TS_LUA_HTTP_CNTL_GET_LOGGING_MODE
+    TS_LUA_HTTP_CNTL_SET_LOGGING_MODE
+    TS_LUA_HTTP_CNTL_GET_INTERCEPT_RETRY_MODE
+    TS_LUA_HTTP_CNTL_SET_INTERCEPT_RETRY_MODE
 
-**context**: do_remap/do_global_*/later
+`TOP <#ts-lua-plugin>`_
 
-**description**: This function can be used to retrieve the request's path.
+ts.mgmt.get_counter
+-------------------
+**syntax:** *val = ts.mgmt.get_counter(RECORD_NAME)*
 
+**context:** do_remap or do_global_* or later.
 
-ts.client_request.set_uri
--------------------------
-**syntax**: *ts.client_request.set_uri(PATH)*
+**description:** This function can be used to retrieve the record value which has a counter type.
 
-**context**: do_remap
+::
 
-**description**: This function can be used to override the request's path.
+    n = ts.mgmt.get_counter('proxy.process.http.incoming_requests')
 
+`TOP <#ts-lua-plugin>`_
 
-ts.client_request.get_uri_args
-------------------------------
-**syntax**: *ts.client_request.get_uri_args()*
+ts.mgmt.get_int
+---------------
+**syntax:** *val = ts.mgmt.get_int(RECORD_NAME)*
 
-**context**: do_remap/do_global_*/later
+**context:** do_remap or do_global_* or later.
 
-**description**: This function can be used to retrieve the request's query string.
+**description:** This function can be used to retrieve the record value which has a int type.
 
+`TOP <#ts-lua-plugin>`_
 
-ts.client_request.set_uri_args
-------------------------------
-**syntax**: *ts.client_request.set_uri_args(QUERY_STRING)*
+ts.mgmt.get_float
+-----------------
+**syntax:** *val = ts.mgmt.get_float(RECORD_NAME)*
 
-**context**: do_remap
+**context:** do_remap or do_global_* or later.
 
-**description**: This function can be used to override the request's query string.
+**description:** This function can be used to retrieve the record value which has a float type.
 
+`TOP <#ts-lua-plugin>`_
 
-ts.client_request.header.HEADER
--------------------------------
-**syntax**: *ts.client_request.header.HEADER = VALUE*
+ts.mgmt.get_string
+------------------
+**syntax:** *val = ts.mgmt.get_string(RECORD_NAME)*
 
-**syntax**: *ts.client_request.header[HEADER] = VALUE*
+**context:** do_remap or do_global_* or later.
 
-**syntax**: *VALUE = ts.client_request.header.HEADER*
+**description:** This function can be used to retrieve the record value which has a string type.
 
-**context**: do_remap/do_global_*/later
+::
 
-**description**: Set, add to, clear or get the current request's HEADER.
+    name = ts.mgmt.get_string('proxy.config.product_name')
 
-Here is an example:::
+`TOP <#ts-lua-plugin>`_
 
-    function do_remap()
-        local req_host = ts.client_request.header.Host
-        ts.client_request.header['Host'] = 'a.tbcdn.cn'
-    end
+Todo
+====
+* ts.fetch
+* ts.cache_xxx
+* `support lua-5.2 <https://github.com/portl4t/ts-lua/wiki/support-Lua-5.2>`_
 
+`TOP <#ts-lua-plugin>`_
 
-TODO
-====
+More docs
+=========
 
-Short Term
-----------
-* document configuration
-* non-blocking I/O operation
-* ts.fetch
+* https://github.com/portl4t/ts-lua
 
-Long Term
----------
-* ts.regex
+`TOP <#ts-lua-plugin>`_
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/Makefile.am
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/Makefile.am b/plugins/experimental/ts_lua/Makefile.am
index 4a696f6..06cc418 100644
--- a/plugins/experimental/ts_lua/Makefile.am
+++ b/plugins/experimental/ts_lua/Makefile.am
@@ -39,6 +39,12 @@ tslua_la_SOURCES = \
   ts_lua_server_response.c \
   ts_lua_transform.c \
   ts_lua_util.c \
-  ts_lua_remap.c
+  ts_lua_remap.c \
+  ts_lua_http_cntl.c \
+  ts_lua_http_config.c \
+  ts_lua_mgmt.c \
+  ts_lua_package.c \
+  ts_lua_string.c \
+  ts_lua_crypto.c
 
 tslua_la_LDFLAGS = $(TS_PLUGIN_LDFLAGS)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 51703b7..cde8902 100644
--- a/plugins/experimental/ts_lua/ts_lua.c
+++ b/plugins/experimental/ts_lua/ts_lua.c
@@ -23,22 +23,27 @@
 
 #include "ts_lua_util.h"
 
-#define TS_LUA_MAX_STATE_COUNT                  2048
+#define TS_LUA_MAX_STATE_COUNT                  512
 
-static volatile int32_t ts_lua_http_next_id = 0;
-static volatile int32_t ts_lua_g_http_next_id = 0;
+static uint64_t ts_lua_http_next_id = 0;
+static uint64_t ts_lua_g_http_next_id = 0;
 
 static ts_lua_main_ctx *ts_lua_main_ctx_array;
 static ts_lua_main_ctx *ts_lua_g_main_ctx_array;
 
 
 TSReturnCode
-TSRemapInit(TSRemapInterface * api_info, char *errbuf ATS_UNUSED, int errbuf_size ATS_UNUSED)
+TSRemapInit(TSRemapInterface * api_info, char *errbuf, int errbuf_size)
 {
   int ret;
 
-  if (!api_info || api_info->size < sizeof(TSRemapInterface))
+  if (!api_info || api_info->size < sizeof(TSRemapInterface)) {
+    strncpy(errbuf, "[TSRemapInit] - Incorrect size of TSRemapInterface structure", errbuf_size - 1);
     return TS_ERROR;
+  }
+
+  if (ts_lua_main_ctx_array != NULL)
+    return TS_SUCCESS;
 
   ts_lua_main_ctx_array = TSmalloc(sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
   memset(ts_lua_main_ctx_array, 0, sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
@@ -55,30 +60,45 @@ TSRemapInit(TSRemapInterface * api_info, char *errbuf ATS_UNUSED, int errbuf_siz
 }
 
 TSReturnCode
-TSRemapNewInstance(int argc, char *argv[], void **ih, char *errbuf ATS_UNUSED, int errbuf_size ATS_UNUSED)
+TSRemapNewInstance(int argc, char *argv[], void **ih, char *errbuf, int errbuf_size)
 {
-  int ret = 0;
+  int fn;
+  int ret;
 
   if (argc < 3) {
-    fprintf(stderr, "[%s] lua script file required !!", __FUNCTION__);
+    strncpy(errbuf, "[TSRemapNewInstance] - lua script file or string is required !!", errbuf_size - 1);
     return TS_ERROR;
   }
 
-  if (strlen(argv[2]) >= TS_LUA_MAX_SCRIPT_FNAME_LENGTH - 16)
+  fn = 1;
+
+  if (argv[2][0] != '/') {
+    fn = 0;
+  } else if (strlen(argv[2]) >= TS_LUA_MAX_SCRIPT_FNAME_LENGTH - 16) {
     return TS_ERROR;
+  }
 
   ts_lua_instance_conf *conf = TSmalloc(sizeof(ts_lua_instance_conf));
   if (!conf) {
-    fprintf(stderr, "[%s] TSmalloc failed !!", __FUNCTION__);
+    fprintf(stderr, "[%s] TSmalloc failed !!\n", __FUNCTION__);
     return TS_ERROR;
   }
 
-  sprintf(conf->script, "%s", argv[2]);
+  memset(conf, 0, sizeof(ts_lua_instance_conf));
+
+  if (fn) {
+    sprintf(conf->script, "%s", argv[2]);
+
+  } else {
+    conf->content = argv[2];
+  }
+
+  ts_lua_init_instance(conf);
 
   ret = ts_lua_add_module(conf, ts_lua_main_ctx_array, TS_LUA_MAX_STATE_COUNT, argc - 2, &argv[2]);
 
   if (ret != 0) {
-    fprintf(stderr, "[%s] ts_lua_add_module failed", __FUNCTION__);
+    fprintf(stderr, "[%s] ts_lua_add_module failed\n", __FUNCTION__);
     return TS_ERROR;
   }
 
@@ -90,6 +110,8 @@ TSRemapNewInstance(int argc, char *argv[], void **ih, char *errbuf ATS_UNUSED, i
 void
 TSRemapDeleteInstance(void *ih)
 {
+  ts_lua_del_module((ts_lua_instance_conf *) ih, ts_lua_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
+  ts_lua_del_instance(ih);
   TSfree(ih);
   return;
 }
@@ -98,7 +120,7 @@ TSRemapStatus
 TSRemapDoRemap(void *ih, TSHttpTxn rh, TSRemapRequestInfo * rri)
 {
   int ret;
-  int64_t req_id;
+  uint64_t req_id;
 
   TSCont contp;
   lua_State *l;
@@ -109,7 +131,8 @@ TSRemapDoRemap(void *ih, TSHttpTxn rh, TSRemapRequestInfo * rri)
   ts_lua_instance_conf *instance_conf;
 
   instance_conf = (ts_lua_instance_conf *) ih;
-  req_id = (int64_t) ts_lua_atomic_increment((&ts_lua_http_next_id), 1);
+  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];
 
@@ -163,7 +186,7 @@ globalHookHandler(TSCont contp, TSEvent event, void *edata)
   TSMLoc hdr_loc;
   TSMLoc url_loc;
 
-  if(!http_ctx->client_request_bufp) {
+  if (!http_ctx->client_request_bufp) {
     if (TSHttpTxnClientReqGet(txnp, &bufp, &hdr_loc) == TS_SUCCESS) {
       http_ctx->client_request_bufp = bufp;
       http_ctx->client_request_hdrp = hdr_loc;
@@ -258,7 +281,7 @@ transactionStartHookHandler(TSCont contp, TSEvent event ATS_UNUSED, void *edata)
 {
   TSHttpTxn txnp = (TSHttpTxn) edata;
 
-  int64_t req_id;
+  uint64_t req_id;
   TSCont txn_contp;
   TSCont global_contp;
 
@@ -267,7 +290,8 @@ transactionStartHookHandler(TSCont contp, TSEvent event ATS_UNUSED, void *edata)
 
   ts_lua_instance_conf *conf = (ts_lua_instance_conf *) TSContDataGet(contp);
 
-  req_id = (int64_t) ts_lua_atomic_increment((&ts_lua_g_http_next_id), 1);
+  req_id = __sync_fetch_and_add(&ts_lua_g_http_next_id, 1);
+
   main_ctx = &ts_lua_g_main_ctx_array[req_id % TS_LUA_MAX_STATE_COUNT];
 
   TSDebug(TS_LUA_DEBUG_TAG, "[%s] req_id: %" PRId64, __FUNCTION__, req_id);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_atomic.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_atomic.c b/plugins/experimental/ts_lua/ts_lua_atomic.c
deleted file mode 100644
index 12f7266..0000000
--- a/plugins/experimental/ts_lua/ts_lua_atomic.c
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
-  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_atomic.h"
-
-#define INK_MEMORY_BARRIER
-
-#define INK_QUEUE_LD64(dst,src) *((uint64_t*)&(dst)) = *((uint64_t*)&(src))
-#define ADDRESS_OF_NEXT(x, offset) ((void **)((char *)x + offset))
-
-#define FROM_PTR(_x) ((void*)(_x))
-#define TO_PTR(_x) ((void*)(_x))
-#define FREELIST_VERSION(_x) (((intptr_t)(_x))>>48)
-#define FREELIST_POINTER(_x) ((void*)(((((intptr_t)(_x))<<16)>>16) | \
-            (((~((((intptr_t)(_x))<<16>>63)-1))>>48)<<48)))
-#define SET_FREELIST_POINTER_VERSION(_x,_p,_v) \
-    _x = ((((intptr_t)(_p))&0x0000FFFFFFFFFFFFULL) | (((_v)&0xFFFFULL) << 48))
-
-
-static inline int64_t ts_lua_atomic_cas64(volatile int64_t * mem, int64_t old, int64_t new_value);
-
-
-void
-ts_lua_atomiclist_init(ts_lua_atomiclist * l, const char *name, uint32_t offset_to_next)
-{
-  l->name = name;
-  l->offset = offset_to_next;
-
-  SET_FREELIST_POINTER_VERSION(l->head, FROM_PTR(0), 0);
-}
-
-void *
-ts_lua_atomiclist_push(ts_lua_atomiclist * l, void *item)
-{
-  int64_t head;
-  int64_t item_pair;
-  int result = 0;
-  volatile void *h = NULL;
-
-  volatile void **adr_of_next = (volatile void **) ADDRESS_OF_NEXT(item, l->offset);
-
-  do {
-    INK_QUEUE_LD64(head, l->head);
-    h = FREELIST_POINTER(head);
-    *adr_of_next = h;
-    SET_FREELIST_POINTER_VERSION(item_pair, FROM_PTR(item), FREELIST_VERSION(head));
-    INK_MEMORY_BARRIER;
-
-    result = ts_lua_atomic_cas64((int64_t *) & l->head, head, item_pair);
-  } while (result == 0);
-
-  return TO_PTR(h);
-}
-
-void *
-ts_lua_atomiclist_popall(ts_lua_atomiclist * l)
-{
-  int64_t item, next;
-  void *ret;
-  int result = 0;
-
-  do {
-    INK_QUEUE_LD64(item, l->head);
-    if (TO_PTR(FREELIST_POINTER(item)) == NULL)
-      return NULL;
-
-    SET_FREELIST_POINTER_VERSION(next, FROM_PTR(NULL), FREELIST_VERSION(item) + 1);
-    result = ts_lua_atomic_cas64((int64_t *) & l->head, item, next);
-  } while (result == 0);
-
-  ret = TO_PTR(FREELIST_POINTER(item));
-
-  return ret;
-}
-
-static inline int64_t
-ts_lua_atomic_cas64(volatile int64_t * mem, int64_t old, int64_t new_value)
-{
-  return __sync_bool_compare_and_swap(mem, old, new_value);
-}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_atomic.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_atomic.h b/plugins/experimental/ts_lua/ts_lua_atomic.h
deleted file mode 100644
index feca1fe..0000000
--- a/plugins/experimental/ts_lua/ts_lua_atomic.h
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
-  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_ATOMIC_H
-#define _TS_LUA_ATOMIC_H
-
-#include <stdio.h>
-#include <stdlib.h>
-#include <stdint.h>
-
-static inline int
-ts_lua_atomic_increment(volatile int32_t * mem, int value)
-{
-  return __sync_fetch_and_add(mem, value);
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_cached_response.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_cached_response.c b/plugins/experimental/ts_lua/ts_lua_cached_response.c
index 9294fcb..20afdb7 100644
--- a/plugins/experimental/ts_lua/ts_lua_cached_response.c
+++ b/plugins/experimental/ts_lua/ts_lua_cached_response.c
@@ -19,10 +19,35 @@
 
 #include "ts_lua_util.h"
 
-static void ts_lua_inject_cached_response_header_misc_api(lua_State * L);
+
+#define TS_LUA_CHECK_CACHED_RESPONSE_HDR(http_ctx)   \
+do {                                                 \
+    TSMBuffer       bufp;                            \
+    TSMLoc          hdrp;                            \
+    if (!http_ctx->cached_response_hdrp) {           \
+        if (TSHttpTxnCachedRespGet(http_ctx->txnp,   \
+                    &bufp,                           \
+                    &hdrp) != TS_SUCCESS) {          \
+            return 0;                                \
+        }                                            \
+        http_ctx->cached_response_bufp = TSMBufferCreate();                                         \
+        http_ctx->cached_response_hdrp = TSHttpHdrCreate(http_ctx->cached_response_bufp);           \
+        TSHttpHdrCopy(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp, bufp, hdrp);  \
+        TSHandleMLocRelease(bufp, TS_NULL_MLOC, hdrp);                                              \
+    }   \
+} while(0)
+
+
+static void ts_lua_inject_cached_response_misc_api(lua_State * L);
 static void ts_lua_inject_cached_response_header_api(lua_State * L);
+static void ts_lua_inject_cached_response_headers_api(lua_State * L);
+
+static int ts_lua_cached_response_header_get(lua_State * L);
+static int ts_lua_cached_response_header_set(lua_State * L);
+static int ts_lua_cached_response_get_headers(lua_State * L);
 
-static int ts_lua_cached_response_header_get_status(lua_State * L);
+static int ts_lua_cached_response_get_status(lua_State * L);
+static int ts_lua_cached_response_get_version(lua_State * L);
 
 
 void
@@ -31,6 +56,9 @@ ts_lua_inject_cached_response_api(lua_State * L)
   lua_newtable(L);
 
   ts_lua_inject_cached_response_header_api(L);
+  ts_lua_inject_cached_response_headers_api(L);
+  ts_lua_inject_cached_response_misc_api(L);
+
 
   lua_setfield(L, -2, "cached_response");
 }
@@ -39,49 +67,157 @@ static void
 ts_lua_inject_cached_response_header_api(lua_State * L)
 {
   lua_newtable(L);              /*  .header */
-  ts_lua_inject_cached_response_header_misc_api(L);
-/*
-    lua_createtable(L, 0, 2);
 
-    lua_pushcfunction(L, ts_lua_cached_response_header_get);
-    lua_setfield(L, -2, "__index");
-    lua_pushcfunction(L, ts_lua_cached_response_header_set);
-    lua_setfield(L, -2, "__newindex");
+  lua_createtable(L, 0, 2);
+
+  lua_pushcfunction(L, ts_lua_cached_response_header_get);
+  lua_setfield(L, -2, "__index");
+  lua_pushcfunction(L, ts_lua_cached_response_header_set);
+  lua_setfield(L, -2, "__newindex");
+
+  lua_setmetatable(L, -2);
 
-    lua_setmetatable(L, -2);
-*/
   lua_setfield(L, -2, "header");
 
   return;
 }
 
 static void
-ts_lua_inject_cached_response_header_misc_api(lua_State * L)
+ts_lua_inject_cached_response_headers_api(lua_State * L)
 {
-  lua_pushcfunction(L, ts_lua_cached_response_header_get_status);
+  lua_pushcfunction(L, ts_lua_cached_response_get_headers);
+  lua_setfield(L, -2, "get_headers");
+}
+
+static void
+ts_lua_inject_cached_response_misc_api(lua_State * L)
+{
+  lua_pushcfunction(L, ts_lua_cached_response_get_status);
   lua_setfield(L, -2, "get_status");
+
+  lua_pushcfunction(L, ts_lua_cached_response_get_version);
+  lua_setfield(L, -2, "get_version");
 }
 
 static int
-ts_lua_cached_response_header_get_status(lua_State * L)
+ts_lua_cached_response_get_status(lua_State * L)
 {
   int status;
   ts_lua_http_ctx *http_ctx;
 
   http_ctx = ts_lua_get_http_ctx(L);
 
-  if (!http_ctx->cached_response_hdrp) {
-    if (TSHttpTxnCachedRespGet(http_ctx->txnp, &http_ctx->cached_response_bufp,
-                               &http_ctx->cached_response_hdrp) != TS_SUCCESS) {
+  TS_LUA_CHECK_CACHED_RESPONSE_HDR(http_ctx);
+
+  status = TSHttpHdrStatusGet(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp);
+
+  lua_pushinteger(L, status);
+
+  return 1;
+}
+
+static int
+ts_lua_cached_response_get_version(lua_State * L)
+{
+  int version;
+  char buf[32];
+  int n;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_CACHED_RESPONSE_HDR(http_ctx);
+
+  version = TSHttpHdrVersionGet(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp);
+
+  n = snprintf(buf, sizeof(buf) - 1, "%d.%d", TS_HTTP_MAJOR(version), TS_HTTP_MINOR(version));
+  lua_pushlstring(L, buf, n);
+
+  return 1;
+}
 
+static int
+ts_lua_cached_response_header_get(lua_State * L)
+{
+  const char *key;
+  const char *val;
+  int val_len;
+  size_t key_len;
+
+  TSMLoc field_loc;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  /*   we skip the first argument that is the table */
+  key = luaL_checklstring(L, 2, &key_len);
+
+  TS_LUA_CHECK_CACHED_RESPONSE_HDR(http_ctx);
+
+  if (key && key_len) {
+    field_loc = TSMimeHdrFieldFind(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp, key, key_len);
+
+    if (field_loc) {
+      val =
+        TSMimeHdrFieldValueStringGet(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp, field_loc, -1,
+                                     &val_len);
+      lua_pushlstring(L, val, val_len);
+      TSHandleMLocRelease(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp, field_loc);
+
+    } else {
       lua_pushnil(L);
-      return 1;
     }
+
+  } else {
+    lua_pushnil(L);
   }
+  return 1;
+}
 
-  status = TSHttpHdrStatusGet(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp);
+static int
+ts_lua_cached_response_header_set(lua_State * L)
+{
+  return 0;
+}
 
-  lua_pushinteger(L, status);
+static int
+ts_lua_cached_response_get_headers(lua_State * L)
+{
+  const char *name;
+  const char *value;
+  int name_len;
+  int value_len;
+  TSMLoc field_loc;
+  TSMLoc next_field_loc;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_CACHED_RESPONSE_HDR(http_ctx);
+
+  lua_newtable(L);
+
+  field_loc = TSMimeHdrFieldGet(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp, 0);
+
+  while (field_loc) {
+
+    name = TSMimeHdrFieldNameGet(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp, field_loc, &name_len);
+    if (name && name_len) {
+
+      value =
+        TSMimeHdrFieldValueStringGet(http_ctx->cached_response_bufp, http_ctx->cached_response_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(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp, field_loc);
+    TSHandleMLocRelease(http_ctx->cached_response_bufp, http_ctx->cached_response_hdrp, field_loc);
+    field_loc = next_field_loc;
+  }
 
   return 1;
 }


[2/3] TS-2723: add new features to ts_lua plugin

Posted by ki...@apache.org.
http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_client_request.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_client_request.c b/plugins/experimental/ts_lua/ts_lua_client_request.c
index 19aa2f0..d45e6e0 100644
--- a/plugins/experimental/ts_lua/ts_lua_client_request.c
+++ b/plugins/experimental/ts_lua/ts_lua_client_request.c
@@ -18,6 +18,7 @@
 
 
 #include "ink_platform.h"
+#include <netinet/in.h>
 #include "ts_lua_util.h"
 
 static void ts_lua_inject_client_request_client_addr_api(lua_State * L);
@@ -25,23 +26,36 @@ static void ts_lua_inject_client_request_server_addr_api(lua_State * L);
 
 static int ts_lua_client_request_header_get(lua_State * L);
 static int ts_lua_client_request_header_set(lua_State * L);
+static int ts_lua_client_request_get_headers(lua_State * L);
 static int ts_lua_client_request_get_url(lua_State * L);
 static int ts_lua_client_request_get_pristine_url(lua_State * L);
+static int ts_lua_client_request_get_url_host(lua_State * L);
+static int ts_lua_client_request_set_url_host(lua_State * L);
+static int ts_lua_client_request_get_url_port(lua_State * L);
+static int ts_lua_client_request_set_url_port(lua_State * L);
+static int ts_lua_client_request_get_url_scheme(lua_State * L);
+static int ts_lua_client_request_set_url_scheme(lua_State * L);
+
 static int ts_lua_client_request_get_uri(lua_State * L);
 static int ts_lua_client_request_set_uri(lua_State * L);
 static int ts_lua_client_request_set_uri_args(lua_State * L);
 static int ts_lua_client_request_get_uri_args(lua_State * L);
 static int ts_lua_client_request_get_method(lua_State * L);
 static int ts_lua_client_request_set_method(lua_State * L);
+static int ts_lua_client_request_get_version(lua_State * L);
+static int ts_lua_client_request_set_version(lua_State * L);
+
 static int ts_lua_client_request_get_body_size(lua_State * L);
 static int ts_lua_client_request_get_header_size(lua_State * L);
 
 static void ts_lua_inject_client_request_socket_api(lua_State * L);
 static void ts_lua_inject_client_request_header_api(lua_State * L);
+static void ts_lua_inject_client_request_headers_api(lua_State * L);
 static void ts_lua_inject_client_request_url_api(lua_State * L);
 static void ts_lua_inject_client_request_uri_api(lua_State * L);
 static void ts_lua_inject_client_request_args_api(lua_State * L);
 static void ts_lua_inject_client_request_method_api(lua_State * L);
+static void ts_lua_inject_client_request_version_api(lua_State * L);
 static void ts_lua_inject_client_request_body_size_api(lua_State * L);
 static void ts_lua_inject_client_request_header_size_api(lua_State * L);
 
@@ -58,10 +72,12 @@ ts_lua_inject_client_request_api(lua_State * L)
 
   ts_lua_inject_client_request_socket_api(L);
   ts_lua_inject_client_request_header_api(L);
+  ts_lua_inject_client_request_headers_api(L);
   ts_lua_inject_client_request_url_api(L);
   ts_lua_inject_client_request_uri_api(L);
   ts_lua_inject_client_request_args_api(L);
   ts_lua_inject_client_request_method_api(L);
+  ts_lua_inject_client_request_version_api(L);
   ts_lua_inject_client_request_body_size_api(L);
   ts_lua_inject_client_request_header_size_api(L);
 
@@ -207,6 +223,52 @@ ts_lua_client_request_header_set(lua_State * L)
 }
 
 static void
+ts_lua_inject_client_request_headers_api(lua_State * L)
+{
+  lua_pushcfunction(L, ts_lua_client_request_get_headers);
+  lua_setfield(L, -2, "get_headers");
+}
+
+static int
+ts_lua_client_request_get_headers(lua_State * L)
+{
+  const char *name;
+  const char *value;
+  int name_len;
+  int value_len;
+  TSMLoc field_loc;
+  TSMLoc next_field_loc;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  lua_newtable(L);
+
+  field_loc = TSMimeHdrFieldGet(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, 0);
+
+  while (field_loc) {
+
+    name = TSMimeHdrFieldNameGet(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, field_loc, &name_len);
+    if (name && name_len) {
+
+      value =
+        TSMimeHdrFieldValueStringGet(http_ctx->client_request_bufp, http_ctx->client_request_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(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, field_loc);
+    TSHandleMLocRelease(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, field_loc);
+    field_loc = next_field_loc;
+  }
+
+  return 1;
+}
+
+static void
 ts_lua_inject_client_request_url_api(lua_State * L)
 {
   lua_pushcfunction(L, ts_lua_client_request_get_url);
@@ -214,6 +276,21 @@ ts_lua_inject_client_request_url_api(lua_State * L)
 
   lua_pushcfunction(L, ts_lua_client_request_get_pristine_url);
   lua_setfield(L, -2, "get_pristine_url");
+
+  lua_pushcfunction(L, ts_lua_client_request_get_url_host);
+  lua_setfield(L, -2, "get_url_host");
+  lua_pushcfunction(L, ts_lua_client_request_set_url_host);
+  lua_setfield(L, -2, "set_url_host");
+
+  lua_pushcfunction(L, ts_lua_client_request_get_url_port);
+  lua_setfield(L, -2, "get_url_port");
+  lua_pushcfunction(L, ts_lua_client_request_set_url_port);
+  lua_setfield(L, -2, "set_url_port");
+
+  lua_pushcfunction(L, ts_lua_client_request_get_url_scheme);
+  lua_setfield(L, -2, "get_url_scheme");
+  lua_pushcfunction(L, ts_lua_client_request_set_url_scheme);
+  lua_setfield(L, -2, "set_url_scheme");
 }
 
 static void
@@ -281,6 +358,106 @@ ts_lua_client_request_get_pristine_url(lua_State * L)
 }
 
 static int
+ts_lua_client_request_get_url_host(lua_State * L)
+{
+  const char *host;
+  int len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  host = TSUrlHostGet(http_ctx->client_request_bufp, http_ctx->client_request_url, &len);
+
+  lua_pushlstring(L, host, len);
+
+  return 1;
+}
+
+static int
+ts_lua_client_request_set_url_host(lua_State * L)
+{
+  const char *host;
+  size_t len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  host = luaL_checklstring(L, 1, &len);
+
+  TSUrlHostSet(http_ctx->client_request_bufp, http_ctx->client_request_url, host, len);
+
+  return 0;
+}
+
+static int
+ts_lua_client_request_get_url_port(lua_State * L)
+{
+  int port;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  port = TSUrlPortGet(http_ctx->client_request_bufp, http_ctx->client_request_url);
+
+  lua_pushnumber(L, port);
+
+  return 1;
+}
+
+static int
+ts_lua_client_request_set_url_port(lua_State * L)
+{
+  int port;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  port = luaL_checkint(L, 1);
+
+  TSUrlPortSet(http_ctx->client_request_bufp, http_ctx->client_request_url, port);
+
+  return 0;
+}
+
+static int
+ts_lua_client_request_get_url_scheme(lua_State * L)
+{
+  const char *scheme;
+  int len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  scheme = TSUrlSchemeGet(http_ctx->client_request_bufp, http_ctx->client_request_url, &len);
+
+  lua_pushlstring(L, scheme, len);
+
+  return 1;
+}
+
+static int
+ts_lua_client_request_set_url_scheme(lua_State * L)
+{
+  const char *scheme;
+  size_t len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  scheme = luaL_checklstring(L, 1, &len);
+
+  TSUrlSchemeSet(http_ctx->client_request_bufp, http_ctx->client_request_url, scheme, len);
+
+  return 0;
+}
+
+static int
 ts_lua_client_request_get_uri(lua_State * L)
 {
   char uri[TS_LUA_MAX_URL_LENGTH];
@@ -537,6 +714,55 @@ ts_lua_client_request_get_body_size(lua_State * L)
 }
 
 static void
+ts_lua_inject_client_request_version_api(lua_State * L)
+{
+  lua_pushcfunction(L, ts_lua_client_request_get_version);
+  lua_setfield(L, -2, "get_version");
+
+  lua_pushcfunction(L, ts_lua_client_request_set_version);
+  lua_setfield(L, -2, "set_version");
+}
+
+static int
+ts_lua_client_request_get_version(lua_State * L)
+{
+  int version;
+  char buf[32];
+  int n;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  version = TSHttpHdrVersionGet(http_ctx->client_request_bufp, http_ctx->client_request_hdrp);
+
+  n = snprintf(buf, sizeof(buf) - 1, "%d.%d", TS_HTTP_MAJOR(version), TS_HTTP_MINOR(version));
+  lua_pushlstring(L, buf, n);
+
+  return 1;
+}
+
+static int
+ts_lua_client_request_set_version(lua_State * L)
+{
+  const char *version;
+  size_t len;
+  int major, minor;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  version = luaL_checklstring(L, 1, &len);
+
+  sscanf(version, "%2u.%2u", &major, &minor);
+
+  TSHttpHdrVersionSet(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, TS_HTTP_VERSION(major, minor));
+
+  return 0;
+}
+
+static void
 ts_lua_inject_client_request_header_size_api(lua_State * L)
 {
   lua_pushcfunction(L, ts_lua_client_request_get_header_size);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_client_response.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_client_response.c b/plugins/experimental/ts_lua/ts_lua_client_response.c
index 01e3664..7e22eef 100644
--- a/plugins/experimental/ts_lua/ts_lua_client_response.c
+++ b/plugins/experimental/ts_lua/ts_lua_client_response.c
@@ -19,12 +19,34 @@
 
 #include "ts_lua_util.h"
 
+#define TS_LUA_CHECK_CLIENT_RESPONSE_HDR(http_ctx)     \
+do {        \
+    if (!http_ctx->client_response_hdrp) {           \
+        if (TSHttpTxnClientRespGet(http_ctx->txnp,   \
+                    &http_ctx->client_response_bufp, \
+                    &http_ctx->client_response_hdrp) != TS_SUCCESS) {    \
+            return 0;   \
+        }   \
+    }   \
+} while(0)
+
+
 static int ts_lua_client_response_header_get(lua_State * L);
 static int ts_lua_client_response_header_set(lua_State * L);
-static int ts_lua_client_response_header_get_status(lua_State * L);
+
+static int ts_lua_client_response_get_headers(lua_State * L);
+
+static int ts_lua_client_response_get_status(lua_State * L);
+static int ts_lua_client_response_set_status(lua_State * L);
+
+static int ts_lua_client_response_set_error_resp(lua_State * L);
+
+static int ts_lua_client_response_get_version(lua_State * L);
+static int ts_lua_client_response_set_version(lua_State * L);
+
 static void ts_lua_inject_client_response_header_api(lua_State * L);
-static void ts_lua_inject_client_response_get_status_api(lua_State * L);
-static void ts_lua_inject_client_response_header_misc_api(lua_State * L);
+static void ts_lua_inject_client_response_headers_api(lua_State * L);
+static void ts_lua_inject_client_response_misc_api(lua_State * L);
 
 
 void
@@ -33,7 +55,8 @@ ts_lua_inject_client_response_api(lua_State * L)
   lua_newtable(L);
 
   ts_lua_inject_client_response_header_api(L);
-  ts_lua_inject_client_response_get_status_api(L);
+  ts_lua_inject_client_response_headers_api(L);
+  ts_lua_inject_client_response_misc_api(L);
 
   lua_setfield(L, -2, "client_response");
 }
@@ -52,7 +75,6 @@ ts_lua_inject_client_response_header_api(lua_State * L)
 
   lua_setmetatable(L, -2);
 
-  ts_lua_inject_client_response_header_misc_api(L);
   lua_setfield(L, -2, "header");
 }
 
@@ -165,34 +187,81 @@ ts_lua_client_response_header_set(lua_State * L)
 }
 
 static void
-ts_lua_inject_client_response_header_misc_api(lua_State * L ATS_UNUSED)
+ts_lua_inject_client_response_headers_api(lua_State * L)
 {
-  return;
+  lua_pushcfunction(L, ts_lua_client_response_get_headers);
+  lua_setfield(L, -2, "get_headers");
+}
+
+static int
+ts_lua_client_response_get_headers(lua_State * L)
+{
+  const char *name;
+  const char *value;
+  int name_len;
+  int value_len;
+  TSMLoc field_loc;
+  TSMLoc next_field_loc;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_CLIENT_RESPONSE_HDR(http_ctx);
+
+  lua_newtable(L);
+
+  field_loc = TSMimeHdrFieldGet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, 0);
+
+  while (field_loc) {
+
+    name = TSMimeHdrFieldNameGet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, field_loc, &name_len);
+    if (name && name_len) {
+
+      value =
+        TSMimeHdrFieldValueStringGet(http_ctx->client_response_bufp, http_ctx->client_response_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(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, field_loc);
+    TSHandleMLocRelease(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, field_loc);
+    field_loc = next_field_loc;
+  }
+
+  return 1;
 }
 
 static void
-ts_lua_inject_client_response_get_status_api(lua_State * L)
+ts_lua_inject_client_response_misc_api(lua_State * L)
 {
-  lua_pushcfunction(L, ts_lua_client_response_header_get_status);
+  lua_pushcfunction(L, ts_lua_client_response_get_status);
   lua_setfield(L, -2, "get_status");
+  lua_pushcfunction(L, ts_lua_client_response_set_status);
+  lua_setfield(L, -2, "set_status");
+
+  lua_pushcfunction(L, ts_lua_client_response_get_version);
+  lua_setfield(L, -2, "get_version");
+  lua_pushcfunction(L, ts_lua_client_response_set_version);
+  lua_setfield(L, -2, "set_version");
+
+  lua_pushcfunction(L, ts_lua_client_response_set_error_resp);
+  lua_setfield(L, -2, "set_error_resp");
+
+  return;
 }
 
 static int
-ts_lua_client_response_header_get_status(lua_State * L)
+ts_lua_client_response_get_status(lua_State * L)
 {
   int status;
   ts_lua_http_ctx *http_ctx;
 
   http_ctx = ts_lua_get_http_ctx(L);
 
-  if (!http_ctx->client_response_hdrp) {
-    if (TSHttpTxnClientRespGet(http_ctx->txnp,
-                               &http_ctx->client_response_bufp, &http_ctx->client_response_hdrp) != TS_SUCCESS) {
-
-      lua_pushnil(L);
-      return 1;
-    }
-  }
+  TS_LUA_CHECK_CLIENT_RESPONSE_HDR(http_ctx);
 
   status = TSHttpHdrStatusGet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp);
 
@@ -200,3 +269,127 @@ ts_lua_client_response_header_get_status(lua_State * L)
 
   return 1;
 }
+
+static int
+ts_lua_client_response_set_status(lua_State * L)
+{
+  int status;
+  const char *reason;
+  int reason_len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_CLIENT_RESPONSE_HDR(http_ctx);
+
+  status = luaL_checkint(L, 1);
+
+  reason = TSHttpHdrReasonLookup(status);
+  reason_len = strlen(reason);
+
+  TSHttpHdrStatusSet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, status);
+  TSHttpHdrReasonSet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, reason, reason_len);
+
+  return 0;
+}
+
+static int
+ts_lua_client_response_get_version(lua_State * L)
+{
+  int version;
+  char buf[32];
+  int n;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_CLIENT_RESPONSE_HDR(http_ctx);
+
+  version = TSHttpHdrVersionGet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp);
+
+  n = snprintf(buf, sizeof(buf) - 1, "%d.%d", TS_HTTP_MAJOR(version), TS_HTTP_MINOR(version));
+  lua_pushlstring(L, buf, n);
+
+  return 1;
+}
+
+static int
+ts_lua_client_response_set_version(lua_State * L)
+{
+  const char *version;
+  size_t len;
+  int major, minor;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_CLIENT_RESPONSE_HDR(http_ctx);
+
+  version = luaL_checklstring(L, 1, &len);
+
+  sscanf(version, "%2u.%2u", &major, &minor);
+
+  TSHttpHdrVersionSet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, TS_HTTP_VERSION(major, minor));
+
+  return 0;
+}
+
+static int
+ts_lua_client_response_set_error_resp(lua_State * L)
+{
+  int n, status;
+  const char *body;
+  const char *reason;
+  int reason_len;
+  size_t body_len;
+  int resp_len;
+  char *resp_buf;
+  TSMLoc field_loc;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+  TS_LUA_CHECK_CLIENT_RESPONSE_HDR(http_ctx);
+
+  n = lua_gettop(L);
+
+  status = luaL_checkinteger(L, 1);
+
+  reason = TSHttpHdrReasonLookup(status);
+  reason_len = strlen(reason);
+
+  TSHttpHdrStatusSet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, status);
+  TSHttpHdrReasonSet(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, reason, reason_len);
+
+  body_len = 0;
+
+  if (n == 2) {
+    body = luaL_checklstring(L, 2, &body_len);
+  }
+
+  if (body_len && body) {
+    resp_buf = TSmalloc(body_len);
+    memcpy(resp_buf, body, body_len);
+    resp_len = body_len;
+
+  } else {
+    resp_buf = TSmalloc(reason_len);
+    memcpy(resp_buf, reason, reason_len);
+    resp_len = reason_len;
+  }
+
+  field_loc = TSMimeHdrFieldFind(http_ctx->client_response_bufp, http_ctx->client_response_hdrp,
+                                 TS_MIME_FIELD_TRANSFER_ENCODING, TS_MIME_LEN_TRANSFER_ENCODING);
+
+  if (field_loc) {
+    TSMimeHdrFieldDestroy(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, field_loc);
+    TSHandleMLocRelease(http_ctx->client_response_bufp, http_ctx->client_response_hdrp, field_loc);
+  }
+
+  TSHttpTxnErrorBodySet(http_ctx->txnp, resp_buf, resp_len, NULL);
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 c96f36a..980cc5f 100644
--- a/plugins/experimental/ts_lua/ts_lua_common.h
+++ b/plugins/experimental/ts_lua/ts_lua_common.h
@@ -63,18 +63,45 @@
 #define TS_LUA_FUNCTION_G_TXN_CLOSE "do_global_txn_close"
 
 
-#define TS_LUA_MAX_SCRIPT_FNAME_LENGTH      1024
-#define TS_LUA_MAX_URL_LENGTH               2048
+#define TS_LUA_DEBUG_TAG                        "ts_lua"
 
-#define TS_LUA_DEBUG_TAG                    "ts_lua"
+#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
+#define TS_LUA_MAX_SHARED_DICT_COUNT            32
+#define TS_LUA_MAX_URL_LENGTH                   2048
+#define TS_LUA_MAX_OVEC_SIZE                    (3 * 32)
+#define TS_LUA_MAX_RESIDENT_PCRE                64
 
+#define TS_LUA_MIN_ALIGN                        sizeof(void*)
+#define TS_LUA_MEM_ALIGN(size)                  (((size) + ((TS_LUA_MIN_ALIGN) - 1)) & ~((TS_LUA_MIN_ALIGN) - 1))
+#define TS_LUA_ALIGN_COUNT(size)                (size / TS_LUA_MIN_ALIGN)
+
+#define TS_LUA_MAKE_VAR_ITEM(X)                 {X, #X}
+
+#define ee(...)     fprintf(stderr, "Lua *** %s: ", __func__); \
+                            fprintf(stderr, __VA_ARGS__);   \
+                            fprintf(stderr, " @ %s line %d.\n", __FILE__, __LINE__)
+
+/* for http config or cntl var */
+typedef struct
+{
+  int nvar;
+  char *svar;
+} ts_lua_var_item;
 
 typedef struct
 {
+  char *content;
   char script[TS_LUA_MAX_SCRIPT_FNAME_LENGTH];
+  void *conf_vars[TS_LUA_MAX_CONFIG_VARS_COUNT];
+
+  int _first:1;                 // create current instance for 1st ts_lua_main_ctx
+  int _last:1;                  // create current instance for the last ts_lua_main_ctx
 } ts_lua_instance_conf;
 
 
+/* global lua state struct */
 typedef struct
 {
   lua_State *lua;
@@ -82,7 +109,7 @@ typedef struct
   int gref;
 } ts_lua_main_ctx;
 
-
+/* lua state for http request */
 typedef struct
 {
   lua_State *lua;
@@ -95,6 +122,8 @@ typedef struct
 
   TSMBuffer server_request_bufp;
   TSMLoc server_request_hdrp;
+  TSMLoc server_request_url;
+
 
   TSMBuffer server_response_bufp;
   TSMLoc server_response_hdrp;
@@ -107,6 +136,8 @@ typedef struct
 
   ts_lua_main_ctx *mctx;
 
+  ts_lua_instance_conf *instance_conf;
+
   int intercept_type;
   int ref;
 
@@ -134,8 +165,23 @@ typedef struct
 
 } ts_lua_transform_ctx;
 
+/* for intercept */
+struct ict_item;
+struct ict_ctx;
+typedef int (*ict_clean) (struct ict_item * item);
 
-typedef struct
+typedef struct ict_item
+{
+  struct ict_item *next;
+  struct ict_ctx *ictx;
+
+  TSCont contp;
+  ict_clean cleanup;
+  void *data;
+  int deleted:1;
+} ts_lua_http_intercept_item;
+
+typedef struct ict_ctx
 {
   lua_State *lua;
   TSCont contp;
@@ -143,10 +189,17 @@ typedef struct
   ts_lua_io_handle output;
   TSVConn net_vc;
 
+  ts_lua_main_ctx *mctx;
   ts_lua_http_ctx *hctx;
+
+  struct ict_item *ict_chain;
+
+  int64_t to_flush;
   int ref;
-  char recv_complete;
-  char send_complete;
+
+  int recv_complete:1;
+  int send_complete:1;
+  int all_ready:1;
 } ts_lua_http_intercept_ctx;
 
 #define TS_LUA_RELEASE_IO_HANDLE(ih) do {   \
@@ -160,4 +213,13 @@ typedef struct
     }                                       \
 } 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/83f16762/plugins/experimental/ts_lua/ts_lua_crypto.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_crypto.c b/plugins/experimental/ts_lua/ts_lua_crypto.c
new file mode 100644
index 0000000..77718d4
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_crypto.c
@@ -0,0 +1,177 @@
+/*
+  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 <openssl/md5.h>
+#include <openssl/sha.h>
+#include "ts_lua_string.h"
+#include "ts_lua_util.h"
+
+
+#define TS_LUA_MD5_DIGEST_LENGTH    16
+#define TS_LUA_SHA_DIGEST_LENGTH    20
+
+static int ts_lua_md5(lua_State * L);
+static int ts_lua_md5_bin(lua_State * L);
+
+static int ts_lua_sha1(lua_State * L);
+static int ts_lua_sha1_bin(lua_State * L);
+
+void
+ts_lua_inject_crypto_api(lua_State * L)
+{
+  /* ts.md5() */
+  lua_pushcfunction(L, ts_lua_md5);
+  lua_setfield(L, -2, "md5");
+
+  /* ts.md5_bin(...) */
+  lua_pushcfunction(L, ts_lua_md5_bin);
+  lua_setfield(L, -2, "md5_bin");
+
+  /* ts.sha1_bin(...) */
+  lua_pushcfunction(L, ts_lua_sha1);
+  lua_setfield(L, -2, "sha1");
+
+  /* ts.sha1_bin(...) */
+  lua_pushcfunction(L, ts_lua_sha1_bin);
+  lua_setfield(L, -2, "sha1_bin");
+}
+
+static int
+ts_lua_md5(lua_State * L)
+{
+  u_char *src;
+  size_t slen;
+
+  MD5_CTX md5_ctx;
+  u_char md5_buf[TS_LUA_MD5_DIGEST_LENGTH];
+  u_char hex_buf[2 * sizeof(md5_buf)];
+
+  if (lua_gettop(L) != 1) {
+    return luaL_error(L, "expecting one argument");
+  }
+
+  if (lua_isnil(L, 1)) {
+    src = (u_char *) "";
+    slen = 0;
+
+  } else {
+    src = (u_char *) luaL_checklstring(L, 1, &slen);
+  }
+
+  MD5_Init(&md5_ctx);
+  MD5_Update(&md5_ctx, src, slen);
+  MD5_Final(md5_buf, &md5_ctx);
+
+  ts_lua_hex_dump(hex_buf, md5_buf, sizeof(md5_buf));
+
+  lua_pushlstring(L, (char *) hex_buf, sizeof(hex_buf));
+
+  return 1;
+}
+
+static int
+ts_lua_md5_bin(lua_State * L)
+{
+  u_char *src;
+  size_t slen;
+
+  MD5_CTX md5_ctx;
+  u_char md5_buf[TS_LUA_MD5_DIGEST_LENGTH];
+
+  if (lua_gettop(L) != 1) {
+    return luaL_error(L, "expecting one argument");
+  }
+
+  if (lua_isnil(L, 1)) {
+    src = (u_char *) "";
+    slen = 0;
+
+  } else {
+    src = (u_char *) luaL_checklstring(L, 1, &slen);
+  }
+
+  MD5_Init(&md5_ctx);
+  MD5_Update(&md5_ctx, src, slen);
+  MD5_Final(md5_buf, &md5_ctx);
+
+  lua_pushlstring(L, (char *) md5_buf, sizeof(md5_buf));
+
+  return 1;
+}
+
+static int
+ts_lua_sha1(lua_State * L)
+{
+  u_char *src;
+  size_t slen;
+
+  SHA_CTX sha;
+  u_char sha_buf[TS_LUA_SHA_DIGEST_LENGTH];
+  u_char hex_buf[2 * sizeof(sha_buf)];
+
+  if (lua_gettop(L) != 1) {
+    return luaL_error(L, "expecting one argument");
+  }
+
+  if (lua_isnil(L, 1)) {
+    src = (u_char *) "";
+    slen = 0;
+
+  } else {
+    src = (u_char *) luaL_checklstring(L, 1, &slen);
+  }
+
+  SHA1_Init(&sha);
+  SHA1_Update(&sha, src, slen);
+  SHA1_Final(sha_buf, &sha);
+
+  ts_lua_hex_dump(hex_buf, sha_buf, sizeof(sha_buf));
+  lua_pushlstring(L, (char *) hex_buf, sizeof(hex_buf));
+
+  return 1;
+}
+
+static int
+ts_lua_sha1_bin(lua_State * L)
+{
+  u_char *src;
+  size_t slen;
+
+  SHA_CTX sha;
+  u_char sha_buf[TS_LUA_SHA_DIGEST_LENGTH];
+
+  if (lua_gettop(L) != 1) {
+    return luaL_error(L, "expecting one argument");
+  }
+
+  if (lua_isnil(L, 1)) {
+    src = (u_char *) "";
+    slen = 0;
+
+  } else {
+    src = (u_char *) luaL_checklstring(L, 1, &slen);
+  }
+
+  SHA1_Init(&sha);
+  SHA1_Update(&sha, src, slen);
+  SHA1_Final(sha_buf, &sha);
+
+  lua_pushlstring(L, (char *) sha_buf, sizeof(sha_buf));
+
+  return 1;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_crypto.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_crypto.h b/plugins/experimental/ts_lua/ts_lua_crypto.h
new file mode 100644
index 0000000..6f0da12
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_crypto.h
@@ -0,0 +1,27 @@
+/*
+  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_CRYPTO_H
+#define _TS_LUA_CRYPTO_H
+
+#include "ts_lua_common.h"
+
+void ts_lua_inject_crypto_api(lua_State * L);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 e70312c..6fb3985 100644
--- a/plugins/experimental/ts_lua/ts_lua_hook.c
+++ b/plugins/experimental/ts_lua/ts_lua_hook.c
@@ -79,7 +79,7 @@ ts_lua_inject_hook_api(lua_State * L)
 static void
 ts_lua_inject_hook_variables(lua_State * L)
 {
-  unsigned int i;
+  int i;
 
   for (i = 0; i < sizeof(ts_lua_hook_id_string) / sizeof(char *); i++) {
     lua_pushinteger(L, i);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 599cd80..9ceb3e8 100644
--- a/plugins/experimental/ts_lua/ts_lua_http.c
+++ b/plugins/experimental/ts_lua/ts_lua_http.c
@@ -19,6 +19,8 @@
 
 #include "ts_lua_util.h"
 #include "ts_lua_http_intercept.h"
+#include "ts_lua_http_config.h"
+#include "ts_lua_http_cntl.h"
 
 typedef enum
 {
@@ -38,6 +40,7 @@ char *ts_lua_cache_lookup_result_string[] = {
 static void ts_lua_inject_http_retset_api(lua_State * L);
 static void ts_lua_inject_http_cache_api(lua_State * L);
 static void ts_lua_inject_http_transform_api(lua_State * L);
+static void ts_lua_inject_http_misc_api(lua_State * L);
 
 static int ts_lua_http_set_retstatus(lua_State * L);
 static int ts_lua_http_set_retbody(lua_State * L);
@@ -45,11 +48,15 @@ static int ts_lua_http_set_resp(lua_State * L);
 
 static int ts_lua_http_get_cache_lookup_status(lua_State * L);
 static int ts_lua_http_set_cache_url(lua_State * L);
+
 static void ts_lua_inject_cache_lookup_result_variables(lua_State * L);
 
 static int ts_lua_http_resp_cache_transformed(lua_State * L);
 static int ts_lua_http_resp_cache_untransformed(lua_State * L);
 
+static int ts_lua_http_is_internal_request(lua_State * L);
+
+
 void
 ts_lua_inject_http_api(lua_State * L)
 {
@@ -59,6 +66,9 @@ ts_lua_inject_http_api(lua_State * L)
   ts_lua_inject_http_cache_api(L);
   ts_lua_inject_http_transform_api(L);
   ts_lua_inject_http_intercept_api(L);
+  ts_lua_inject_http_config_api(L);
+  ts_lua_inject_http_cntl_api(L);
+  ts_lua_inject_http_misc_api(L);
 
   lua_setfield(L, -2, "http");
 }
@@ -99,9 +109,16 @@ ts_lua_inject_http_transform_api(lua_State * L)
 }
 
 static void
+ts_lua_inject_http_misc_api(lua_State * L)
+{
+  lua_pushcfunction(L, ts_lua_http_is_internal_request);
+  lua_setfield(L, -2, "is_internal_request");
+}
+
+static void
 ts_lua_inject_cache_lookup_result_variables(lua_State * L)
 {
-  unsigned int i;
+  int i;
 
   for (i = 0; i < sizeof(ts_lua_cache_lookup_result_string) / sizeof(char *); i++) {
     lua_pushinteger(L, i);
@@ -225,3 +242,23 @@ ts_lua_http_resp_cache_untransformed(lua_State * L)
 
   return 0;
 }
+
+static int
+ts_lua_http_is_internal_request(lua_State * L)
+{
+  TSReturnCode ret;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  ret = TSHttpIsInternalRequest(http_ctx->txnp);
+
+  if (ret == TS_SUCCESS) {
+    lua_pushnumber(L, 1);
+
+  } else {
+    lua_pushnumber(L, 0);
+  }
+
+  return 1;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_http_cntl.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_http_cntl.c b/plugins/experimental/ts_lua/ts_lua_http_cntl.c
new file mode 100644
index 0000000..e88bf74
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_http_cntl.c
@@ -0,0 +1,101 @@
+/*
+  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_util.h"
+
+typedef enum
+{
+  TS_LUA_HTTP_CNTL_GET_LOGGING_MODE = TS_HTTP_CNTL_GET_LOGGING_MODE,
+  TS_LUA_HTTP_CNTL_SET_LOGGING_MODE = TS_HTTP_CNTL_SET_LOGGING_MODE,
+  TS_LUA_HTTP_CNTL_GET_INTERCEPT_RETRY_MODE = TS_HTTP_CNTL_GET_INTERCEPT_RETRY_MODE,
+  TS_LUA_HTTP_CNTL_SET_INTERCEPT_RETRY_MODE = TS_HTTP_CNTL_SET_INTERCEPT_RETRY_MODE
+} TSLuaHttpCntlType;
+
+
+ts_lua_var_item ts_lua_http_cntl_type_vars[] = {
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_HTTP_CNTL_GET_LOGGING_MODE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_HTTP_CNTL_SET_LOGGING_MODE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_HTTP_CNTL_GET_INTERCEPT_RETRY_MODE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_HTTP_CNTL_SET_INTERCEPT_RETRY_MODE)
+};
+
+
+static void ts_lua_inject_http_cntl_variables(lua_State * L);
+
+static int ts_lua_http_cntl_set(lua_State * L);
+static int ts_lua_http_cntl_get(lua_State * L);
+
+
+void
+ts_lua_inject_http_cntl_api(lua_State * L)
+{
+  ts_lua_inject_http_cntl_variables(L);
+
+  lua_pushcfunction(L, ts_lua_http_cntl_set);
+  lua_setfield(L, -2, "cntl_set");
+
+  lua_pushcfunction(L, ts_lua_http_cntl_get);
+  lua_setfield(L, -2, "cntl_get");
+}
+
+static void
+ts_lua_inject_http_cntl_variables(lua_State * L)
+{
+  int i;
+
+  for (i = 0; i < sizeof(ts_lua_http_cntl_type_vars) / sizeof(ts_lua_var_item); i++) {
+    lua_pushinteger(L, ts_lua_http_cntl_type_vars[i].nvar);
+    lua_setglobal(L, ts_lua_http_cntl_type_vars[i].svar);
+  }
+}
+
+static int
+ts_lua_http_cntl_set(lua_State * L)
+{
+  int cntl_type;
+  int value;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  cntl_type = luaL_checkinteger(L, 1);
+  value = luaL_checkinteger(L, 2);
+
+  TSHttpTxnCntl(http_ctx->txnp, cntl_type, value ? TS_HTTP_CNTL_ON : TS_HTTP_CNTL_OFF);
+
+  return 0;
+}
+
+static int
+ts_lua_http_cntl_get(lua_State * L)
+{
+  int cntl_type;
+  int64_t value;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  cntl_type = luaL_checkinteger(L, 1);
+
+  TSHttpTxnCntl(http_ctx->txnp, cntl_type, &value);
+
+  lua_pushnumber(L, value);
+
+  return 1;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_http_cntl.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_http_cntl.h b/plugins/experimental/ts_lua/ts_lua_http_cntl.h
new file mode 100644
index 0000000..9613084
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_http_cntl.h
@@ -0,0 +1,27 @@
+/*
+  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_HTTP_CNTL_H
+#define _TS_LUA_HTTP_CNTL_H
+
+#include "ts_lua_common.h"
+
+void ts_lua_inject_http_cntl_api(lua_State * L);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_http_config.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_http_config.c b/plugins/experimental/ts_lua/ts_lua_http_config.c
new file mode 100644
index 0000000..9209e23
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_http_config.c
@@ -0,0 +1,304 @@
+/*
+  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_util.h"
+
+typedef enum
+{
+  TS_LUA_CONFIG_URL_REMAP_PRISTINE_HOST_HDR = TS_CONFIG_URL_REMAP_PRISTINE_HOST_HDR,
+  TS_LUA_CONFIG_HTTP_CHUNKING_ENABLED = TS_CONFIG_HTTP_CHUNKING_ENABLED,
+  TS_LUA_CONFIG_HTTP_NEGATIVE_CACHING_ENABLED = TS_CONFIG_HTTP_NEGATIVE_CACHING_ENABLED,
+  TS_LUA_CONFIG_HTTP_NEGATIVE_CACHING_LIFETIME = TS_CONFIG_HTTP_CACHE_WHEN_TO_REVALIDATE,
+  TS_LUA_CONFIG_HTTP_CACHE_WHEN_TO_REVALIDATE = TS_CONFIG_HTTP_CACHE_WHEN_TO_REVALIDATE,
+  TS_LUA_CONFIG_HTTP_KEEP_ALIVE_ENABLED_IN = TS_CONFIG_HTTP_KEEP_ALIVE_ENABLED_IN,
+  TS_LUA_CONFIG_HTTP_KEEP_ALIVE_ENABLED_OUT = TS_CONFIG_HTTP_KEEP_ALIVE_ENABLED_OUT,
+  TS_LUA_CONFIG_HTTP_KEEP_ALIVE_POST_OUT = TS_CONFIG_HTTP_KEEP_ALIVE_POST_OUT,
+  TS_LUA_CONFIG_HTTP_SHARE_SERVER_SESSIONS = TS_CONFIG_HTTP_SHARE_SERVER_SESSIONS,
+  TS_LUA_CONFIG_NET_SOCK_RECV_BUFFER_SIZE_OUT = TS_CONFIG_NET_SOCK_RECV_BUFFER_SIZE_OUT,
+  TS_LUA_CONFIG_NET_SOCK_SEND_BUFFER_SIZE_OUT = TS_CONFIG_NET_SOCK_SEND_BUFFER_SIZE_OUT,
+  TS_LUA_CONFIG_NET_SOCK_OPTION_FLAG_OUT = TS_CONFIG_NET_SOCK_OPTION_FLAG_OUT,
+  TS_LUA_CONFIG_HTTP_FORWARD_PROXY_AUTH_TO_PARENT = TS_CONFIG_HTTP_FORWARD_PROXY_AUTH_TO_PARENT,
+  TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_FROM = TS_CONFIG_HTTP_ANONYMIZE_REMOVE_FROM,
+  TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_REFERER = TS_CONFIG_HTTP_ANONYMIZE_REMOVE_REFERER,
+  TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_USER_AGENT = TS_CONFIG_HTTP_ANONYMIZE_REMOVE_USER_AGENT,
+  TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_COOKIE = TS_CONFIG_HTTP_ANONYMIZE_REMOVE_COOKIE,
+  TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_CLIENT_IP = TS_CONFIG_HTTP_ANONYMIZE_REMOVE_CLIENT_IP,
+  TS_LUA_CONFIG_HTTP_ANONYMIZE_INSERT_CLIENT_IP = TS_CONFIG_HTTP_ANONYMIZE_INSERT_CLIENT_IP,
+  TS_LUA_CONFIG_HTTP_RESPONSE_SERVER_ENABLED = TS_CONFIG_HTTP_RESPONSE_SERVER_ENABLED,
+  TS_LUA_CONFIG_HTTP_INSERT_SQUID_X_FORWARDED_FOR = TS_CONFIG_HTTP_INSERT_SQUID_X_FORWARDED_FOR,
+  TS_LUA_CONFIG_HTTP_SERVER_TCP_INIT_CWND = TS_CONFIG_HTTP_SERVER_TCP_INIT_CWND,
+  TS_LUA_CONFIG_HTTP_SEND_HTTP11_REQUESTS = TS_CONFIG_HTTP_SEND_HTTP11_REQUESTS,
+  TS_LUA_CONFIG_HTTP_CACHE_HTTP = TS_CONFIG_HTTP_CACHE_HTTP,
+  TS_LUA_CONFIG_HTTP_CACHE_CLUSTER_CACHE_LOCAL = TS_CONFIG_HTTP_CACHE_CLUSTER_CACHE_LOCAL,
+  TS_LUA_CONFIG_HTTP_CACHE_IGNORE_CLIENT_NO_CACHE = TS_CONFIG_HTTP_CACHE_IGNORE_CLIENT_NO_CACHE,
+  TS_LUA_CONFIG_HTTP_CACHE_IGNORE_CLIENT_CC_MAX_AGE = TS_CONFIG_HTTP_CACHE_IGNORE_CLIENT_CC_MAX_AGE,
+  TS_LUA_CONFIG_HTTP_CACHE_IMS_ON_CLIENT_NO_CACHE = TS_CONFIG_HTTP_CACHE_IMS_ON_CLIENT_NO_CACHE,
+  TS_LUA_CONFIG_HTTP_CACHE_IGNORE_SERVER_NO_CACHE = TS_CONFIG_HTTP_CACHE_IGNORE_SERVER_NO_CACHE,
+  TS_LUA_CONFIG_HTTP_CACHE_CACHE_RESPONSES_TO_COOKIES = TS_CONFIG_HTTP_CACHE_CACHE_RESPONSES_TO_COOKIES,
+  TS_LUA_CONFIG_HTTP_CACHE_IGNORE_AUTHENTICATION = TS_CONFIG_HTTP_CACHE_IGNORE_AUTHENTICATION,
+  TS_LUA_CONFIG_HTTP_CACHE_CACHE_URLS_THAT_LOOK_DYNAMIC = TS_CONFIG_HTTP_CACHE_CACHE_URLS_THAT_LOOK_DYNAMIC,
+  TS_LUA_CONFIG_HTTP_CACHE_REQUIRED_HEADERS = TS_CONFIG_HTTP_CACHE_REQUIRED_HEADERS,
+  TS_LUA_CONFIG_HTTP_INSERT_REQUEST_VIA_STR = TS_CONFIG_HTTP_INSERT_REQUEST_VIA_STR,
+  TS_LUA_CONFIG_HTTP_INSERT_RESPONSE_VIA_STR = TS_CONFIG_HTTP_INSERT_RESPONSE_VIA_STR,
+  TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_MIN_LIFETIME = TS_CONFIG_HTTP_CACHE_HEURISTIC_MIN_LIFETIME,
+  TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_MAX_LIFETIME = TS_CONFIG_HTTP_CACHE_HEURISTIC_MAX_LIFETIME,
+  TS_LUA_CONFIG_HTTP_CACHE_GUARANTEED_MIN_LIFETIME = TS_CONFIG_HTTP_CACHE_GUARANTEED_MIN_LIFETIME,
+  TS_LUA_CONFIG_HTTP_CACHE_GUARANTEED_MAX_LIFETIME = TS_CONFIG_HTTP_CACHE_GUARANTEED_MAX_LIFETIME,
+  TS_LUA_CONFIG_HTTP_CACHE_MAX_STALE_AGE = TS_CONFIG_HTTP_CACHE_MAX_STALE_AGE,
+  TS_LUA_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_IN = TS_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_IN,
+  TS_LUA_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_OUT = TS_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_OUT,
+  TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN = TS_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN,
+  TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT = TS_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT,
+  TS_LUA_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT = TS_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT,
+  TS_LUA_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS = TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS,
+  TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES = TS_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES,
+  TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER = TS_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER,
+  TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES = TS_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES,
+  TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_TIMEOUT = TS_CONFIG_HTTP_CONNECT_ATTEMPTS_TIMEOUT,
+  TS_LUA_CONFIG_HTTP_POST_CONNECT_ATTEMPTS_TIMEOUT = TS_CONFIG_HTTP_POST_CONNECT_ATTEMPTS_TIMEOUT,
+  TS_LUA_CONFIG_HTTP_DOWN_SERVER_CACHE_TIME = TS_CONFIG_HTTP_DOWN_SERVER_CACHE_TIME,
+  TS_LUA_CONFIG_HTTP_DOWN_SERVER_ABORT_THRESHOLD = TS_CONFIG_HTTP_DOWN_SERVER_ABORT_THRESHOLD,
+  TS_LUA_CONFIG_HTTP_CACHE_FUZZ_TIME = TS_CONFIG_HTTP_CACHE_FUZZ_TIME,
+  TS_LUA_CONFIG_HTTP_CACHE_FUZZ_MIN_TIME = TS_CONFIG_HTTP_CACHE_FUZZ_MIN_TIME,
+  TS_LUA_CONFIG_HTTP_DOC_IN_CACHE_SKIP_DNS = TS_CONFIG_HTTP_DOC_IN_CACHE_SKIP_DNS,
+  TS_LUA_CONFIG_HTTP_RESPONSE_SERVER_STR = TS_CONFIG_HTTP_RESPONSE_SERVER_STR,
+  TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_LM_FACTOR = TS_CONFIG_HTTP_CACHE_HEURISTIC_LM_FACTOR,
+  TS_LUA_CONFIG_HTTP_CACHE_FUZZ_PROBABILITY = TS_CONFIG_HTTP_CACHE_FUZZ_PROBABILITY,
+  TS_LUA_CONFIG_NET_SOCK_PACKET_MARK_OUT = TS_CONFIG_NET_SOCK_PACKET_MARK_OUT,
+  TS_LUA_CONFIG_NET_SOCK_PACKET_TOS_OUT = TS_CONFIG_NET_SOCK_PACKET_TOS_OUT,
+  TS_LUA_CONFIG_LAST_ENTRY = TS_CONFIG_LAST_ENTRY
+} TSLuaOverridableConfigKey;
+
+
+ts_lua_var_item ts_lua_http_config_vars[] = {
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_URL_REMAP_PRISTINE_HOST_HDR),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CHUNKING_ENABLED),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_NEGATIVE_CACHING_ENABLED),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_NEGATIVE_CACHING_LIFETIME),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_WHEN_TO_REVALIDATE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_KEEP_ALIVE_ENABLED_IN),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_KEEP_ALIVE_ENABLED_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_KEEP_ALIVE_POST_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_SHARE_SERVER_SESSIONS),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_NET_SOCK_RECV_BUFFER_SIZE_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_NET_SOCK_SEND_BUFFER_SIZE_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_NET_SOCK_OPTION_FLAG_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_FORWARD_PROXY_AUTH_TO_PARENT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_FROM),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_REFERER),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_USER_AGENT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_COOKIE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ANONYMIZE_REMOVE_CLIENT_IP),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ANONYMIZE_INSERT_CLIENT_IP),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_RESPONSE_SERVER_ENABLED),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_INSERT_SQUID_X_FORWARDED_FOR),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_SERVER_TCP_INIT_CWND),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_SEND_HTTP11_REQUESTS),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_HTTP),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_CLUSTER_CACHE_LOCAL),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_IGNORE_CLIENT_NO_CACHE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_IGNORE_CLIENT_CC_MAX_AGE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_IMS_ON_CLIENT_NO_CACHE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_IGNORE_SERVER_NO_CACHE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_CACHE_RESPONSES_TO_COOKIES),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_IGNORE_AUTHENTICATION),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_CACHE_URLS_THAT_LOOK_DYNAMIC),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_REQUIRED_HEADERS),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_INSERT_REQUEST_VIA_STR),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_INSERT_RESPONSE_VIA_STR),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_MIN_LIFETIME),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_MAX_LIFETIME),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_GUARANTEED_MIN_LIFETIME),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_GUARANTEED_MAX_LIFETIME),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_MAX_STALE_AGE),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_IN),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_TIMEOUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_POST_CONNECT_ATTEMPTS_TIMEOUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_DOWN_SERVER_CACHE_TIME),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_DOWN_SERVER_ABORT_THRESHOLD),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_FUZZ_TIME),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_FUZZ_MIN_TIME),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_DOC_IN_CACHE_SKIP_DNS),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_RESPONSE_SERVER_STR),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_HEURISTIC_LM_FACTOR),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_FUZZ_PROBABILITY),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_NET_SOCK_PACKET_MARK_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_NET_SOCK_PACKET_TOS_OUT),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_LAST_ENTRY),
+};
+
+static void ts_lua_inject_http_config_variables(lua_State * L);
+
+static int ts_lua_http_config_int_set(lua_State * L);
+static int ts_lua_http_config_int_get(lua_State * L);
+static int ts_lua_http_config_float_set(lua_State * L);
+static int ts_lua_http_config_float_get(lua_State * L);
+static int ts_lua_http_config_string_set(lua_State * L);
+static int ts_lua_http_config_string_get(lua_State * L);
+
+
+void
+ts_lua_inject_http_config_api(lua_State * L)
+{
+  ts_lua_inject_http_config_variables(L);
+
+  lua_pushcfunction(L, ts_lua_http_config_int_set);
+  lua_setfield(L, -2, "config_int_set");
+
+  lua_pushcfunction(L, ts_lua_http_config_int_get);
+  lua_setfield(L, -2, "config_int_get");
+
+  lua_pushcfunction(L, ts_lua_http_config_float_set);
+  lua_setfield(L, -2, "config_float_set");
+
+  lua_pushcfunction(L, ts_lua_http_config_float_get);
+  lua_setfield(L, -2, "config_float_get");
+
+  lua_pushcfunction(L, ts_lua_http_config_string_set);
+  lua_setfield(L, -2, "config_string_set");
+
+  lua_pushcfunction(L, ts_lua_http_config_string_get);
+  lua_setfield(L, -2, "config_string_get");
+}
+
+static void
+ts_lua_inject_http_config_variables(lua_State * L)
+{
+  int i;
+
+  for (i = 0; i < sizeof(ts_lua_http_config_vars) / sizeof(ts_lua_var_item); i++) {
+    lua_pushinteger(L, ts_lua_http_config_vars[i].nvar);
+    lua_setglobal(L, ts_lua_http_config_vars[i].svar);
+  }
+}
+
+static int
+ts_lua_http_config_int_set(lua_State * L)
+{
+  int conf;
+  int value;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  conf = luaL_checkinteger(L, 1);
+  value = luaL_checkinteger(L, 2);
+
+  TSHttpTxnConfigIntSet(http_ctx->txnp, conf, value);
+
+  return 0;
+}
+
+static int
+ts_lua_http_config_int_get(lua_State * L)
+{
+  int conf;
+  int64_t value;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  conf = luaL_checkinteger(L, 1);
+
+  TSHttpTxnConfigIntGet(http_ctx->txnp, conf, &value);
+
+  lua_pushnumber(L, value);
+
+  return 1;
+}
+
+static int
+ts_lua_http_config_float_set(lua_State * L)
+{
+  int conf;
+  float value;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  conf = luaL_checkinteger(L, 1);
+  value = luaL_checknumber(L, 2);
+
+  TSHttpTxnConfigFloatSet(http_ctx->txnp, conf, value);
+
+  return 0;
+}
+
+static int
+ts_lua_http_config_float_get(lua_State * L)
+{
+  int conf;
+  float value;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  conf = luaL_checkinteger(L, 1);
+
+  TSHttpTxnConfigFloatGet(http_ctx->txnp, conf, &value);
+
+  lua_pushnumber(L, value);
+
+  return 1;
+}
+
+static int
+ts_lua_http_config_string_set(lua_State * L)
+{
+  int conf;
+  const char *value;
+  size_t value_len;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  conf = luaL_checkinteger(L, 1);
+  value = luaL_checklstring(L, 2, &value_len);
+
+  TSHttpTxnConfigStringSet(http_ctx->txnp, conf, value, value_len);
+
+  return 0;
+}
+
+static int
+ts_lua_http_config_string_get(lua_State * L)
+{
+  int conf;
+  const char *value;
+  int value_len;
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  conf = luaL_checkinteger(L, 1);
+
+  TSHttpTxnConfigStringGet(http_ctx->txnp, conf, &value, &value_len);
+
+  lua_pushlstring(L, value, value_len);
+
+  return 1;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_http_config.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_http_config.h b/plugins/experimental/ts_lua/ts_lua_http_config.h
new file mode 100644
index 0000000..0cf00f8
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_http_config.h
@@ -0,0 +1,27 @@
+/*
+  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_HTTP_CONFIG_H
+#define _TS_LUA_HTTP_CONFIG_H
+
+#include "ts_lua_common.h"
+
+void ts_lua_inject_http_config_api(lua_State * L);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 88d41cd..21e142a 100644
--- a/plugins/experimental/ts_lua/ts_lua_http_intercept.c
+++ b/plugins/experimental/ts_lua/ts_lua_http_intercept.c
@@ -35,10 +35,12 @@ static void ts_lua_http_intercept_process(ts_lua_http_ctx * http_ctx, TSVConn co
 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);
-static int ts_lua_http_intercept_run_coroutine(ts_lua_http_intercept_ctx * ictx);
+static int ts_lua_http_intercept_run_coroutine(ts_lua_http_intercept_ctx * ictx, int n);
 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);
+
 
 void
 ts_lua_inject_http_intercept_api(lua_State * L)
@@ -153,6 +155,9 @@ ts_lua_http_intercept_process(ts_lua_http_ctx * http_ctx, TSVConn conn)
   // set up read.
   ts_lua_http_intercept_setup_read(ictx);
 
+  // set up write.
+  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);
@@ -160,7 +165,7 @@ ts_lua_http_intercept_process(ts_lua_http_ctx * http_ctx, TSVConn conn)
     lua_getglobal(l, TS_LUA_FUNCTION_HTTP_SERVER_INTERCEPT);
   }
 
-  ts_lua_http_intercept_run_coroutine(ictx);
+  ts_lua_http_intercept_run_coroutine(ictx, 0);
 
   TSMutexUnlock(mtxp);
 
@@ -185,7 +190,7 @@ 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)
 {
-  int ret;
+  int ret, n;
   TSMutex mtxp;
   ts_lua_http_intercept_ctx *ictx;
 
@@ -199,9 +204,10 @@ ts_lua_http_intercept_handler(TSCont contp, TSEvent event, void *edata)
     ret = ts_lua_http_intercept_process_write(event, ictx);
 
   } else {
-    mtxp = ictx->hctx->mctx->mutexp;
+    mtxp = ictx->mctx->mutexp;
+    n = (int64_t) edata & 0xFFFF;
     TSMutexLock(mtxp);
-    ret = ts_lua_http_intercept_run_coroutine(ictx);
+    ret = ts_lua_http_intercept_run_coroutine(ictx, n);
   }
 
   if (ret || (ictx->send_complete && ictx->recv_complete)) {
@@ -209,7 +215,7 @@ ts_lua_http_intercept_handler(TSCont contp, TSEvent event, void *edata)
     TSContDestroy(contp);
 
     if (!mtxp) {
-      mtxp = ictx->hctx->mctx->mutexp;
+      mtxp = ictx->mctx->mutexp;
       TSMutexLock(mtxp);
     }
 
@@ -223,24 +229,31 @@ ts_lua_http_intercept_handler(TSCont contp, TSEvent event, void *edata)
 }
 
 static int
-ts_lua_http_intercept_run_coroutine(ts_lua_http_intercept_ctx * ictx)
+ts_lua_http_intercept_run_coroutine(ts_lua_http_intercept_ctx * ictx, int n)
 {
   int ret;
-  const char *res;
-  size_t res_len;
+  int64_t avail;
+  int64_t done;
   lua_State *L;
 
   L = ictx->lua;
 
-  ret = lua_resume(L, 0);
+  ret = lua_resume(L, n);
 
   switch (ret) {
 
   case 0:                      // finished
-    res = lua_tolstring(L, -1, &res_len);
-    ts_lua_http_intercept_setup_write(ictx);
-    TSIOBufferWrite(ictx->output.buffer, res, res_len);
-    TSVIONBytesSet(ictx->output.vio, res_len);
+    avail = TSIOBufferReaderAvail(ictx->output.reader);
+    done = TSVIONDoneGet(ictx->output.vio);
+    TSVIONBytesSet(ictx->output.vio, avail + done);
+    ictx->all_ready = 1;
+
+    if (avail) {
+      TSVIOReenable(ictx->output.vio);
+
+    } else {
+      ictx->send_complete = 1;
+    }
     break;
 
   case 1:                      // yield
@@ -280,10 +293,31 @@ ts_lua_http_intercept_process_read(TSEvent event, ts_lua_http_intercept_ctx * ic
 static int
 ts_lua_http_intercept_process_write(TSEvent event, ts_lua_http_intercept_ctx * ictx)
 {
+  int64_t done, avail;
+
   switch (event) {
   case TS_EVENT_VCONN_WRITE_READY:
-    if (TSIOBufferReaderAvail(ictx->output.reader))
+
+    avail = TSIOBufferReaderAvail(ictx->output.reader);
+
+    if (ictx->all_ready) {
+      TSVIOReenable(ictx->output.vio);
+
+    } else if (ictx->to_flush > 0) {    // ts.flush()
+
+      done = TSVIONDoneGet(ictx->output.vio);
+
+      if (ictx->to_flush > done) {
+        TSVIOReenable(ictx->output.vio);
+
+      } else {                  // we had flush all the data we want
+        ictx->to_flush = 0;
+        ts_lua_flush_launch(ictx);      // wake up
+      }
+
+    } else if (avail > 0) {
       TSVIOReenable(ictx->output.vio);
+    }
     break;
 
   case TS_EVENT_VCONN_WRITE_COMPLETE:

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_log.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_log.c b/plugins/experimental/ts_lua/ts_lua_log.c
index 400589c..b0861c2 100644
--- a/plugins/experimental/ts_lua/ts_lua_log.c
+++ b/plugins/experimental/ts_lua/ts_lua_log.c
@@ -91,7 +91,7 @@ ts_lua_log_object_write(lua_State * L)
 
   text = luaL_checklstring(L, 1, &text_len);
   if (log) {
-    TSTextLogObjectWrite(log, "%s", text);
+    TSTextLogObjectWrite(log, (char *) text, NULL);
   } else {
     TSError("[%s] log is not exsited!", __FUNCTION__);
   }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_mgmt.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_mgmt.c b/plugins/experimental/ts_lua/ts_lua_mgmt.c
new file mode 100644
index 0000000..4cbbc83
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_mgmt.c
@@ -0,0 +1,106 @@
+/*
+  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_util.h"
+
+static int ts_lua_mgmt_get_int(lua_State * L);
+static int ts_lua_mgmt_get_counter(lua_State * L);
+static int ts_lua_mgmt_get_float(lua_State * L);
+static int ts_lua_mgmt_get_string(lua_State * L);
+
+
+void
+ts_lua_inject_mgmt_api(lua_State * L)
+{
+  lua_newtable(L);
+
+  lua_pushcfunction(L, ts_lua_mgmt_get_int);
+  lua_setfield(L, -2, "get_int");
+
+  lua_pushcfunction(L, ts_lua_mgmt_get_counter);
+  lua_setfield(L, -2, "get_counter");
+
+  lua_pushcfunction(L, ts_lua_mgmt_get_float);
+  lua_setfield(L, -2, "get_float");
+
+  lua_pushcfunction(L, ts_lua_mgmt_get_string);
+  lua_setfield(L, -2, "get_string");
+
+  lua_setfield(L, -2, "mgmt");
+}
+
+static int
+ts_lua_mgmt_get_int(lua_State * L)
+{
+  const char *name;
+  size_t name_len;
+  TSMgmtInt int_val;
+
+  name = luaL_checklstring(L, 1, &name_len);
+  TSMgmtIntGet(name, &int_val);
+
+  lua_pushinteger(L, int_val);
+
+  return 1;
+}
+
+static int
+ts_lua_mgmt_get_counter(lua_State * L)
+{
+  const char *name;
+  size_t name_len;
+  TSMgmtCounter counter_val;
+
+  name = luaL_checklstring(L, 1, &name_len);
+  TSMgmtCounterGet(name, &counter_val);
+
+  lua_pushinteger(L, counter_val);
+
+  return 1;
+}
+
+static int
+ts_lua_mgmt_get_float(lua_State * L)
+{
+  const char *name;
+  size_t name_len;
+  TSMgmtFloat float_val;
+
+  name = luaL_checklstring(L, 1, &name_len);
+  TSMgmtFloatGet(name, &float_val);
+
+  lua_pushnumber(L, float_val);
+
+  return 1;
+}
+
+static int
+ts_lua_mgmt_get_string(lua_State * L)
+{
+  const char *name;
+  size_t name_len;
+  TSMgmtString str_val;
+
+  name = luaL_checklstring(L, 1, &name_len);
+  TSMgmtStringGet(name, &str_val);
+
+  lua_pushstring(L, str_val);
+
+  return 1;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_mgmt.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_mgmt.h b/plugins/experimental/ts_lua/ts_lua_mgmt.h
new file mode 100644
index 0000000..d6d79e0
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_mgmt.h
@@ -0,0 +1,27 @@
+/*
+  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_MGMT_H
+#define _TS_LUA_MGMT_H
+
+#include "ts_lua_common.h"
+
+void ts_lua_inject_mgmt_api(lua_State * L);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/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 05a8155..4af4da0 100644
--- a/plugins/experimental/ts_lua/ts_lua_misc.c
+++ b/plugins/experimental/ts_lua/ts_lua_misc.c
@@ -24,6 +24,15 @@ 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_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
@@ -44,6 +53,14 @@ 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
@@ -62,7 +79,7 @@ ts_lua_debug(lua_State * L)
   const char *msg;
 
   msg = luaL_checkstring(L, 1);
-  TSDebug(TS_LUA_DEBUG_TAG, "%s", msg);
+  TSDebug(TS_LUA_DEBUG_TAG, msg, NULL);
   return 0;
 }
 
@@ -72,7 +89,7 @@ ts_lua_error(lua_State * L)
   const char *msg;
 
   msg = luaL_checkstring(L, 1);
-  TSError("%s", msg);
+  TSError(msg, NULL);
   return 0;
 }
 
@@ -80,11 +97,134 @@ static int
 ts_lua_sleep(lua_State * L)
 {
   int sec;
+  TSAction action;
+  TSCont contp;
+  ts_lua_http_intercept_item *node;
   ts_lua_http_intercept_ctx *ictx;
 
   ictx = ts_lua_get_http_intercept_ctx(L);
   sec = luaL_checknumber(L, 1);
 
-  TSContSchedule(ictx->contp, sec * 1000, TS_THREAD_POOL_DEFAULT);
+  contp = TSContCreate(ts_lua_sleep_handler, TSContMutexGet(ictx->contp));
+  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);
+
   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)
+{
+  ts_lua_http_intercept_item *item = TSContDataGet(contp);
+
+  ts_lua_sleep_cleanup(item);
+
+  TSContCall(item->ictx->contp, event, 0);
+
+  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);
+
+  return 0;
+}
+
+static int
+ts_lua_flush_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;
+}
+
+static int
+ts_lua_flush_handler(TSCont contp, TSEvent event, void *edata)
+{
+  ts_lua_http_intercept_item *item = TSContDataGet(contp);
+
+  ts_lua_flush_cleanup(item);
+
+  TSContCall(item->ictx->contp, event, 0);
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_package.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_package.c b/plugins/experimental/ts_lua/ts_lua_package.c
new file mode 100644
index 0000000..3d77f23
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_package.c
@@ -0,0 +1,297 @@
+/*
+  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_util.h"
+
+#define TS_LUA_MAX_PACKAGE_PATH_LEN     256
+#define TS_LUA_MAX_PACKAGE_NUM          64
+
+
+typedef struct
+{
+  size_t len;
+  char *name;
+} ts_lua_package_path;
+
+
+static int g_path_cnt = 0;
+static ts_lua_package_path g_path[TS_LUA_MAX_PACKAGE_NUM];
+
+static int g_cpath_cnt = 0;
+static ts_lua_package_path g_cpath[TS_LUA_MAX_PACKAGE_NUM];
+
+
+static int ts_lua_add_package_path(lua_State * L);
+static int ts_lua_add_package_cpath(lua_State * L);
+static int ts_lua_add_package_path_items(lua_State * L, ts_lua_package_path * pp, int n);
+static int ts_lua_add_package_cpath_items(lua_State * L, ts_lua_package_path * pp, int n);
+
+
+void
+ts_lua_inject_package_api(lua_State * L)
+{
+  /* ts.add_package_path() */
+  lua_pushcfunction(L, ts_lua_add_package_path);
+  lua_setfield(L, -2, "add_package_path");
+
+  /* ts.add_package_cpath(...) */
+  lua_pushcfunction(L, ts_lua_add_package_cpath);
+  lua_setfield(L, -2, "add_package_cpath");
+}
+
+static int
+ts_lua_add_package_path(lua_State * L)
+{
+  ts_lua_instance_conf *conf;
+  const char *data;
+  const char *ptr, *end, *hit;
+  size_t dlen;
+  size_t i, n, item_len;
+  ts_lua_package_path pp[TS_LUA_MAX_PACKAGE_NUM];
+  ts_lua_package_path *elt;
+
+  conf = ts_lua_get_instance_conf(L);
+  if (conf == NULL) {
+    return luaL_error(L, "cann't get the instance conf.");
+  }
+
+  data = luaL_checklstring(L, 1, &dlen);
+  end = data + dlen;
+
+  ptr = data;
+  n = 0;
+
+  while (ptr < end) {
+    hit = memchr(ptr, ';', end - ptr);
+
+    if (hit) {
+      item_len = hit - ptr;
+
+    } else {
+      item_len = end - ptr;
+    }
+
+    if (item_len > 0) {
+
+      for (i = 0; i < g_path_cnt; i++) {
+        if (g_path[i].len == item_len && memcmp(g_path[i].name, ptr, item_len) == 0)    // exist
+        {
+          break;
+        }
+      }
+
+      if (i >= g_path_cnt) {
+
+        if (n + i >= TS_LUA_MAX_PACKAGE_NUM)
+          return luaL_error(L, "extended package path number exceeds %d.", TS_LUA_MAX_PACKAGE_NUM);
+
+        pp[n].name = (char *) ptr;
+        pp[n].len = item_len;
+        n++;
+      }
+    }
+
+    ptr += item_len + 1;        // ??
+  }
+
+  if (n > 0) {
+    ts_lua_add_package_path_items(L, pp, n);
+
+    if (conf->_last) {
+      elt = &g_path[g_path_cnt];
+
+      for (i = 0; i < n; i++) {
+        elt->len = pp[i].len;
+        elt->name = (char *) TSmalloc(pp[i].len);
+        memcpy(elt->name, pp[i].name, pp[i].len);
+        elt++;
+      }
+
+      g_path_cnt += n;
+    }
+  }
+
+  return 0;
+}
+
+static int
+ts_lua_add_package_path_items(lua_State * L, ts_lua_package_path * pp, int n)
+{
+  int i, base;
+  const char *old_path;
+  char new_path[2048];
+  size_t old_path_len, new_path_len;
+
+  base = lua_gettop(L);
+
+  lua_getglobal(L, "package");
+
+  if (!lua_istable(L, -1)) {
+    return luaL_error(L, "'package' table does not exist.");
+  }
+
+  lua_getfield(L, -1, "path");  /* get old package.path */
+
+  old_path = lua_tolstring(L, -1, &old_path_len);
+  if (old_path[old_path_len - 1] == ';')
+    old_path_len--;
+
+  new_path_len = snprintf(new_path, sizeof(new_path) - 32, "%.*s", (int) old_path_len, old_path);
+
+  for (i = 0; i < n; i++) {
+    if (new_path_len + pp[i].len + 1 >= sizeof(new_path)) {
+      fprintf(stderr, "extended package.path is too long.\n");
+      return -1;
+    }
+
+    new_path[new_path_len++] = ';';
+    memcpy(new_path + new_path_len, pp[i].name, pp[i].len);
+    new_path_len += pp[i].len;
+  }
+
+  new_path[new_path_len] = 0;
+
+  lua_pushlstring(L, new_path, new_path_len);
+  lua_setfield(L, -3, "path");
+
+  lua_settop(L, base);
+
+  return 0;
+}
+
+
+static int
+ts_lua_add_package_cpath(lua_State * L)
+{
+  ts_lua_instance_conf *conf;
+  const char *data;
+  const char *ptr, *end, *hit;
+  size_t dlen;
+  size_t i, n, item_len;
+  ts_lua_package_path pp[TS_LUA_MAX_PACKAGE_NUM];
+  ts_lua_package_path *elt;
+
+  conf = ts_lua_get_instance_conf(L);
+  if (conf == NULL) {
+    return luaL_error(L, "cann't get the instance conf.");
+  }
+
+  data = luaL_checklstring(L, 1, &dlen);
+  end = data + dlen;
+
+  ptr = data;
+  n = 0;
+
+  while (ptr < end) {
+    hit = memchr(ptr, ';', end - ptr);
+
+    if (hit) {
+      item_len = hit - ptr;
+
+    } else {
+      item_len = end - ptr;
+    }
+
+    if (item_len > 0) {
+
+      for (i = 0; i < g_cpath_cnt; i++) {
+        if (g_cpath[i].len == item_len && memcmp(g_cpath[i].name, ptr, item_len) == 0)  // exist
+        {
+          break;
+        }
+      }
+
+      if (i >= g_cpath_cnt) {
+
+        if (n + i >= TS_LUA_MAX_PACKAGE_NUM)
+          return luaL_error(L, "extended package cpath number exceeds %d.", TS_LUA_MAX_PACKAGE_NUM);
+
+        pp[n].name = (char *) ptr;
+        pp[n].len = item_len;
+        n++;
+      }
+    }
+
+    ptr += item_len + 1;        // ??
+  }
+
+  if (n > 0) {
+    ts_lua_add_package_cpath_items(L, pp, n);
+
+    if (conf->_last) {
+      elt = &g_cpath[g_cpath_cnt];
+
+      for (i = 0; i < n; i++) {
+        elt->len = pp[i].len;
+        elt->name = (char *) TSmalloc(pp[i].len);
+        memcpy(elt->name, pp[i].name, pp[i].len);
+        elt++;
+      }
+
+      g_cpath_cnt += n;
+    }
+  }
+
+  return 0;
+}
+
+static int
+ts_lua_add_package_cpath_items(lua_State * L, ts_lua_package_path * pp, int n)
+{
+  int i, base;
+  const char *old_path;
+  char new_path[2048];
+  size_t old_path_len, new_path_len;
+
+  base = lua_gettop(L);
+
+  lua_getglobal(L, "package");
+
+  if (!lua_istable(L, -1)) {
+    return luaL_error(L, "'package' table does not exist.");
+  }
+
+  lua_getfield(L, -1, "cpath"); /* get old package.cpath */
+
+  old_path = lua_tolstring(L, -1, &old_path_len);
+  if (old_path[old_path_len - 1] == ';')
+    old_path_len--;
+
+  new_path_len = snprintf(new_path, sizeof(new_path) - 32, "%.*s", (int) old_path_len, old_path);
+
+  for (i = 0; i < n; i++) {
+    if (new_path_len + pp[i].len + 1 >= sizeof(new_path)) {
+      fprintf(stderr, "extended package.cpath is too long.\n");
+      return -1;
+    }
+
+    new_path[new_path_len++] = ';';
+    memcpy(new_path + new_path_len, pp[i].name, pp[i].len);
+    new_path_len += pp[i].len;
+  }
+
+  new_path[new_path_len] = 0;
+
+  lua_pushlstring(L, new_path, new_path_len);
+  lua_setfield(L, -3, "cpath");
+
+  lua_settop(L, base);
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_package.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_package.h b/plugins/experimental/ts_lua/ts_lua_package.h
new file mode 100644
index 0000000..ae5a467
--- /dev/null
+++ b/plugins/experimental/ts_lua/ts_lua_package.h
@@ -0,0 +1,27 @@
+/*
+  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_PACKAGE_H
+#define _TS_LUA_PACKAGE_H
+
+#include "ts_lua_common.h"
+
+void ts_lua_inject_package_api(lua_State * L);
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_remap.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_remap.c b/plugins/experimental/ts_lua/ts_lua_remap.c
index 1678d22..4df7ccf 100644
--- a/plugins/experimental/ts_lua/ts_lua_remap.c
+++ b/plugins/experimental/ts_lua/ts_lua_remap.c
@@ -22,27 +22,19 @@
 
 typedef enum
 {
-  TS_LUA_REMAP_NO_REMAP = 0,
-  TS_LUA_REMAP_DID_REMAP,
-  TS_LUA_REMAP_NO_REMAP_STOP,
-  TS_LUA_REMAP_DID_REMAP_STOP,
-  TS_LUA_REMAP_ERROR = -1,
+  TS_LUA_REMAP_NO_REMAP = TSREMAP_NO_REMAP,
+  TS_LUA_REMAP_DID_REMAP = TSREMAP_DID_REMAP,
+  TS_LUA_REMAP_NO_REMAP_STOP = TSREMAP_NO_REMAP_STOP,
+  TS_LUA_REMAP_DID_REMAP_STOP = TSREMAP_DID_REMAP_STOP,
+  TS_LUA_REMAP_ERROR = TSREMAP_ERROR
 } TSLuaRemapStatus;
 
-int ts_lua_remap_status_id[] = {
-  TS_LUA_REMAP_NO_REMAP,
-  TS_LUA_REMAP_DID_REMAP,
-  TS_LUA_REMAP_NO_REMAP_STOP,
-  TS_LUA_REMAP_DID_REMAP_STOP,
-  TS_LUA_REMAP_ERROR,
-};
-
-char *ts_lua_remap_status_string[] = {
-  "TS_LUA_REMAP_NO_REMAP",
-  "TS_LUA_REMAP_DID_REMAP",
-  "TS_LUA_REMAP_NO_REMAP_STOP",
-  "TS_LUA_REMAP_DID_REMAP_STOP",
-  "TS_LUA_REMAP_ERROR",
+ts_lua_var_item ts_lua_remap_status_vars[] = {
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_REMAP_NO_REMAP),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_REMAP_DID_REMAP),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_REMAP_NO_REMAP_STOP),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_REMAP_DID_REMAP_STOP),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_REMAP_ERROR)
 };
 
 
@@ -60,8 +52,8 @@ ts_lua_inject_remap_variables(lua_State * L)
 {
   int i;
 
-  for (i = 0; i < sizeof(ts_lua_remap_status_string) / sizeof(char *); i++) {
-    lua_pushinteger(L, ts_lua_remap_status_id[i]);
-    lua_setglobal(L, ts_lua_remap_status_string[i]);
+  for (i = 0; i < sizeof(ts_lua_remap_status_vars) / sizeof(ts_lua_var_item); i++) {
+    lua_pushinteger(L, ts_lua_remap_status_vars[i].nvar);
+    lua_setglobal(L, ts_lua_remap_status_vars[i].svar);
   }
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/83f16762/plugins/experimental/ts_lua/ts_lua_server_request.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_server_request.c b/plugins/experimental/ts_lua/ts_lua_server_request.c
index 3a38a6c..4f43ea4 100644
--- a/plugins/experimental/ts_lua/ts_lua_server_request.c
+++ b/plugins/experimental/ts_lua/ts_lua_server_request.c
@@ -20,14 +20,47 @@
 #include <arpa/inet.h>
 #include "ts_lua_util.h"
 
+#define TS_LUA_CHECK_SERVER_REQUEST_HDR(http_ctx)     \
+do {        \
+    if (!http_ctx->server_request_hdrp) {           \
+        if (TSHttpTxnServerReqGet(http_ctx->txnp,   \
+                    &http_ctx->server_request_bufp, \
+                    &http_ctx->server_request_hdrp) != TS_SUCCESS) {    \
+            return 0;   \
+        }   \
+    }   \
+} while(0)
+
+
+#define TS_LUA_CHECK_SERVER_REQUEST_URL(http_ctx)   \
+do {        \
+    if (!http_ctx->server_request_url) {            \
+        TS_LUA_CHECK_SERVER_REQUEST_HDR(http_ctx);  \
+        if (TSHttpHdrUrlGet(http_ctx->server_request_bufp,   \
+                    http_ctx->server_request_hdrp, \
+                    &http_ctx->server_request_url) != TS_SUCCESS) {    \
+            return 0;   \
+        }   \
+    }   \
+} while(0)
+
+
 static void ts_lua_inject_server_request_header_api(lua_State * L);
+static void ts_lua_inject_server_request_headers_api(lua_State * L);
 static void ts_lua_inject_server_request_get_header_size_api(lua_State * L);
 static void ts_lua_inject_server_request_get_body_size_api(lua_State * L);
+static void ts_lua_inject_server_request_uri_api(lua_State * L);
+static void ts_lua_inject_server_request_uri_args_api(lua_State * L);
 
 static int ts_lua_server_request_header_get(lua_State * L);
 static int ts_lua_server_request_header_set(lua_State * L);
+static int ts_lua_server_request_get_headers(lua_State * L);
 static int ts_lua_server_request_get_header_size(lua_State * L);
 static int ts_lua_server_request_get_body_size(lua_State * L);
+static int ts_lua_server_request_get_uri(lua_State * L);
+static int ts_lua_server_request_set_uri(lua_State * L);
+static int ts_lua_server_request_set_uri_args(lua_State * L);
+static int ts_lua_server_request_get_uri_args(lua_State * L);
 
 
 void
@@ -36,9 +69,13 @@ ts_lua_inject_server_request_api(lua_State * L)
   lua_newtable(L);
 
   ts_lua_inject_server_request_header_api(L);
+  ts_lua_inject_server_request_headers_api(L);
   ts_lua_inject_server_request_get_header_size_api(L);
   ts_lua_inject_server_request_get_body_size_api(L);
 
+  ts_lua_inject_server_request_uri_api(L);
+  ts_lua_inject_server_request_uri_args_api(L);
+
   lua_setfield(L, -2, "server_request");
 }
 
@@ -168,6 +205,54 @@ ts_lua_server_request_header_set(lua_State * L)
 }
 
 static void
+ts_lua_inject_server_request_headers_api(lua_State * L)
+{
+  lua_pushcfunction(L, ts_lua_server_request_get_headers);
+  lua_setfield(L, -2, "get_headers");
+}
+
+static int
+ts_lua_server_request_get_headers(lua_State * L)
+{
+  const char *name;
+  const char *value;
+  int name_len;
+  int value_len;
+  TSMLoc field_loc;
+  TSMLoc next_field_loc;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_REQUEST_HDR(http_ctx);
+
+  lua_newtable(L);
+
+  field_loc = TSMimeHdrFieldGet(http_ctx->server_request_bufp, http_ctx->server_request_hdrp, 0);
+
+  while (field_loc) {
+
+    name = TSMimeHdrFieldNameGet(http_ctx->server_request_bufp, http_ctx->server_request_hdrp, field_loc, &name_len);
+    if (name && name_len) {
+
+      value =
+        TSMimeHdrFieldValueStringGet(http_ctx->server_request_bufp, http_ctx->server_request_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(http_ctx->server_request_bufp, http_ctx->server_request_hdrp, field_loc);
+    TSHandleMLocRelease(http_ctx->server_request_bufp, http_ctx->server_request_hdrp, field_loc);
+    field_loc = next_field_loc;
+  }
+
+  return 1;
+}
+
+static void
 ts_lua_inject_server_request_get_header_size_api(lua_State * L)
 {
   lua_pushcfunction(L, ts_lua_server_request_get_header_size);
@@ -208,3 +293,112 @@ ts_lua_server_request_get_body_size(lua_State * L)
 
   return 1;
 }
+
+static void
+ts_lua_inject_server_request_uri_api(lua_State * L)
+{
+  lua_pushcfunction(L, ts_lua_server_request_set_uri);
+  lua_setfield(L, -2, "set_uri");
+
+  lua_pushcfunction(L, ts_lua_server_request_get_uri);
+  lua_setfield(L, -2, "get_uri");
+}
+
+static int
+ts_lua_server_request_get_uri(lua_State * L)
+{
+  char uri[TS_LUA_MAX_URL_LENGTH];
+  const char *path;
+  int path_len;
+  int uri_len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_REQUEST_URL(http_ctx);
+
+  path = TSUrlPathGet(http_ctx->server_request_bufp, http_ctx->server_request_url, &path_len);
+
+  uri_len = snprintf(uri, TS_LUA_MAX_URL_LENGTH, "/%.*s", path_len, path);
+
+  lua_pushlstring(L, uri, uri_len);
+
+  return 1;
+}
+
+static int
+ts_lua_server_request_set_uri(lua_State * L)
+{
+  const char *path;
+  size_t path_len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_REQUEST_URL(http_ctx);
+
+  path = luaL_checklstring(L, 1, &path_len);
+
+  if (*path == '/') {
+    path++;
+    path_len--;
+  }
+
+  TSUrlPathSet(http_ctx->server_request_bufp, http_ctx->server_request_url, path, path_len);
+
+  return 0;
+}
+
+static void
+ts_lua_inject_server_request_uri_args_api(lua_State * L)
+{
+  lua_pushcfunction(L, ts_lua_server_request_set_uri_args);
+  lua_setfield(L, -2, "set_uri_args");
+
+  lua_pushcfunction(L, ts_lua_server_request_get_uri_args);
+  lua_setfield(L, -2, "get_uri_args");
+}
+
+static int
+ts_lua_server_request_set_uri_args(lua_State * L)
+{
+  const char *param;
+  size_t param_len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_REQUEST_URL(http_ctx);
+
+  param = luaL_checklstring(L, 1, &param_len);
+  TSUrlHttpQuerySet(http_ctx->server_request_bufp, http_ctx->server_request_url, param, param_len);
+
+  return 0;
+}
+
+static int
+ts_lua_server_request_get_uri_args(lua_State * L)
+{
+  const char *param;
+  int param_len;
+
+  ts_lua_http_ctx *http_ctx;
+
+  http_ctx = ts_lua_get_http_ctx(L);
+
+  TS_LUA_CHECK_SERVER_REQUEST_URL(http_ctx);
+
+  param = TSUrlHttpQueryGet(http_ctx->server_request_bufp, http_ctx->server_request_url, &param_len);
+
+  if (param && param_len > 0) {
+    lua_pushlstring(L, param, param_len);
+
+  } else {
+    lua_pushnil(L);
+  }
+
+  return 1;
+}