You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@trafficserver.apache.org by James Peach <jp...@apache.org> on 2014/04/16 18:03:58 UTC

Re: git commit: TS-2555 add global plugin support for ts_lua plugin

On Apr 15, 2014, at 10:34 PM, kichan@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master 9f509d401 -> 904e1f7c2
> 
> 
> TS-2555 add global plugin support for 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/904e1f7c
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/904e1f7c
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/904e1f7c
> 
> Branch: refs/heads/master
> Commit: 904e1f7c2995dbb4a8afea7cd99e32f0fd844f46
> Parents: 9f509d4
> Author: Kit Chan <ki...@apache.org>
> Authored: Tue Apr 15 22:30:01 2014 -0700
> Committer: Kit Chan <ki...@apache.org>
> Committed: Tue Apr 15 22:30:01 2014 -0700
> 
> ----------------------------------------------------------------------
> plugins/experimental/ts_lua/ts_lua.c        | 155 ++++++++++++++++++++++-
> plugins/experimental/ts_lua/ts_lua_common.h |   8 ++
> plugins/experimental/ts_lua/ts_lua_util.c   |   9 ++
> 3 files changed, 170 insertions(+), 2 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ca78665..32cb58c 100644
> --- a/plugins/experimental/ts_lua/ts_lua.c
> +++ b/plugins/experimental/ts_lua/ts_lua.c
> @@ -26,9 +26,10 @@
> #define TS_LUA_MAX_STATE_COUNT                  2048
> 
> static volatile int32_t ts_lua_http_next_id = 0;
> +static volatile int32_t ts_lua_g_http_next_id = 0;
> 
> ts_lua_main_ctx         *ts_lua_main_ctx_array;
> -
> +ts_lua_main_ctx         *ts_lua_g_main_ctx_array;
> 
> TSReturnCode
> TSRemapInit(TSRemapInterface *api_info, char * errbuf ATS_UNUSED , int errbuf_size ATS_UNUSED )
> @@ -119,7 +120,7 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
>     http_ctx->client_request_bufp = rri->requestBufp;
>     http_ctx->client_request_hdrp = rri->requestHdrp;
>     http_ctx->client_request_url = rri->requestUrl;
> -
> +    http_ctx->remap = 1;
>     l = http_ctx->lua;
> 
>     lua_getglobal(l, TS_LUA_FUNCTION_REMAP);
> @@ -146,3 +147,153 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
>     return ret;
> }
> 
> +static int
> +globalHookHandler(TSCont contp, TSEvent event, void *edata) {
> +  TSHttpTxn txnp = (TSHttpTxn) edata;
> +
> +  int ret = 0;
> +  int64_t req_id;
> +
> +  lua_State *l;
> +  TSCont txn_contp;
> +
> +  ts_lua_main_ctx     *main_ctx;
> +  ts_lua_http_ctx     *http_ctx;
> +
> +  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);
> +  main_ctx = &ts_lua_g_main_ctx_array[req_id%TS_LUA_MAX_STATE_COUNT];

Won't this cause different hook invocations on the same transaction to execute in different lua states?

> +
> +  TSMutexLock(main_ctx->mutexp);
> +
> +  http_ctx = ts_lua_create_http_ctx(main_ctx, conf);
> +  http_ctx->txnp = txnp;
> +  http_ctx->remap = 0;
> +
> +  TSMBuffer bufp;
> +  TSMLoc hdr_loc;
> +  TSMLoc url_loc;
> +
> +  if(TSHttpTxnClientReqGet(txnp, &bufp, &hdr_loc) == TS_SUCCESS) {
> +    http_ctx->client_request_bufp = bufp;
> +    http_ctx->client_request_hdrp = hdr_loc;
> +    if (TSHttpHdrUrlGet(bufp, hdr_loc, &url_loc) == TS_SUCCESS) {
> +      http_ctx->client_request_url = url_loc;
> +    }
> +  }
> +
> +  if(!http_ctx->client_request_hdrp) {
> +    TSMutexUnlock(main_ctx->mutexp);
> +    TSHttpTxnReenable(txnp,TS_EVENT_HTTP_CONTINUE);
> +    return 0;
> +  }
> + 
> +  l = http_ctx->lua;
> +
> +  switch (event) {
> +  case TS_EVENT_HTTP_READ_REQUEST_HDR:
> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_REQUEST);
> +    break;
> +
> +  case TS_EVENT_HTTP_SEND_REQUEST_HDR:
> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_REQUEST);
> +    break;
> +
> +  case TS_EVENT_HTTP_READ_RESPONSE_HDR:
> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_RESPONSE);
> +    break;
> +
> +  case TS_EVENT_HTTP_SEND_RESPONSE_HDR:
> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_RESPONSE);
> +    break;
> +
> +  case TS_EVENT_HTTP_CACHE_LOOKUP_COMPLETE:
> +    lua_getglobal(l, TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE);
> +    break;
> +
> +  default:
> +    TSMutexUnlock(main_ctx->mutexp);
> +    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
> +    return 0;
> +    break;
> +  }
> +
> +  if (lua_type(l, -1) != LUA_TFUNCTION) {
> +      TSMutexUnlock(main_ctx->mutexp);
> +      TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
> +      return 0;
> +  }
> +
> +  txn_contp = TSContCreate(ts_lua_http_cont_handler, NULL);
> +  TSContDataSet(txn_contp, http_ctx);
> +  http_ctx->main_contp = txn_contp;
> +
> +  if (lua_pcall(l, 0, 1, 0) != 0) {
> +      fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
> +  }
> +
> +  ret = lua_tointeger(l, -1);
> +  lua_pop(l, 1);
> +
> +  TSHttpTxnHookAdd(txnp, TS_HTTP_TXN_CLOSE_HOOK, txn_contp);
> +
> +  TSMutexUnlock(main_ctx->mutexp);
> +
> +  TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
> +  return ret;
> +}
> +
> +void
> +TSPluginInit(int argc, const char *argv[]) {
> +    int ret = 0;
> +    ts_lua_g_main_ctx_array = TSmalloc(sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
> +    memset(ts_lua_g_main_ctx_array, 0, sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
> +    
> +    ret = ts_lua_create_vm(ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
> +    
> +    if (ret) {
> +      ts_lua_destroy_vm(ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
> +      TSfree(ts_lua_g_main_ctx_array);
> +      return;
> +    }
> +    
> +    if (argc < 2) {
> +      TSError("[%s] lua script file required !!", __FUNCTION__);
> +      return;
> +    }
> +    
> +    if (strlen(argv[1]) >= TS_LUA_MAX_SCRIPT_FNAME_LENGTH - 16) {
> +      TSError("[%s] lua script file name too long !!", __FUNCTION__);
> +      return;
> +    }
> +    
> +    ts_lua_instance_conf *conf = TSmalloc(sizeof(ts_lua_instance_conf));
> +    if (!conf) {
> +      TSError("[%s] TSmalloc failed !!", __FUNCTION__);
> +      return;
> +    }
> +    
> +    sprintf(conf->script, "%s", argv[1]);
> +    
> +    ret = ts_lua_add_module(conf, ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT, argc-1, (char**)&argv[1]);
> +    
> +    if (ret != 0) {
> +      TSError("[%s] ts_lua_add_module failed", __FUNCTION__);
> +      return;
> +    }
> +
> +    TSCont global_contp = TSContCreate(globalHookHandler, NULL);
> +    if (!global_contp) {
> +      TSError("[%s] Could not create global continuation", __FUNCTION__);
> +      return;
> +    }
> +    TSContDataSet(global_contp, conf);
> +
> +    TSHttpHookAdd(TS_HTTP_READ_REQUEST_HDR_HOOK, global_contp);
> +    TSHttpHookAdd(TS_HTTP_SEND_REQUEST_HDR_HOOK, global_contp);
> +    TSHttpHookAdd(TS_HTTP_READ_RESPONSE_HDR_HOOK, global_contp);
> +    TSHttpHookAdd(TS_HTTP_SEND_RESPONSE_HDR_HOOK, global_contp);
> +    TSHttpHookAdd(TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK, global_contp);

So there's a fixed number of hooks, and they are *always* registered simply by loading the plugin? Simply loading the plugin is going to do a bunch of work on every event even if there's nothing registered on that hook AFAICT.


> + 
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 66ba35e..4c84e3f 100644
> --- a/plugins/experimental/ts_lua/ts_lua_common.h
> +++ b/plugins/experimental/ts_lua/ts_lua_common.h
> @@ -41,6 +41,12 @@
> #define TS_LUA_FUNCTION_READ_RESPONSE           "do_read_response"
> #define TS_LUA_FUNCTION_SEND_RESPONSE           "do_send_response"
> 
> +#define TS_LUA_FUNCTION_G_SEND_REQUEST "do_global_send_request"
> +#define TS_LUA_FUNCTION_G_READ_REQUEST "do_global_read_request"
> +#define TS_LUA_FUNCTION_G_SEND_RESPONSE "do_global_send_response"
> +#define TS_LUA_FUNCTION_G_READ_RESPONSE "do_global_read_response"
> +#define TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE "do_global_cache_lookup_complete"
> +
> #define TS_LUA_MAX_SCRIPT_FNAME_LENGTH      1024
> #define TS_LUA_MAX_URL_LENGTH               2048
> 
> @@ -85,6 +91,8 @@ typedef struct {
>     int         intercept_type;
>     int         ref;
> 
> +    int         remap;
> +
> } ts_lua_http_ctx;
> 
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ea20fa9..81231ee 100644
> --- a/plugins/experimental/ts_lua/ts_lua_util.c
> +++ b/plugins/experimental/ts_lua/ts_lua_util.c
> @@ -280,6 +280,15 @@ ts_lua_destroy_http_ctx(ts_lua_http_ctx* http_ctx)
> 
>     main_ctx = http_ctx->mctx;
> 
> +    if(!http_ctx->remap) {
> +      if(http_ctx->client_request_bufp) {
> +        if(http_ctx->client_request_url) {
> +          TSHandleMLocRelease(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, http_ctx->client_request_url);
> +        } 
> +        TSHandleMLocRelease(http_ctx->client_request_bufp, TS_NULL_MLOC, http_ctx->client_request_hdrp);
> +      }
> +    } 
> +
>     if (http_ctx->server_request_bufp) {
>         TSHandleMLocRelease(http_ctx->server_request_bufp, TS_NULL_MLOC, http_ctx->server_request_hdrp);
>     }
> 


Re: git commit: TS-2555 add global plugin support for ts_lua plugin

Posted by James Peach <jp...@apache.org>.
On Apr 16, 2014, at 7:19 PM, 李刚(阙寒) <qu...@taobao.com> wrote:

> Maybe we can use the same lua_States in both remap and global plugins.

They have different lifetimes, and script authors probably expect some degree of isolation between plugin instances. Take a look at how the lua plugin solves this problem; you might be able to do something similar.

> 在 2014-4-17,上午12:03,James Peach <jp...@apache.org> 写道:
> 
>> On Apr 15, 2014, at 10:34 PM, kichan@apache.org wrote:
>> 
>>> Repository: trafficserver
>>> Updated Branches:
>>> refs/heads/master 9f509d401 -> 904e1f7c2
>>> 
>>> 
>>> TS-2555 add global plugin support for 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/904e1f7c
>>> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/904e1f7c
>>> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/904e1f7c
>>> 
>>> Branch: refs/heads/master
>>> Commit: 904e1f7c2995dbb4a8afea7cd99e32f0fd844f46
>>> Parents: 9f509d4
>>> Author: Kit Chan <ki...@apache.org>
>>> Authored: Tue Apr 15 22:30:01 2014 -0700
>>> Committer: Kit Chan <ki...@apache.org>
>>> Committed: Tue Apr 15 22:30:01 2014 -0700
>>> 
>>> ----------------------------------------------------------------------
>>> plugins/experimental/ts_lua/ts_lua.c        | 155 ++++++++++++++++++++++-
>>> plugins/experimental/ts_lua/ts_lua_common.h |   8 ++
>>> plugins/experimental/ts_lua/ts_lua_util.c   |   9 ++
>>> 3 files changed, 170 insertions(+), 2 deletions(-)
>>> ----------------------------------------------------------------------
>>> 
>>> 
>>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ca78665..32cb58c 100644
>>> --- a/plugins/experimental/ts_lua/ts_lua.c
>>> +++ b/plugins/experimental/ts_lua/ts_lua.c
>>> @@ -26,9 +26,10 @@
>>> #define TS_LUA_MAX_STATE_COUNT                  2048
>>> 
>>> static volatile int32_t ts_lua_http_next_id = 0;
>>> +static volatile int32_t ts_lua_g_http_next_id = 0;
>>> 
>>> ts_lua_main_ctx         *ts_lua_main_ctx_array;
>>> -
>>> +ts_lua_main_ctx         *ts_lua_g_main_ctx_array;
>>> 
>>> TSReturnCode
>>> TSRemapInit(TSRemapInterface *api_info, char * errbuf ATS_UNUSED , int errbuf_size ATS_UNUSED )
>>> @@ -119,7 +120,7 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
>>>   http_ctx->client_request_bufp = rri->requestBufp;
>>>   http_ctx->client_request_hdrp = rri->requestHdrp;
>>>   http_ctx->client_request_url = rri->requestUrl;
>>> -
>>> +    http_ctx->remap = 1;
>>>   l = http_ctx->lua;
>>> 
>>>   lua_getglobal(l, TS_LUA_FUNCTION_REMAP);
>>> @@ -146,3 +147,153 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
>>>   return ret;
>>> }
>>> 
>>> +static int
>>> +globalHookHandler(TSCont contp, TSEvent event, void *edata) {
>>> +  TSHttpTxn txnp = (TSHttpTxn) edata;
>>> +
>>> +  int ret = 0;
>>> +  int64_t req_id;
>>> +
>>> +  lua_State *l;
>>> +  TSCont txn_contp;
>>> +
>>> +  ts_lua_main_ctx     *main_ctx;
>>> +  ts_lua_http_ctx     *http_ctx;
>>> +
>>> +  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);
>>> +  main_ctx = &ts_lua_g_main_ctx_array[req_id%TS_LUA_MAX_STATE_COUNT];
>> 
>> Won't this cause different hook invocations on the same transaction to execute in different lua states?
>> 
>>> +
>>> +  TSMutexLock(main_ctx->mutexp);
>>> +
>>> +  http_ctx = ts_lua_create_http_ctx(main_ctx, conf);
>>> +  http_ctx->txnp = txnp;
>>> +  http_ctx->remap = 0;
>>> +
>>> +  TSMBuffer bufp;
>>> +  TSMLoc hdr_loc;
>>> +  TSMLoc url_loc;
>>> +
>>> +  if(TSHttpTxnClientReqGet(txnp, &bufp, &hdr_loc) == TS_SUCCESS) {
>>> +    http_ctx->client_request_bufp = bufp;
>>> +    http_ctx->client_request_hdrp = hdr_loc;
>>> +    if (TSHttpHdrUrlGet(bufp, hdr_loc, &url_loc) == TS_SUCCESS) {
>>> +      http_ctx->client_request_url = url_loc;
>>> +    }
>>> +  }
>>> +
>>> +  if(!http_ctx->client_request_hdrp) {
>>> +    TSMutexUnlock(main_ctx->mutexp);
>>> +    TSHttpTxnReenable(txnp,TS_EVENT_HTTP_CONTINUE);
>>> +    return 0;
>>> +  }
>>> + 
>>> +  l = http_ctx->lua;
>>> +
>>> +  switch (event) {
>>> +  case TS_EVENT_HTTP_READ_REQUEST_HDR:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_REQUEST);
>>> +    break;
>>> +
>>> +  case TS_EVENT_HTTP_SEND_REQUEST_HDR:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_REQUEST);
>>> +    break;
>>> +
>>> +  case TS_EVENT_HTTP_READ_RESPONSE_HDR:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_RESPONSE);
>>> +    break;
>>> +
>>> +  case TS_EVENT_HTTP_SEND_RESPONSE_HDR:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_RESPONSE);
>>> +    break;
>>> +
>>> +  case TS_EVENT_HTTP_CACHE_LOOKUP_COMPLETE:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE);
>>> +    break;
>>> +
>>> +  default:
>>> +    TSMutexUnlock(main_ctx->mutexp);
>>> +    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>>> +    return 0;
>>> +    break;
>>> +  }
>>> +
>>> +  if (lua_type(l, -1) != LUA_TFUNCTION) {
>>> +      TSMutexUnlock(main_ctx->mutexp);
>>> +      TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>>> +      return 0;
>>> +  }
>>> +
>>> +  txn_contp = TSContCreate(ts_lua_http_cont_handler, NULL);
>>> +  TSContDataSet(txn_contp, http_ctx);
>>> +  http_ctx->main_contp = txn_contp;
>>> +
>>> +  if (lua_pcall(l, 0, 1, 0) != 0) {
>>> +      fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
>>> +  }
>>> +
>>> +  ret = lua_tointeger(l, -1);
>>> +  lua_pop(l, 1);
>>> +
>>> +  TSHttpTxnHookAdd(txnp, TS_HTTP_TXN_CLOSE_HOOK, txn_contp);
>>> +
>>> +  TSMutexUnlock(main_ctx->mutexp);
>>> +
>>> +  TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>>> +  return ret;
>>> +}
>>> +
>>> +void
>>> +TSPluginInit(int argc, const char *argv[]) {
>>> +    int ret = 0;
>>> +    ts_lua_g_main_ctx_array = TSmalloc(sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
>>> +    memset(ts_lua_g_main_ctx_array, 0, sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
>>> +    
>>> +    ret = ts_lua_create_vm(ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
>>> +    
>>> +    if (ret) {
>>> +      ts_lua_destroy_vm(ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
>>> +      TSfree(ts_lua_g_main_ctx_array);
>>> +      return;
>>> +    }
>>> +    
>>> +    if (argc < 2) {
>>> +      TSError("[%s] lua script file required !!", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +    
>>> +    if (strlen(argv[1]) >= TS_LUA_MAX_SCRIPT_FNAME_LENGTH - 16) {
>>> +      TSError("[%s] lua script file name too long !!", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +    
>>> +    ts_lua_instance_conf *conf = TSmalloc(sizeof(ts_lua_instance_conf));
>>> +    if (!conf) {
>>> +      TSError("[%s] TSmalloc failed !!", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +    
>>> +    sprintf(conf->script, "%s", argv[1]);
>>> +    
>>> +    ret = ts_lua_add_module(conf, ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT, argc-1, (char**)&argv[1]);
>>> +    
>>> +    if (ret != 0) {
>>> +      TSError("[%s] ts_lua_add_module failed", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +
>>> +    TSCont global_contp = TSContCreate(globalHookHandler, NULL);
>>> +    if (!global_contp) {
>>> +      TSError("[%s] Could not create global continuation", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +    TSContDataSet(global_contp, conf);
>>> +
>>> +    TSHttpHookAdd(TS_HTTP_READ_REQUEST_HDR_HOOK, global_contp);
>>> +    TSHttpHookAdd(TS_HTTP_SEND_REQUEST_HDR_HOOK, global_contp);
>>> +    TSHttpHookAdd(TS_HTTP_READ_RESPONSE_HDR_HOOK, global_contp);
>>> +    TSHttpHookAdd(TS_HTTP_SEND_RESPONSE_HDR_HOOK, global_contp);
>>> +    TSHttpHookAdd(TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK, global_contp);
>> 
>> So there's a fixed number of hooks, and they are *always* registered simply by loading the plugin? Simply loading the plugin is going to do a bunch of work on every event even if there's nothing registered on that hook AFAICT.
>> 
>> 
>>> + 
>>> +}
>>> 
>>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 66ba35e..4c84e3f 100644
>>> --- a/plugins/experimental/ts_lua/ts_lua_common.h
>>> +++ b/plugins/experimental/ts_lua/ts_lua_common.h
>>> @@ -41,6 +41,12 @@
>>> #define TS_LUA_FUNCTION_READ_RESPONSE           "do_read_response"
>>> #define TS_LUA_FUNCTION_SEND_RESPONSE           "do_send_response"
>>> 
>>> +#define TS_LUA_FUNCTION_G_SEND_REQUEST "do_global_send_request"
>>> +#define TS_LUA_FUNCTION_G_READ_REQUEST "do_global_read_request"
>>> +#define TS_LUA_FUNCTION_G_SEND_RESPONSE "do_global_send_response"
>>> +#define TS_LUA_FUNCTION_G_READ_RESPONSE "do_global_read_response"
>>> +#define TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE "do_global_cache_lookup_complete"
>>> +
>>> #define TS_LUA_MAX_SCRIPT_FNAME_LENGTH      1024
>>> #define TS_LUA_MAX_URL_LENGTH               2048
>>> 
>>> @@ -85,6 +91,8 @@ typedef struct {
>>>   int         intercept_type;
>>>   int         ref;
>>> 
>>> +    int         remap;
>>> +
>>> } ts_lua_http_ctx;
>>> 
>>> 
>>> 
>>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ea20fa9..81231ee 100644
>>> --- a/plugins/experimental/ts_lua/ts_lua_util.c
>>> +++ b/plugins/experimental/ts_lua/ts_lua_util.c
>>> @@ -280,6 +280,15 @@ ts_lua_destroy_http_ctx(ts_lua_http_ctx* http_ctx)
>>> 
>>>   main_ctx = http_ctx->mctx;
>>> 
>>> +    if(!http_ctx->remap) {
>>> +      if(http_ctx->client_request_bufp) {
>>> +        if(http_ctx->client_request_url) {
>>> +          TSHandleMLocRelease(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, http_ctx->client_request_url);
>>> +        } 
>>> +        TSHandleMLocRelease(http_ctx->client_request_bufp, TS_NULL_MLOC, http_ctx->client_request_hdrp);
>>> +      }
>>> +    } 
>>> +
>>>   if (http_ctx->server_request_bufp) {
>>>       TSHandleMLocRelease(http_ctx->server_request_bufp, TS_NULL_MLOC, http_ctx->server_request_hdrp);
>>>   }
>>> 
> 


Re: git commit: TS-2555 add global plugin support for ts_lua plugin

Posted by James Peach <jp...@apache.org>.
On Apr 17, 2014, at 9:01 AM, Leif Hedstrom <zw...@apache.org> wrote:

> 
> On Apr 16, 2014, at 8:19 PM, 李刚(阙寒) <qu...@taobao.com> wrote:
> 
>> Maybe we can use the same lua_States in both remap and global plugins.
>> 
>> 
> 
> 
> Not exactly related, but one thing that really bothers me with the new plugin is how in a remap rule (do_remap()) it always saves away the context, and creates a new TXN continuation to free it at the end of the TXN. That seems like a waste for many (not all) remap type plugins, which just do rewrites of URLs (why does the CTX have to persist?).
> 
> One thing that we should consider now that Lua is part of the core is to add stuff to e.g. the HttpSM. For example, maybe we can have Lua CTX member(s) there, and do automatic cleanup if set (i.e. avoid the “cleanup” continuation)? That would imply adding new plugin APIs, such as TSLuaContextSet(txnp, ctx) and ctx = TSLuaContextGet(txnp).

If you want to pull Lua into core, why would you need a lua plugin at all? Surely the core would just execute lua code in that case. If the config is lua too, then remapping could be a matter of just attaching a lua function to a URL matching expression.

> But, regardless, start thinking about how Lua should integrate better with the core :). We can / should assume that Lua is always available, and long term, the consensus was to try to move most (if not all) into the core.
> 
> Cheers,
> 
> — Leif
> 


Re: git commit: TS-2555 add global plugin support for ts_lua plugin

Posted by James Peach <jp...@apache.org>.
On Apr 17, 2014, at 9:01 AM, Leif Hedstrom <zw...@apache.org> wrote:

> 
> On Apr 16, 2014, at 8:19 PM, 李刚(阙寒) <qu...@taobao.com> wrote:
> 
>> Maybe we can use the same lua_States in both remap and global plugins.
>> 
>> 
> 
> 
> Not exactly related, but one thing that really bothers me with the new plugin is how in a remap rule (do_remap()) it always saves away the context, and creates a new TXN continuation to free it at the end of the TXN. That seems like a waste for many (not all) remap type plugins, which just do rewrites of URLs (why does the CTX have to persist?).
> 
> One thing that we should consider now that Lua is part of the core is to add stuff to e.g. the HttpSM. For example, maybe we can have Lua CTX member(s) there, and do automatic cleanup if set (i.e. avoid the “cleanup” continuation)? That would imply adding new plugin APIs, such as TSLuaContextSet(txnp, ctx) and ctx = TSLuaContextGet(txnp).

If you want to pull Lua into core, why would you need a lua plugin at all? Surely the core would just execute lua code in that case. If the config is lua too, then remapping could be a matter of just attaching a lua function to a URL matching expression.

> But, regardless, start thinking about how Lua should integrate better with the core :). We can / should assume that Lua is always available, and long term, the consensus was to try to move most (if not all) into the core.
> 
> Cheers,
> 
> — Leif
> 


Re: git commit: TS-2555 add global plugin support for ts_lua plugin

Posted by Leif Hedstrom <zw...@apache.org>.
On Apr 16, 2014, at 8:19 PM, 李刚(阙寒) <qu...@taobao.com> wrote:

> Maybe we can use the same lua_States in both remap and global plugins.
> 
> 


Not exactly related, but one thing that really bothers me with the new plugin is how in a remap rule (do_remap()) it always saves away the context, and creates a new TXN continuation to free it at the end of the TXN. That seems like a waste for many (not all) remap type plugins, which just do rewrites of URLs (why does the CTX have to persist?).

One thing that we should consider now that Lua is part of the core is to add stuff to e.g. the HttpSM. For example, maybe we can have Lua CTX member(s) there, and do automatic cleanup if set (i.e. avoid the “cleanup” continuation)? That would imply adding new plugin APIs, such as TSLuaContextSet(txnp, ctx) and ctx = TSLuaContextGet(txnp).

But, regardless, start thinking about how Lua should integrate better with the core :). We can / should assume that Lua is always available, and long term, the consensus was to try to move most (if not all) into the core.

Cheers,

— Leif


Re: git commit: TS-2555 add global plugin support for ts_lua plugin

Posted by James Peach <jp...@apache.org>.
On Apr 16, 2014, at 7:19 PM, 李刚(阙寒) <qu...@taobao.com> wrote:

> Maybe we can use the same lua_States in both remap and global plugins.

They have different lifetimes, and script authors probably expect some degree of isolation between plugin instances. Take a look at how the lua plugin solves this problem; you might be able to do something similar.

> 在 2014-4-17,上午12:03,James Peach <jp...@apache.org> 写道:
> 
>> On Apr 15, 2014, at 10:34 PM, kichan@apache.org wrote:
>> 
>>> Repository: trafficserver
>>> Updated Branches:
>>> refs/heads/master 9f509d401 -> 904e1f7c2
>>> 
>>> 
>>> TS-2555 add global plugin support for 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/904e1f7c
>>> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/904e1f7c
>>> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/904e1f7c
>>> 
>>> Branch: refs/heads/master
>>> Commit: 904e1f7c2995dbb4a8afea7cd99e32f0fd844f46
>>> Parents: 9f509d4
>>> Author: Kit Chan <ki...@apache.org>
>>> Authored: Tue Apr 15 22:30:01 2014 -0700
>>> Committer: Kit Chan <ki...@apache.org>
>>> Committed: Tue Apr 15 22:30:01 2014 -0700
>>> 
>>> ----------------------------------------------------------------------
>>> plugins/experimental/ts_lua/ts_lua.c        | 155 ++++++++++++++++++++++-
>>> plugins/experimental/ts_lua/ts_lua_common.h |   8 ++
>>> plugins/experimental/ts_lua/ts_lua_util.c   |   9 ++
>>> 3 files changed, 170 insertions(+), 2 deletions(-)
>>> ----------------------------------------------------------------------
>>> 
>>> 
>>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ca78665..32cb58c 100644
>>> --- a/plugins/experimental/ts_lua/ts_lua.c
>>> +++ b/plugins/experimental/ts_lua/ts_lua.c
>>> @@ -26,9 +26,10 @@
>>> #define TS_LUA_MAX_STATE_COUNT                  2048
>>> 
>>> static volatile int32_t ts_lua_http_next_id = 0;
>>> +static volatile int32_t ts_lua_g_http_next_id = 0;
>>> 
>>> ts_lua_main_ctx         *ts_lua_main_ctx_array;
>>> -
>>> +ts_lua_main_ctx         *ts_lua_g_main_ctx_array;
>>> 
>>> TSReturnCode
>>> TSRemapInit(TSRemapInterface *api_info, char * errbuf ATS_UNUSED , int errbuf_size ATS_UNUSED )
>>> @@ -119,7 +120,7 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
>>>   http_ctx->client_request_bufp = rri->requestBufp;
>>>   http_ctx->client_request_hdrp = rri->requestHdrp;
>>>   http_ctx->client_request_url = rri->requestUrl;
>>> -
>>> +    http_ctx->remap = 1;
>>>   l = http_ctx->lua;
>>> 
>>>   lua_getglobal(l, TS_LUA_FUNCTION_REMAP);
>>> @@ -146,3 +147,153 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
>>>   return ret;
>>> }
>>> 
>>> +static int
>>> +globalHookHandler(TSCont contp, TSEvent event, void *edata) {
>>> +  TSHttpTxn txnp = (TSHttpTxn) edata;
>>> +
>>> +  int ret = 0;
>>> +  int64_t req_id;
>>> +
>>> +  lua_State *l;
>>> +  TSCont txn_contp;
>>> +
>>> +  ts_lua_main_ctx     *main_ctx;
>>> +  ts_lua_http_ctx     *http_ctx;
>>> +
>>> +  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);
>>> +  main_ctx = &ts_lua_g_main_ctx_array[req_id%TS_LUA_MAX_STATE_COUNT];
>> 
>> Won't this cause different hook invocations on the same transaction to execute in different lua states?
>> 
>>> +
>>> +  TSMutexLock(main_ctx->mutexp);
>>> +
>>> +  http_ctx = ts_lua_create_http_ctx(main_ctx, conf);
>>> +  http_ctx->txnp = txnp;
>>> +  http_ctx->remap = 0;
>>> +
>>> +  TSMBuffer bufp;
>>> +  TSMLoc hdr_loc;
>>> +  TSMLoc url_loc;
>>> +
>>> +  if(TSHttpTxnClientReqGet(txnp, &bufp, &hdr_loc) == TS_SUCCESS) {
>>> +    http_ctx->client_request_bufp = bufp;
>>> +    http_ctx->client_request_hdrp = hdr_loc;
>>> +    if (TSHttpHdrUrlGet(bufp, hdr_loc, &url_loc) == TS_SUCCESS) {
>>> +      http_ctx->client_request_url = url_loc;
>>> +    }
>>> +  }
>>> +
>>> +  if(!http_ctx->client_request_hdrp) {
>>> +    TSMutexUnlock(main_ctx->mutexp);
>>> +    TSHttpTxnReenable(txnp,TS_EVENT_HTTP_CONTINUE);
>>> +    return 0;
>>> +  }
>>> + 
>>> +  l = http_ctx->lua;
>>> +
>>> +  switch (event) {
>>> +  case TS_EVENT_HTTP_READ_REQUEST_HDR:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_REQUEST);
>>> +    break;
>>> +
>>> +  case TS_EVENT_HTTP_SEND_REQUEST_HDR:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_REQUEST);
>>> +    break;
>>> +
>>> +  case TS_EVENT_HTTP_READ_RESPONSE_HDR:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_RESPONSE);
>>> +    break;
>>> +
>>> +  case TS_EVENT_HTTP_SEND_RESPONSE_HDR:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_RESPONSE);
>>> +    break;
>>> +
>>> +  case TS_EVENT_HTTP_CACHE_LOOKUP_COMPLETE:
>>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE);
>>> +    break;
>>> +
>>> +  default:
>>> +    TSMutexUnlock(main_ctx->mutexp);
>>> +    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>>> +    return 0;
>>> +    break;
>>> +  }
>>> +
>>> +  if (lua_type(l, -1) != LUA_TFUNCTION) {
>>> +      TSMutexUnlock(main_ctx->mutexp);
>>> +      TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>>> +      return 0;
>>> +  }
>>> +
>>> +  txn_contp = TSContCreate(ts_lua_http_cont_handler, NULL);
>>> +  TSContDataSet(txn_contp, http_ctx);
>>> +  http_ctx->main_contp = txn_contp;
>>> +
>>> +  if (lua_pcall(l, 0, 1, 0) != 0) {
>>> +      fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
>>> +  }
>>> +
>>> +  ret = lua_tointeger(l, -1);
>>> +  lua_pop(l, 1);
>>> +
>>> +  TSHttpTxnHookAdd(txnp, TS_HTTP_TXN_CLOSE_HOOK, txn_contp);
>>> +
>>> +  TSMutexUnlock(main_ctx->mutexp);
>>> +
>>> +  TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>>> +  return ret;
>>> +}
>>> +
>>> +void
>>> +TSPluginInit(int argc, const char *argv[]) {
>>> +    int ret = 0;
>>> +    ts_lua_g_main_ctx_array = TSmalloc(sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
>>> +    memset(ts_lua_g_main_ctx_array, 0, sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
>>> +    
>>> +    ret = ts_lua_create_vm(ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
>>> +    
>>> +    if (ret) {
>>> +      ts_lua_destroy_vm(ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
>>> +      TSfree(ts_lua_g_main_ctx_array);
>>> +      return;
>>> +    }
>>> +    
>>> +    if (argc < 2) {
>>> +      TSError("[%s] lua script file required !!", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +    
>>> +    if (strlen(argv[1]) >= TS_LUA_MAX_SCRIPT_FNAME_LENGTH - 16) {
>>> +      TSError("[%s] lua script file name too long !!", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +    
>>> +    ts_lua_instance_conf *conf = TSmalloc(sizeof(ts_lua_instance_conf));
>>> +    if (!conf) {
>>> +      TSError("[%s] TSmalloc failed !!", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +    
>>> +    sprintf(conf->script, "%s", argv[1]);
>>> +    
>>> +    ret = ts_lua_add_module(conf, ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT, argc-1, (char**)&argv[1]);
>>> +    
>>> +    if (ret != 0) {
>>> +      TSError("[%s] ts_lua_add_module failed", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +
>>> +    TSCont global_contp = TSContCreate(globalHookHandler, NULL);
>>> +    if (!global_contp) {
>>> +      TSError("[%s] Could not create global continuation", __FUNCTION__);
>>> +      return;
>>> +    }
>>> +    TSContDataSet(global_contp, conf);
>>> +
>>> +    TSHttpHookAdd(TS_HTTP_READ_REQUEST_HDR_HOOK, global_contp);
>>> +    TSHttpHookAdd(TS_HTTP_SEND_REQUEST_HDR_HOOK, global_contp);
>>> +    TSHttpHookAdd(TS_HTTP_READ_RESPONSE_HDR_HOOK, global_contp);
>>> +    TSHttpHookAdd(TS_HTTP_SEND_RESPONSE_HDR_HOOK, global_contp);
>>> +    TSHttpHookAdd(TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK, global_contp);
>> 
>> So there's a fixed number of hooks, and they are *always* registered simply by loading the plugin? Simply loading the plugin is going to do a bunch of work on every event even if there's nothing registered on that hook AFAICT.
>> 
>> 
>>> + 
>>> +}
>>> 
>>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 66ba35e..4c84e3f 100644
>>> --- a/plugins/experimental/ts_lua/ts_lua_common.h
>>> +++ b/plugins/experimental/ts_lua/ts_lua_common.h
>>> @@ -41,6 +41,12 @@
>>> #define TS_LUA_FUNCTION_READ_RESPONSE           "do_read_response"
>>> #define TS_LUA_FUNCTION_SEND_RESPONSE           "do_send_response"
>>> 
>>> +#define TS_LUA_FUNCTION_G_SEND_REQUEST "do_global_send_request"
>>> +#define TS_LUA_FUNCTION_G_READ_REQUEST "do_global_read_request"
>>> +#define TS_LUA_FUNCTION_G_SEND_RESPONSE "do_global_send_response"
>>> +#define TS_LUA_FUNCTION_G_READ_RESPONSE "do_global_read_response"
>>> +#define TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE "do_global_cache_lookup_complete"
>>> +
>>> #define TS_LUA_MAX_SCRIPT_FNAME_LENGTH      1024
>>> #define TS_LUA_MAX_URL_LENGTH               2048
>>> 
>>> @@ -85,6 +91,8 @@ typedef struct {
>>>   int         intercept_type;
>>>   int         ref;
>>> 
>>> +    int         remap;
>>> +
>>> } ts_lua_http_ctx;
>>> 
>>> 
>>> 
>>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ea20fa9..81231ee 100644
>>> --- a/plugins/experimental/ts_lua/ts_lua_util.c
>>> +++ b/plugins/experimental/ts_lua/ts_lua_util.c
>>> @@ -280,6 +280,15 @@ ts_lua_destroy_http_ctx(ts_lua_http_ctx* http_ctx)
>>> 
>>>   main_ctx = http_ctx->mctx;
>>> 
>>> +    if(!http_ctx->remap) {
>>> +      if(http_ctx->client_request_bufp) {
>>> +        if(http_ctx->client_request_url) {
>>> +          TSHandleMLocRelease(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, http_ctx->client_request_url);
>>> +        } 
>>> +        TSHandleMLocRelease(http_ctx->client_request_bufp, TS_NULL_MLOC, http_ctx->client_request_hdrp);
>>> +      }
>>> +    } 
>>> +
>>>   if (http_ctx->server_request_bufp) {
>>>       TSHandleMLocRelease(http_ctx->server_request_bufp, TS_NULL_MLOC, http_ctx->server_request_hdrp);
>>>   }
>>> 
> 


Re: git commit: TS-2555 add global plugin support for ts_lua plugin

Posted by Shu Kit Chan <ch...@gmail.com>.
I will take some time to refine this over the weekend. Thanks.


On Wed, Apr 16, 2014 at 7:19 PM, 李刚(阙寒) <qu...@taobao.com> wrote:

> Maybe we can use the same lua_States in both remap and global plugins.
>
> 在 2014-4-17,上午12:03,James Peach <jp...@apache.org> 写道:
>
> > On Apr 15, 2014, at 10:34 PM, kichan@apache.org wrote:
> >
> >> Repository: trafficserver
> >> Updated Branches:
> >> refs/heads/master 9f509d401 -> 904e1f7c2
> >>
> >>
> >> TS-2555 add global plugin support for 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/904e1f7c
> >> Tree:
> http://git-wip-us.apache.org/repos/asf/trafficserver/tree/904e1f7c
> >> Diff:
> http://git-wip-us.apache.org/repos/asf/trafficserver/diff/904e1f7c
> >>
> >> Branch: refs/heads/master
> >> Commit: 904e1f7c2995dbb4a8afea7cd99e32f0fd844f46
> >> Parents: 9f509d4
> >> Author: Kit Chan <ki...@apache.org>
> >> Authored: Tue Apr 15 22:30:01 2014 -0700
> >> Committer: Kit Chan <ki...@apache.org>
> >> Committed: Tue Apr 15 22:30:01 2014 -0700
> >>
> >> ----------------------------------------------------------------------
> >> plugins/experimental/ts_lua/ts_lua.c        | 155
> ++++++++++++++++++++++-
> >> plugins/experimental/ts_lua/ts_lua_common.h |   8 ++
> >> plugins/experimental/ts_lua/ts_lua_util.c   |   9 ++
> >> 3 files changed, 170 insertions(+), 2 deletions(-)
> >> ----------------------------------------------------------------------
> >>
> >>
> >>
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ca78665..32cb58c 100644
> >> --- a/plugins/experimental/ts_lua/ts_lua.c
> >> +++ b/plugins/experimental/ts_lua/ts_lua.c
> >> @@ -26,9 +26,10 @@
> >> #define TS_LUA_MAX_STATE_COUNT                  2048
> >>
> >> static volatile int32_t ts_lua_http_next_id = 0;
> >> +static volatile int32_t ts_lua_g_http_next_id = 0;
> >>
> >> ts_lua_main_ctx         *ts_lua_main_ctx_array;
> >> -
> >> +ts_lua_main_ctx         *ts_lua_g_main_ctx_array;
> >>
> >> TSReturnCode
> >> TSRemapInit(TSRemapInterface *api_info, char * errbuf ATS_UNUSED , int
> errbuf_size ATS_UNUSED )
> >> @@ -119,7 +120,7 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh,
> TSRemapRequestInfo *rri)
> >>    http_ctx->client_request_bufp = rri->requestBufp;
> >>    http_ctx->client_request_hdrp = rri->requestHdrp;
> >>    http_ctx->client_request_url = rri->requestUrl;
> >> -
> >> +    http_ctx->remap = 1;
> >>    l = http_ctx->lua;
> >>
> >>    lua_getglobal(l, TS_LUA_FUNCTION_REMAP);
> >> @@ -146,3 +147,153 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh,
> TSRemapRequestInfo *rri)
> >>    return ret;
> >> }
> >>
> >> +static int
> >> +globalHookHandler(TSCont contp, TSEvent event, void *edata) {
> >> +  TSHttpTxn txnp = (TSHttpTxn) edata;
> >> +
> >> +  int ret = 0;
> >> +  int64_t req_id;
> >> +
> >> +  lua_State *l;
> >> +  TSCont txn_contp;
> >> +
> >> +  ts_lua_main_ctx     *main_ctx;
> >> +  ts_lua_http_ctx     *http_ctx;
> >> +
> >> +  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);
> >> +  main_ctx = &ts_lua_g_main_ctx_array[req_id%TS_LUA_MAX_STATE_COUNT];
> >
> > Won't this cause different hook invocations on the same transaction to
> execute in different lua states?
> >
> >> +
> >> +  TSMutexLock(main_ctx->mutexp);
> >> +
> >> +  http_ctx = ts_lua_create_http_ctx(main_ctx, conf);
> >> +  http_ctx->txnp = txnp;
> >> +  http_ctx->remap = 0;
> >> +
> >> +  TSMBuffer bufp;
> >> +  TSMLoc hdr_loc;
> >> +  TSMLoc url_loc;
> >> +
> >> +  if(TSHttpTxnClientReqGet(txnp, &bufp, &hdr_loc) == TS_SUCCESS) {
> >> +    http_ctx->client_request_bufp = bufp;
> >> +    http_ctx->client_request_hdrp = hdr_loc;
> >> +    if (TSHttpHdrUrlGet(bufp, hdr_loc, &url_loc) == TS_SUCCESS) {
> >> +      http_ctx->client_request_url = url_loc;
> >> +    }
> >> +  }
> >> +
> >> +  if(!http_ctx->client_request_hdrp) {
> >> +    TSMutexUnlock(main_ctx->mutexp);
> >> +    TSHttpTxnReenable(txnp,TS_EVENT_HTTP_CONTINUE);
> >> +    return 0;
> >> +  }
> >> +
> >> +  l = http_ctx->lua;
> >> +
> >> +  switch (event) {
> >> +  case TS_EVENT_HTTP_READ_REQUEST_HDR:
> >> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_REQUEST);
> >> +    break;
> >> +
> >> +  case TS_EVENT_HTTP_SEND_REQUEST_HDR:
> >> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_REQUEST);
> >> +    break;
> >> +
> >> +  case TS_EVENT_HTTP_READ_RESPONSE_HDR:
> >> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_RESPONSE);
> >> +    break;
> >> +
> >> +  case TS_EVENT_HTTP_SEND_RESPONSE_HDR:
> >> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_RESPONSE);
> >> +    break;
> >> +
> >> +  case TS_EVENT_HTTP_CACHE_LOOKUP_COMPLETE:
> >> +    lua_getglobal(l, TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE);
> >> +    break;
> >> +
> >> +  default:
> >> +    TSMutexUnlock(main_ctx->mutexp);
> >> +    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
> >> +    return 0;
> >> +    break;
> >> +  }
> >> +
> >> +  if (lua_type(l, -1) != LUA_TFUNCTION) {
> >> +      TSMutexUnlock(main_ctx->mutexp);
> >> +      TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
> >> +      return 0;
> >> +  }
> >> +
> >> +  txn_contp = TSContCreate(ts_lua_http_cont_handler, NULL);
> >> +  TSContDataSet(txn_contp, http_ctx);
> >> +  http_ctx->main_contp = txn_contp;
> >> +
> >> +  if (lua_pcall(l, 0, 1, 0) != 0) {
> >> +      fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
> >> +  }
> >> +
> >> +  ret = lua_tointeger(l, -1);
> >> +  lua_pop(l, 1);
> >> +
> >> +  TSHttpTxnHookAdd(txnp, TS_HTTP_TXN_CLOSE_HOOK, txn_contp);
> >> +
> >> +  TSMutexUnlock(main_ctx->mutexp);
> >> +
> >> +  TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
> >> +  return ret;
> >> +}
> >> +
> >> +void
> >> +TSPluginInit(int argc, const char *argv[]) {
> >> +    int ret = 0;
> >> +    ts_lua_g_main_ctx_array = TSmalloc(sizeof(ts_lua_main_ctx) *
> TS_LUA_MAX_STATE_COUNT);
> >> +    memset(ts_lua_g_main_ctx_array, 0, sizeof(ts_lua_main_ctx) *
> TS_LUA_MAX_STATE_COUNT);
> >> +
> >> +    ret = ts_lua_create_vm(ts_lua_g_main_ctx_array,
> TS_LUA_MAX_STATE_COUNT);
> >> +
> >> +    if (ret) {
> >> +      ts_lua_destroy_vm(ts_lua_g_main_ctx_array,
> TS_LUA_MAX_STATE_COUNT);
> >> +      TSfree(ts_lua_g_main_ctx_array);
> >> +      return;
> >> +    }
> >> +
> >> +    if (argc < 2) {
> >> +      TSError("[%s] lua script file required !!", __FUNCTION__);
> >> +      return;
> >> +    }
> >> +
> >> +    if (strlen(argv[1]) >= TS_LUA_MAX_SCRIPT_FNAME_LENGTH - 16) {
> >> +      TSError("[%s] lua script file name too long !!", __FUNCTION__);
> >> +      return;
> >> +    }
> >> +
> >> +    ts_lua_instance_conf *conf =
> TSmalloc(sizeof(ts_lua_instance_conf));
> >> +    if (!conf) {
> >> +      TSError("[%s] TSmalloc failed !!", __FUNCTION__);
> >> +      return;
> >> +    }
> >> +
> >> +    sprintf(conf->script, "%s", argv[1]);
> >> +
> >> +    ret = ts_lua_add_module(conf, ts_lua_g_main_ctx_array,
> TS_LUA_MAX_STATE_COUNT, argc-1, (char**)&argv[1]);
> >> +
> >> +    if (ret != 0) {
> >> +      TSError("[%s] ts_lua_add_module failed", __FUNCTION__);
> >> +      return;
> >> +    }
> >> +
> >> +    TSCont global_contp = TSContCreate(globalHookHandler, NULL);
> >> +    if (!global_contp) {
> >> +      TSError("[%s] Could not create global continuation",
> __FUNCTION__);
> >> +      return;
> >> +    }
> >> +    TSContDataSet(global_contp, conf);
> >> +
> >> +    TSHttpHookAdd(TS_HTTP_READ_REQUEST_HDR_HOOK, global_contp);
> >> +    TSHttpHookAdd(TS_HTTP_SEND_REQUEST_HDR_HOOK, global_contp);
> >> +    TSHttpHookAdd(TS_HTTP_READ_RESPONSE_HDR_HOOK, global_contp);
> >> +    TSHttpHookAdd(TS_HTTP_SEND_RESPONSE_HDR_HOOK, global_contp);
> >> +    TSHttpHookAdd(TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK, global_contp);
> >
> > So there's a fixed number of hooks, and they are *always* registered
> simply by loading the plugin? Simply loading the plugin is going to do a
> bunch of work on every event even if there's nothing registered on that
> hook AFAICT.
> >
> >
> >> +
> >> +}
> >>
> >>
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 66ba35e..4c84e3f 100644
> >> --- a/plugins/experimental/ts_lua/ts_lua_common.h
> >> +++ b/plugins/experimental/ts_lua/ts_lua_common.h
> >> @@ -41,6 +41,12 @@
> >> #define TS_LUA_FUNCTION_READ_RESPONSE           "do_read_response"
> >> #define TS_LUA_FUNCTION_SEND_RESPONSE           "do_send_response"
> >>
> >> +#define TS_LUA_FUNCTION_G_SEND_REQUEST "do_global_send_request"
> >> +#define TS_LUA_FUNCTION_G_READ_REQUEST "do_global_read_request"
> >> +#define TS_LUA_FUNCTION_G_SEND_RESPONSE "do_global_send_response"
> >> +#define TS_LUA_FUNCTION_G_READ_RESPONSE "do_global_read_response"
> >> +#define TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE
> "do_global_cache_lookup_complete"
> >> +
> >> #define TS_LUA_MAX_SCRIPT_FNAME_LENGTH      1024
> >> #define TS_LUA_MAX_URL_LENGTH               2048
> >>
> >> @@ -85,6 +91,8 @@ typedef struct {
> >>    int         intercept_type;
> >>    int         ref;
> >>
> >> +    int         remap;
> >> +
> >> } ts_lua_http_ctx;
> >>
> >>
> >>
> >>
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ea20fa9..81231ee 100644
> >> --- a/plugins/experimental/ts_lua/ts_lua_util.c
> >> +++ b/plugins/experimental/ts_lua/ts_lua_util.c
> >> @@ -280,6 +280,15 @@ ts_lua_destroy_http_ctx(ts_lua_http_ctx* http_ctx)
> >>
> >>    main_ctx = http_ctx->mctx;
> >>
> >> +    if(!http_ctx->remap) {
> >> +      if(http_ctx->client_request_bufp) {
> >> +        if(http_ctx->client_request_url) {
> >> +          TSHandleMLocRelease(http_ctx->client_request_bufp,
> http_ctx->client_request_hdrp, http_ctx->client_request_url);
> >> +        }
> >> +        TSHandleMLocRelease(http_ctx->client_request_bufp,
> TS_NULL_MLOC, http_ctx->client_request_hdrp);
> >> +      }
> >> +    }
> >> +
> >>    if (http_ctx->server_request_bufp) {
> >>        TSHandleMLocRelease(http_ctx->server_request_bufp, TS_NULL_MLOC,
> http_ctx->server_request_hdrp);
> >>    }
> >>
>
>

Re: git commit: TS-2555 add global plugin support for ts_lua plugin

Posted by Leif Hedstrom <zw...@apache.org>.
On Apr 16, 2014, at 8:19 PM, 李刚(阙寒) <qu...@taobao.com> wrote:

> Maybe we can use the same lua_States in both remap and global plugins.
> 
> 


Not exactly related, but one thing that really bothers me with the new plugin is how in a remap rule (do_remap()) it always saves away the context, and creates a new TXN continuation to free it at the end of the TXN. That seems like a waste for many (not all) remap type plugins, which just do rewrites of URLs (why does the CTX have to persist?).

One thing that we should consider now that Lua is part of the core is to add stuff to e.g. the HttpSM. For example, maybe we can have Lua CTX member(s) there, and do automatic cleanup if set (i.e. avoid the “cleanup” continuation)? That would imply adding new plugin APIs, such as TSLuaContextSet(txnp, ctx) and ctx = TSLuaContextGet(txnp).

But, regardless, start thinking about how Lua should integrate better with the core :). We can / should assume that Lua is always available, and long term, the consensus was to try to move most (if not all) into the core.

Cheers,

— Leif


Re: git commit: TS-2555 add global plugin support for ts_lua plugin

Posted by "李刚(阙寒)" <qu...@taobao.com>.
Maybe we can use the same lua_States in both remap and global plugins.

在 2014-4-17,上午12:03,James Peach <jp...@apache.org> 写道:

> On Apr 15, 2014, at 10:34 PM, kichan@apache.org wrote:
> 
>> Repository: trafficserver
>> Updated Branches:
>> refs/heads/master 9f509d401 -> 904e1f7c2
>> 
>> 
>> TS-2555 add global plugin support for 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/904e1f7c
>> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/904e1f7c
>> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/904e1f7c
>> 
>> Branch: refs/heads/master
>> Commit: 904e1f7c2995dbb4a8afea7cd99e32f0fd844f46
>> Parents: 9f509d4
>> Author: Kit Chan <ki...@apache.org>
>> Authored: Tue Apr 15 22:30:01 2014 -0700
>> Committer: Kit Chan <ki...@apache.org>
>> Committed: Tue Apr 15 22:30:01 2014 -0700
>> 
>> ----------------------------------------------------------------------
>> plugins/experimental/ts_lua/ts_lua.c        | 155 ++++++++++++++++++++++-
>> plugins/experimental/ts_lua/ts_lua_common.h |   8 ++
>> plugins/experimental/ts_lua/ts_lua_util.c   |   9 ++
>> 3 files changed, 170 insertions(+), 2 deletions(-)
>> ----------------------------------------------------------------------
>> 
>> 
>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ca78665..32cb58c 100644
>> --- a/plugins/experimental/ts_lua/ts_lua.c
>> +++ b/plugins/experimental/ts_lua/ts_lua.c
>> @@ -26,9 +26,10 @@
>> #define TS_LUA_MAX_STATE_COUNT                  2048
>> 
>> static volatile int32_t ts_lua_http_next_id = 0;
>> +static volatile int32_t ts_lua_g_http_next_id = 0;
>> 
>> ts_lua_main_ctx         *ts_lua_main_ctx_array;
>> -
>> +ts_lua_main_ctx         *ts_lua_g_main_ctx_array;
>> 
>> TSReturnCode
>> TSRemapInit(TSRemapInterface *api_info, char * errbuf ATS_UNUSED , int errbuf_size ATS_UNUSED )
>> @@ -119,7 +120,7 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
>>    http_ctx->client_request_bufp = rri->requestBufp;
>>    http_ctx->client_request_hdrp = rri->requestHdrp;
>>    http_ctx->client_request_url = rri->requestUrl;
>> -
>> +    http_ctx->remap = 1;
>>    l = http_ctx->lua;
>> 
>>    lua_getglobal(l, TS_LUA_FUNCTION_REMAP);
>> @@ -146,3 +147,153 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
>>    return ret;
>> }
>> 
>> +static int
>> +globalHookHandler(TSCont contp, TSEvent event, void *edata) {
>> +  TSHttpTxn txnp = (TSHttpTxn) edata;
>> +
>> +  int ret = 0;
>> +  int64_t req_id;
>> +
>> +  lua_State *l;
>> +  TSCont txn_contp;
>> +
>> +  ts_lua_main_ctx     *main_ctx;
>> +  ts_lua_http_ctx     *http_ctx;
>> +
>> +  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);
>> +  main_ctx = &ts_lua_g_main_ctx_array[req_id%TS_LUA_MAX_STATE_COUNT];
> 
> Won't this cause different hook invocations on the same transaction to execute in different lua states?
> 
>> +
>> +  TSMutexLock(main_ctx->mutexp);
>> +
>> +  http_ctx = ts_lua_create_http_ctx(main_ctx, conf);
>> +  http_ctx->txnp = txnp;
>> +  http_ctx->remap = 0;
>> +
>> +  TSMBuffer bufp;
>> +  TSMLoc hdr_loc;
>> +  TSMLoc url_loc;
>> +
>> +  if(TSHttpTxnClientReqGet(txnp, &bufp, &hdr_loc) == TS_SUCCESS) {
>> +    http_ctx->client_request_bufp = bufp;
>> +    http_ctx->client_request_hdrp = hdr_loc;
>> +    if (TSHttpHdrUrlGet(bufp, hdr_loc, &url_loc) == TS_SUCCESS) {
>> +      http_ctx->client_request_url = url_loc;
>> +    }
>> +  }
>> +
>> +  if(!http_ctx->client_request_hdrp) {
>> +    TSMutexUnlock(main_ctx->mutexp);
>> +    TSHttpTxnReenable(txnp,TS_EVENT_HTTP_CONTINUE);
>> +    return 0;
>> +  }
>> + 
>> +  l = http_ctx->lua;
>> +
>> +  switch (event) {
>> +  case TS_EVENT_HTTP_READ_REQUEST_HDR:
>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_REQUEST);
>> +    break;
>> +
>> +  case TS_EVENT_HTTP_SEND_REQUEST_HDR:
>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_REQUEST);
>> +    break;
>> +
>> +  case TS_EVENT_HTTP_READ_RESPONSE_HDR:
>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_READ_RESPONSE);
>> +    break;
>> +
>> +  case TS_EVENT_HTTP_SEND_RESPONSE_HDR:
>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_SEND_RESPONSE);
>> +    break;
>> +
>> +  case TS_EVENT_HTTP_CACHE_LOOKUP_COMPLETE:
>> +    lua_getglobal(l, TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE);
>> +    break;
>> +
>> +  default:
>> +    TSMutexUnlock(main_ctx->mutexp);
>> +    TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>> +    return 0;
>> +    break;
>> +  }
>> +
>> +  if (lua_type(l, -1) != LUA_TFUNCTION) {
>> +      TSMutexUnlock(main_ctx->mutexp);
>> +      TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>> +      return 0;
>> +  }
>> +
>> +  txn_contp = TSContCreate(ts_lua_http_cont_handler, NULL);
>> +  TSContDataSet(txn_contp, http_ctx);
>> +  http_ctx->main_contp = txn_contp;
>> +
>> +  if (lua_pcall(l, 0, 1, 0) != 0) {
>> +      fprintf(stderr, "lua_pcall failed: %s\n", lua_tostring(l, -1));
>> +  }
>> +
>> +  ret = lua_tointeger(l, -1);
>> +  lua_pop(l, 1);
>> +
>> +  TSHttpTxnHookAdd(txnp, TS_HTTP_TXN_CLOSE_HOOK, txn_contp);
>> +
>> +  TSMutexUnlock(main_ctx->mutexp);
>> +
>> +  TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
>> +  return ret;
>> +}
>> +
>> +void
>> +TSPluginInit(int argc, const char *argv[]) {
>> +    int ret = 0;
>> +    ts_lua_g_main_ctx_array = TSmalloc(sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
>> +    memset(ts_lua_g_main_ctx_array, 0, sizeof(ts_lua_main_ctx) * TS_LUA_MAX_STATE_COUNT);
>> +    
>> +    ret = ts_lua_create_vm(ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
>> +    
>> +    if (ret) {
>> +      ts_lua_destroy_vm(ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT);
>> +      TSfree(ts_lua_g_main_ctx_array);
>> +      return;
>> +    }
>> +    
>> +    if (argc < 2) {
>> +      TSError("[%s] lua script file required !!", __FUNCTION__);
>> +      return;
>> +    }
>> +    
>> +    if (strlen(argv[1]) >= TS_LUA_MAX_SCRIPT_FNAME_LENGTH - 16) {
>> +      TSError("[%s] lua script file name too long !!", __FUNCTION__);
>> +      return;
>> +    }
>> +    
>> +    ts_lua_instance_conf *conf = TSmalloc(sizeof(ts_lua_instance_conf));
>> +    if (!conf) {
>> +      TSError("[%s] TSmalloc failed !!", __FUNCTION__);
>> +      return;
>> +    }
>> +    
>> +    sprintf(conf->script, "%s", argv[1]);
>> +    
>> +    ret = ts_lua_add_module(conf, ts_lua_g_main_ctx_array, TS_LUA_MAX_STATE_COUNT, argc-1, (char**)&argv[1]);
>> +    
>> +    if (ret != 0) {
>> +      TSError("[%s] ts_lua_add_module failed", __FUNCTION__);
>> +      return;
>> +    }
>> +
>> +    TSCont global_contp = TSContCreate(globalHookHandler, NULL);
>> +    if (!global_contp) {
>> +      TSError("[%s] Could not create global continuation", __FUNCTION__);
>> +      return;
>> +    }
>> +    TSContDataSet(global_contp, conf);
>> +
>> +    TSHttpHookAdd(TS_HTTP_READ_REQUEST_HDR_HOOK, global_contp);
>> +    TSHttpHookAdd(TS_HTTP_SEND_REQUEST_HDR_HOOK, global_contp);
>> +    TSHttpHookAdd(TS_HTTP_READ_RESPONSE_HDR_HOOK, global_contp);
>> +    TSHttpHookAdd(TS_HTTP_SEND_RESPONSE_HDR_HOOK, global_contp);
>> +    TSHttpHookAdd(TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK, global_contp);
> 
> So there's a fixed number of hooks, and they are *always* registered simply by loading the plugin? Simply loading the plugin is going to do a bunch of work on every event even if there's nothing registered on that hook AFAICT.
> 
> 
>> + 
>> +}
>> 
>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 66ba35e..4c84e3f 100644
>> --- a/plugins/experimental/ts_lua/ts_lua_common.h
>> +++ b/plugins/experimental/ts_lua/ts_lua_common.h
>> @@ -41,6 +41,12 @@
>> #define TS_LUA_FUNCTION_READ_RESPONSE           "do_read_response"
>> #define TS_LUA_FUNCTION_SEND_RESPONSE           "do_send_response"
>> 
>> +#define TS_LUA_FUNCTION_G_SEND_REQUEST "do_global_send_request"
>> +#define TS_LUA_FUNCTION_G_READ_REQUEST "do_global_read_request"
>> +#define TS_LUA_FUNCTION_G_SEND_RESPONSE "do_global_send_response"
>> +#define TS_LUA_FUNCTION_G_READ_RESPONSE "do_global_read_response"
>> +#define TS_LUA_FUNCTION_G_CACHE_LOOKUP_COMPLETE "do_global_cache_lookup_complete"
>> +
>> #define TS_LUA_MAX_SCRIPT_FNAME_LENGTH      1024
>> #define TS_LUA_MAX_URL_LENGTH               2048
>> 
>> @@ -85,6 +91,8 @@ typedef struct {
>>    int         intercept_type;
>>    int         ref;
>> 
>> +    int         remap;
>> +
>> } ts_lua_http_ctx;
>> 
>> 
>> 
>> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/904e1f7c/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 ea20fa9..81231ee 100644
>> --- a/plugins/experimental/ts_lua/ts_lua_util.c
>> +++ b/plugins/experimental/ts_lua/ts_lua_util.c
>> @@ -280,6 +280,15 @@ ts_lua_destroy_http_ctx(ts_lua_http_ctx* http_ctx)
>> 
>>    main_ctx = http_ctx->mctx;
>> 
>> +    if(!http_ctx->remap) {
>> +      if(http_ctx->client_request_bufp) {
>> +        if(http_ctx->client_request_url) {
>> +          TSHandleMLocRelease(http_ctx->client_request_bufp, http_ctx->client_request_hdrp, http_ctx->client_request_url);
>> +        } 
>> +        TSHandleMLocRelease(http_ctx->client_request_bufp, TS_NULL_MLOC, http_ctx->client_request_hdrp);
>> +      }
>> +    } 
>> +
>>    if (http_ctx->server_request_bufp) {
>>        TSHandleMLocRelease(http_ctx->server_request_bufp, TS_NULL_MLOC, http_ctx->server_request_hdrp);
>>    }
>>