You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by zt...@apache.org on 2019/05/10 15:19:19 UTC

[hawq] branch taoz updated: refresh gpfdist server/client code

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

ztao1987 pushed a commit to branch taoz
in repository https://gitbox.apache.org/repos/asf/hawq.git


The following commit(s) were added to refs/heads/taoz by this push:
     new 1bf9530  refresh gpfdist server/client code
1bf9530 is described below

commit 1bf9530fb6fcd1c76051a9482d5edd566a116848
Author: ztao1987 <zh...@gmail.com>
AuthorDate: Fri May 10 23:18:35 2019 +0800

    refresh gpfdist server/client code
---
 src/backend/access/external/Makefile         |    2 +-
 src/backend/access/external/url.c            | 1254 +----
 src/backend/access/external/url_curl.c       | 1336 ++++++
 src/backend/utils/misc/fstream/gfile.c       |   47 +-
 src/backend/utils/misc/guc.c                 |   26 +
 src/bin/gpfdist/Makefile                     |    2 +-
 src/bin/gpfdist/src/gpfdist/glob.c           |   55 +-
 src/bin/gpfdist/src/gpfdist/gpfdist.c        | 6549 ++++++++++++++------------
 src/bin/gpfdist/src/gpfdist/gpfdist_helper.c |   30 +
 src/bin/gpfdist/src/gpfdist/gpfdist_helper.h |    7 +
 src/bin/gpfdist/src/gpfdist/gpfxdist.h       |   19 -
 src/bin/gpfdist/src/gpfdist/include/glob.h   |    8 +-
 src/bin/gpfdist/src/gpfdist/transform.c      |  172 +-
 src/bin/gpfdist/src/gpfdist/transform.h      |  247 +
 src/include/access/url.h                     |   19 +-
 src/include/fstream/gfile.h                  |    4 +-
 src/include/utils/guc.h                      |    3 +
 17 files changed, 5285 insertions(+), 4495 deletions(-)

diff --git a/src/backend/access/external/Makefile b/src/backend/access/external/Makefile
index cc52f2f..9c95258 100644
--- a/src/backend/access/external/Makefile
+++ b/src/backend/access/external/Makefile
@@ -26,7 +26,7 @@ subdir = src/backend/access/external
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = fileam.o url.o libchurl.o hd_work_mgr.o pxfuriparser.o pxfheaders.o \
+OBJS = fileam.o url.o url_curl.o libchurl.o hd_work_mgr.o pxfuriparser.o pxfheaders.o \
 pxfmasterapi.o ha_config.o pxfcomutils.o pxfutils.o pxffilters.o pxfanalyze.o \
 plugstorage.o
 
diff --git a/src/backend/access/external/url.c b/src/backend/access/external/url.c
index 7aadc95..28620da 100644
--- a/src/backend/access/external/url.c
+++ b/src/backend/access/external/url.c
@@ -46,7 +46,6 @@
 #include "utils/guc.h"
 #include "utils/builtins.h"
 
-#include <arpa/inet.h>
 #include <fstream/gfile.h>
 
 #include <assert.h>
@@ -57,8 +56,6 @@
 #include <unistd.h>
 #include <stdint.h>
 
-#include <curl/curl.h>
-
 #include <fstream/fstream.h>
 
 #include "cdb/cdbsreh.h"
@@ -77,13 +74,10 @@
 #define TRUE 1
 #endif
 
-#define HOST_NAME_SIZE 100
-
 static char *interpretError(int exitCode, char *buf, size_t buflen, char *err, size_t errlen);
 static const char *getSignalNameFromCode(int signo);
 static int popen_with_stderr(int *rwepipe, const char *exe, bool forwrite);
 static int pclose_with_stderr(int pid, int *rwepipe, char *buf, int len);
-static void gp_proto0_write_done(URL_FILE *file);
 static int32  InvokeExtProtocol(void		*ptr, 
 								size_t 		nbytes, 
 								URL_FILE 	*file, 
@@ -91,331 +85,6 @@ static int32  InvokeExtProtocol(void		*ptr,
 								bool		last_call,
 								ExternalSelectDesc desc,
 								List		**psplits);
-void extract_http_domain(char* i_path, char* o_domain, int dlen);
-
-
-/* we use a global one for convenience */
-CURLM *multi_handle = 0;
-
-/*
- * header_callback
- *
- * when a header arrives from the server curl calls this routine. In here we
- * extract the information we are interested in from the header, and store it
- * in the passed in callback argument (URL_FILE *) which lives in our
- * application.
- */
-static size_t
-header_callback(void *ptr_, size_t size, size_t nmemb, void *userp)
-{
-    URL_FILE*	url = (URL_FILE *)userp;
-	char*		ptr = ptr_;
-	int 		len = size * nmemb;
-	int 		i;
-	char 		buf[20];
-
-	Assert(size == 1);
-
-	/*
-	 * parse the http response line (code and message) from
-	 * the http header that we get. Basically it's the whole
-	 * first line (e.g: "HTTP/1.0 400 time out"). We do this
-	 * in order to capture any error message that comes from
-	 * gpfdist, and later use it to report the error string in
-	 * check_response() to the database user.
-	 */
-	if (url->u.curl.http_response == 0)
-	{
-		int 	n = nmemb;
-		char* 	p;
-
-		if (n > 0 && 0 != (p = palloc(n+1)))
-		{
-			memcpy(p, ptr, n);
-			p[n] = 0;
-
-			if (n > 0 && (p[n-1] == '\r' || p[n-1] == '\n'))
-				p[--n] = 0;
-
-			if (n > 0 && (p[n-1] == '\r' || p[n-1] == '\n'))
-				p[--n] = 0;
-
-			url->u.curl.http_response = p;
-		}
-	}
-
-	/*
-	 * extract the GP-PROTO value from the HTTP header.
-	 */
-	if (len > 10 && *ptr == 'X' && 0 == strncmp("X-GP-PROTO", ptr, 10))
-	{
-		ptr += 10;
-		len -= 10;
-
-		while (len > 0 && (*ptr == ' ' || *ptr == '\t'))
-		{
-			ptr++;
-			len--;
-		}
-
-		if (len > 0 && *ptr == ':')
-		{
-			ptr++;
-			len--;
-
-			while (len > 0 && (*ptr == ' ' || *ptr == '\t'))
-			{
-				ptr++;
-				len--;
-			}
-
-			for (i = 0; i < sizeof(buf) - 1 && i < len; i++)
-				buf[i] = ptr[i];
-
-			buf[i] = 0;
-			url->u.curl.gp_proto = strtol(buf, 0, 0);
-
-			// elog(NOTICE, "X-GP-PROTO: %s (%d)", buf, url->u.curl.gp_proto);
-		}
-	}
-
-	return size * nmemb;
-}
-
-
-/*
- * write_callback
- *
- * when data arrives from gpfdist server and curl is ready to write it
- * to our application, it calls this routine. In here we will store the
- * data in the application variable (URL_FILE *)file which is the passed 
- * in the fourth argument as a part of the callback settings.
- *
- * we return the number of bytes written to the application buffer
- */
-static size_t
-write_callback(char *buffer,
-               size_t size,
-               size_t nitems,
-               void *userp)
-{
-    URL_FILE*	file = (URL_FILE *)userp;
-	curlctl_t*	curl = &file->u.curl;
-	const int 	nbytes = size * nitems;
-	int 		n;
-
-	//elog(NOTICE, "write_callback %d", nbytes);
-
-	/*
-	 * if insufficient space in buffer make more space
-	 */
-	if (curl->in.top + nbytes >= curl->in.max)
-	{
-		/* compact ? */
-		if (curl->in.bot)
-		{
-			n = curl->in.top - curl->in.bot;
-			memmove(curl->in.ptr, curl->in.ptr + curl->in.bot, n);
-			curl->in.bot = 0;
-			curl->in.top = n;
-		}
-
-		/* if still insufficient space in buffer, then do realloc */
-		if (curl->in.top + nbytes >= curl->in.max)
-		{
-			char *newbuf;
-
-			n = curl->in.top - curl->in.bot + nbytes + 1024;
-			newbuf = realloc(curl->in.ptr, n);
-
-			if (!newbuf)
-			{
-				elog(ERROR, "out of memory (curl write_callback)");
-			}
-
-			curl->in.ptr = newbuf;
-			curl->in.max = n;
-			// elog(NOTICE, "max now at %d", n);
-
-			Assert(curl->in.top + nbytes < curl->in.max);
-		}
-	}
-
-	/* enough space. copy buffer into curl->buf */
-	memcpy(curl->in.ptr + curl->in.top, buffer, nbytes);
-	curl->in.top += nbytes;
-
-	return nbytes;
-}
-
-/*
- * check_response
- *
- * If got an HTTP response with an error code from the server (gpfdist), report
- * the error code and message it to the database user and abort operation.
- */
-static int
-check_response(URL_FILE *file, int *rc, const char **response_string)
-{
-	long 		response_code;
-	char*		effective_url = NULL;
-	CURL* 		curl = file->u.curl.handle;
-	static char buffer[30];
-
-	/* get the response code from curl */
-	if (curl_easy_getinfo(curl, CURLINFO_RESPONSE_CODE, &response_code) != CURLE_OK)
-	{
-		*rc = 500;
-		*response_string = "curl_easy_getinfo failed";
-		return -1;
-	}
-	*rc = response_code;
-	snprintf(buffer, sizeof buffer, "Response Code=%d", (int)response_code);
-	*response_string = buffer;
-
-	if (curl_easy_getinfo(curl, CURLINFO_EFFECTIVE_URL, &effective_url) != CURLE_OK)
-		return -1;
-
-	if (! (200 <= response_code && response_code < 300))
-	{
-		if (response_code == 0)
-		{
-			long 		oserrno = 0;
-			static char	connmsg[64];
-			
-			/* get the os level errno, and string representation of it */
-			if (curl_easy_getinfo(curl, CURLINFO_OS_ERRNO, &oserrno) == CURLE_OK)
-			{
-				if (oserrno != 0)
-					snprintf(connmsg, sizeof connmsg, "error code = %d (%s)", 
-							 (int) oserrno, strerror((int)oserrno));
-			}
-			
-			ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE),
-							errmsg("connection with gpfdist failed for %s. "
-									"effective url: %s. %s", file->url, effective_url,
-									(oserrno != 0 ? connmsg : "")),
-						    errOmitLocation(true)));
-		}
-		else
-		{
-			/* we need to sleep 1 sec to avoid this condition:
-			   1- seg X gets an error message from gpfdist
-			   2- seg Y gets a 500 error
-			   3- seg Y report error before seg X, and error message
-			   in seg X is thrown away.
-			*/
-			pg_usleep(1000000);
-			elog(ERROR, "http response code %ld from gpfdist (%s): %s",
-				 response_code, file->url,
-				 file->u.curl.http_response ? file->u.curl.http_response : "?");
-		}
-	}
-
-	return 0;
-}
-
-
-/*
- * fill_buffer
- *
- * Attempt to fill the read buffer up to requested number of bytes.
- * We first check if we already have the number of bytes that we
- * want already in the buffer (from write_callback), and we do
- * a select on the socket only if we don't have enough.
- * 
- * return 0 if successful; raises ERROR otherwise.
- */
-static int
-fill_buffer(URL_FILE *file, int want)
-{
-    fd_set 	fdread;
-    fd_set 	fdwrite;
-    fd_set 	fdexcep;
-    int 	maxfd;
-    struct 	timeval timeout;
-    int 	nfds, e;
-	curlctl_t* curl = &file->u.curl;
-
-	/* elog(NOTICE, "= still_running %d, bot %d, top %d, want %d",
-	   file->u.curl.still_running, curl->in.bot, curl->in.top, want);
-	*/
-
-    /* attempt to fill buffer */
-	while (curl->still_running && curl->in.top - curl->in.bot < want)
-    {
-        FD_ZERO(&fdread);
-        FD_ZERO(&fdwrite);
-        FD_ZERO(&fdexcep);
-
-		CHECK_FOR_INTERRUPTS();
-
-        /* set a suitable timeout to fail on */
-        timeout.tv_sec = 5;
-        timeout.tv_usec = 0;
-
-        /* get file descriptors from the transfers */
-        if (0 != (e = curl_multi_fdset(multi_handle, &fdread, &fdwrite, &fdexcep, &maxfd)))
-		{
-			elog(ERROR, "internal error: curl_multi_fdset failed (%d - %s)",
-						e, curl_easy_strerror(e));
-		}
-
-		if (maxfd <= 0)
-		{
-			curl->still_running = 0;
-			break;
-		}
-
-        if (-1 == (nfds = select(maxfd+1, &fdread, &fdwrite, &fdexcep, &timeout)))
-		{
-			if (errno == EINTR || errno == EAGAIN)
-				continue;
-			elog(ERROR, "internal error: select failed on curl_multi_fdset (maxfd %d) (%d - %s)",
-				 maxfd, errno, strerror(errno));
-		}
-
-		if (nfds > 0)
-		{
-            /* timeout or readable/writable sockets */
-            /* note we *could* be more efficient and not wait for
-             * CURLM_CALL_MULTI_PERFORM to clear here and check it on re-entry
-             * but that gets messy */
-			while (CURLM_CALL_MULTI_PERFORM ==
-				   (e = curl_multi_perform(multi_handle, &file->u.curl.still_running)));
-
-			if (e != 0)
-			{
-				elog(ERROR, "internal error: curl_multi_perform failed (%d - %s)",
-					 e, curl_easy_strerror(e));
-			}
-        }
-
-		/* elog(NOTICE, "- still_running %d, bot %d, top %d, want %d",
-		   file->u.curl.still_running, curl->in.bot, curl->in.top, want);
-		*/
-    }
-
-    return 0;
-}
-
-
-
-static int
-set_httpheader(URL_FILE *fcurl, const char *name, const char *value)
-{
-	char tmp[1024];
-
-	if (strlen(name) + strlen(value) + 5 > sizeof(tmp))
-	{
-		elog(ERROR, "set_httpheader name/value is too long. name = %s, value=%s", name, value);
-	}
-
-	sprintf(tmp, "%s: %s", name, value);
-	fcurl->u.curl.x_httpheader = curl_slist_append(fcurl->u.curl.x_httpheader, tmp);
-
-	return 0;
-}
 
 
 static int
@@ -489,190 +158,6 @@ make_command(const char *cmd, extvar_t * ev, char *buf)
 	return sz + 1;				/* add NUL terminator */
 }
 
-static char *
-local_strstr(const char *str1, const char *str2)
-{	
-	char *cp = (char *) str1;
-	char *s1, *s2;
-
-	if ( !*str2 )
-		return((char *)str1);
-
-	while (*cp)
-    {
-		s1 = cp;
-		s2 = (char *) str2;
-
-		while (*s1 && (*s1==*s2))
-			s1++, s2++;
-
-		if (!*s2)
-			return(cp);
-
-		cp++;
-	}
-
-	return(NULL);
-}
-
-/*
- * This function purpose is to make sure that the URL string contains a numerical IP address.
- * The input URL is in the parameter url. The output result URL is in the output parameter - buf.
- * When parameter - url already contains a numerical ip, then output parameter - buf will be a copy of url.
- * For this case calling getDnsAddress method inside make_url, will serve the purpose of IP validation.
- * But when parameter - url will contain a domain name, then the domain name substring will be changed to a numerical
- * ip address in the buf output parameter.
- */
-static int
-make_url(const char *url, char *buf, bool is_ipv6)
-{
-	char *authority_start = local_strstr(url, "//");
-	char *authority_end;
-	char *hostname_start;
-	char *hostname_end;
-	char hostname[HOST_NAME_SIZE];
-	char *hostip = NULL;
-	char portstr[9];
-	int len;
-	char *p;
-	int port = 80; /* default for http */
-	bool  domain_resolved_to_ipv6 = false;
-		
-	if (! authority_start)
-	{
-		elog(ERROR, "illegal url '%s'", url);
-	}
-	
-	authority_start += 2;
-	authority_end = strchr(authority_start, '/');
-	if (! authority_end)
-		authority_end = authority_start + strlen(authority_start);
-	
-	hostname_start = strchr(authority_start, '@');
-	if (! (hostname_start && hostname_start < authority_end))
-		hostname_start = authority_start;
-	
-	
-	if ( is_ipv6 ) /* IPV6 */
-	{
-		int len;
-		
-		hostname_end = strchr(hostname_start, ']');
-		hostname_end += 1;
-		
-		/* port number exists in this url. get it */
-		len = authority_end - hostname_end;
-		if (len > 8)
-			ereport(ERROR,
-					(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
-					 errmsg("<port> substring size must not exceed %d characters", 8)));
-		
-		memcpy(portstr, hostname_end + 1, len);
-		portstr[len] = 0;
-		port = atoi(portstr);
-		
-		/* skippping the brackets */
-		hostname_end -=1;
-		hostname_start +=1;
-	}
-	else 
-	{
-		hostname_end = strchr(hostname_start, ':');
-		if (! (hostname_end && hostname_end < authority_end))
-		{
-			hostname_end = authority_end;
-		}
-		else
-		{
-			/* port number exists in this url. get it */
-			int len = authority_end - hostname_end;
-			if (len > 8)
-				ereport(ERROR,
-						(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
-						 errmsg("<port> substring size must not exceed %d characters", 8)));			
-			
-			memcpy(portstr, hostname_end + 1, len);
-			portstr[len] = 0;
-			port = atoi(portstr);
-		}
-	}
-	
-	if ( !port ) 
-		ereport(ERROR,
-				(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
-				 errmsg("<port> substring must contain only digits")));	
-	
-	if (hostname_end - hostname_start >= sizeof(hostname))
-	{
-		elog(ERROR, "hostname too long for url '%s'", url);
-	}
-	
-	memcpy(hostname, hostname_start, hostname_end - hostname_start);
-	hostname[hostname_end - hostname_start] = 0;
-	
-	hostip = getDnsAddress(hostname, port, ERROR);
-	
-	/*
-	 * test for the case where the URL originaly contained a domain name (so is_ipv6 was set to false)
-	 * but the DNS resolution in getDnsAddress returned an IPv6 address so know we also have to put the
-	 * square brackets [..] in the URL string.
-	 */
-	if ( strchr(hostip, ':') != NULL && !is_ipv6 ) /* hit the case where a domain name returned an IPv6 address */
-		domain_resolved_to_ipv6 = true;
-	 
-	
-	if (!buf)
-	{
-		int len = strlen(url) + 1 - strlen(hostname) + strlen(hostip);
-		if ( domain_resolved_to_ipv6 )
-			len += 2; /* for the square brackets */
-		return len;
-	}
-	
-	p = buf;
-	len = hostname_start - url;
-	strncpy(p, url, len); 
-	p += len; 
-	url += len;
-	
-	len = strlen(hostname);
-	url += len;
-	
-	len = strlen(hostip);
-	if ( domain_resolved_to_ipv6 )
-	{
-		*p = '[';
-		p++;
-	}
-	strncpy(p, hostip, len); 
-	p += len;
-	if ( domain_resolved_to_ipv6 )
-	{
-		*p = ']';
-		p++;
-	}	
-	
-	strcpy(p, url);
-	return p - buf;
-}
-
-/*
- * extract_http_domain
- *
- * extracts the domain string from a http url
- */
-void extract_http_domain(char* i_path, char* o_domain, int dlen)
-{
-	int domsz, cpsz;
-	char* p_st = (char*)local_strstr(i_path, "//");
-	p_st = p_st + 2;
-	char* p_en = strchr(p_st, '/');
-	
-	domsz = p_en - p_st;
-	cpsz = ( domsz < dlen ) ? domsz : dlen;
-	memcpy(o_domain, p_st, cpsz);
-}
-
 /**
  * execute_fopen()
  *
@@ -741,20 +226,6 @@ url_execute_fopen(char* url, char *cmd, bool forwrite, extvar_t *ev)
 	return file;
 }
 
-static bool 
-url_has_ipv6_format (char *url)
-{
-	bool is6 = false;
-	char *ipv6 = local_strstr(url, "://[");
-	
-	if ( ipv6 )
-		ipv6 = strchr(ipv6, ']');
-	if ( ipv6 )
-		is6 = true;
-		
-	return is6;
-}
-
 /*
  * url_fopen
  *
@@ -794,6 +265,8 @@ url_fopen(char *url,
 		/* Execute command */
 		cmd  = url + strlen(exec_prefix);
 		return url_execute_fopen(url, cmd, forwrite, ev);
+	} else if (IS_HTTP_URI(url) || IS_GPFDIST_URI(url)) {
+    return url_curl_fopen(url, forwrite, ev, pstate);
 	}
 
 	if (!(file = (URL_FILE *)malloc(sizeof(URL_FILE) + strlen(url) + 1)))
@@ -845,250 +318,6 @@ url_fopen(char *url,
 			return NULL;
 		}
     }
-    else if (IS_HTTP_URI(url) || IS_GPFDIST_URI(url))
-    {
-		int sz;
-
-		bool is_ipv6 = url_has_ipv6_format(file->url);
-		
-		sz = make_url(file->url, 0, is_ipv6);
-		
-        file->type = CFTYPE_CURL; /* marked as URL */
-
-		if (sz < 0)
-		{
-			const char* url_cpy = pstrdup(file->url);
-			
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "illegal URL: %s", url_cpy);
-		}
-
-		file->u.curl.curl_url = palloc(sz);
-		memset(file->u.curl.curl_url, 0, sz);
-		file->u.curl.for_write = forwrite;
-		
-		make_url(file->url, file->u.curl.curl_url, is_ipv6);
-		/*
-		 * We need to call is_url_ipv6 for the case where inside make_url function
-		 * a domain name was transformed to an IPv6 address.
-		 */
-		if ( !is_ipv6 )
-			is_ipv6 = url_has_ipv6_format( file->u.curl.curl_url);
-
-		if (IS_GPFDIST_URI(file->u.curl.curl_url))
-		{
-			/* replace gpfdist:// with http:// */
-			file->u.curl.curl_url += 3;
-			memcpy(file->u.curl.curl_url, "http", 4);
-			pstate->header_line = 0;
-		}
-
-		/* initialize a curl session and get a libcurl handle for it */
-        if (! (file->u.curl.handle = curl_easy_init()))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_init failed");
-		}
-
-
-        if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_URL, file->u.curl.curl_url)))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_setopt CURLOPT_URL error (%d - %s)",
-				 e, curl_easy_strerror(e));
-		}
-        if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_VERBOSE, FALSE)))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_setopt CURLOPT_VERBOSE error (%d - %s)",
-				 e, curl_easy_strerror(e));
-		}
-		/* set callback for each header received from server */
-		if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_HEADERFUNCTION, header_callback)))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_setopt CURLOPT_HEADERFUNCTION error (%d - %s)",
-				 e, curl_easy_strerror(e));
-		}
-		/* 'file' is the application variable that gets passed to header_callback */
-		if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_WRITEHEADER, file)))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_setopt CURLOPT_WRITEHEADER error (%d - %s)",
-				 e, curl_easy_strerror(e));
-		}
-		/* set callback for each data block arriving from server to be written to application */
-        if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_WRITEFUNCTION, write_callback)))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_setopt CURLOPT_WRITEFUNCTION error (%d - %s)",
-				 e, curl_easy_strerror(e));
-		}
-		/* 'file' is the application variable that gets passed to write_callback */
-        if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_WRITEDATA, file)))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_setopt CURLOPT_WRITEDATA error (%d - %s)",
-				 e, curl_easy_strerror(e));
-		}
-				
-		if ( !is_ipv6 )
-			ip_mode = CURL_IPRESOLVE_V4;
-		else 
-			ip_mode = CURL_IPRESOLVE_V6;
-		if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_IPRESOLVE, ip_mode)))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_setopt CURLOPT_IPRESOLVE error (%d - %s)",
-				 e, curl_easy_strerror(e));
-		}
-		
-		/*
-		 * set up a linked list of http headers. start with common headers
-		 * needed for read and write operations, and continue below with 
-		 * more specifics
-		 */			
-		file->u.curl.x_httpheader = NULL;
-		
-		/*
-		 * support multihomed http use cases. see MPP-11874
-		 */
-		if (IS_HTTP_URI(url))
-		{
-			char domain[HOST_NAME_SIZE] = {0};
-			extract_http_domain(file->url, domain, HOST_NAME_SIZE);
-			set_httpheader(file, "Host", domain);
-		}
-		
-		
-		if (set_httpheader(file, "X-GP-XID", ev->GP_XID) ||
-			set_httpheader(file, "X-GP-CID", ev->GP_CID) ||
-			set_httpheader(file, "X-GP-SN", ev->GP_SN) ||
-			set_httpheader(file, "X-GP-SEGMENT-ID", ev->GP_SEGMENT_ID) ||
-			set_httpheader(file, "X-GP-SEGMENT-COUNT", ev->GP_SEGMENT_COUNT))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: some header value is too long");
-		}
-
-		if(forwrite)
-		{
-			/* write specific headers */
-			if(set_httpheader(file, "X-GP-PROTO", "0") ||
-			   set_httpheader(file, "Content-Type", "text/xml"))
-			{
-				url_fclose(file, false, pstate->cur_relname);
-				elog(ERROR, "internal error: some header value is too long");
-			}
-		}
-		else
-		{
-			/* read specific - (TODO: unclear why some of these are needed) */
-			if(set_httpheader(file, "X-GP-PROTO", "1") ||
-			   set_httpheader(file, "X-GP-MASTER_HOST", ev->GP_MASTER_HOST) ||
-			   set_httpheader(file, "X-GP-MASTER_PORT", ev->GP_MASTER_PORT) ||
-			   set_httpheader(file, "X-GP-CSVOPT", ev->GP_CSVOPT) ||
-			   set_httpheader(file, "X-GP_SEG_PG_CONF", ev->GP_SEG_PG_CONF) ||
-			   set_httpheader(file, "X-GP_SEG_DATADIR", ev->GP_SEG_DATADIR) ||
-			   set_httpheader(file, "X-GP-DATABASE", ev->GP_DATABASE) ||
-			   set_httpheader(file, "X-GP-USER", ev->GP_USER) ||
-			   set_httpheader(file, "X-GP-SEG-PORT", ev->GP_SEG_PORT) ||
-			   set_httpheader(file, "X-GP-SESSION-ID", ev->GP_SESSION_ID))
-			{
-				url_fclose(file, false, pstate->cur_relname);
-				elog(ERROR, "internal error: some header value is too long");
-			}
-		}
-		
-		{
-			/*
-			 * MPP-13031
-			 * copy #transform fragment, if present, into X-GP-TRANSFORM header
-			 */
-			char* p = local_strstr(file->url, "#transform=");
-			if (p && p[11]) 
-			{
-				if (set_httpheader(file, "X-GP-TRANSFORM", p+11))
-				{
-					url_fclose(file, false, pstate->cur_relname);
-					elog(ERROR, "internal error: X-GP-TRANSFORM header value is too long");
-				}
-			}
-		}
-		
-		if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_HTTPHEADER, file->u.curl.x_httpheader)))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			elog(ERROR, "internal error: curl_easy_setopt CURLOPT_HTTPHEADER error (%d - %s)",
-				 e, curl_easy_strerror(e));
-        }
-
-        if (!multi_handle)
-		{
-            if (! (multi_handle = curl_multi_init()))
-			{
-				url_fclose(file, false, pstate->cur_relname);
-				elog(ERROR, "internal error: curl_multi_init failed");
-			}
-		}
-
-        /* 
-         * lets check our connection.
-         * start the fetch if we're SELECTing (GET request), or write an
-         * empty message if we're INSERTing (POST request) 
-         */
-        if (!forwrite)
-        {
-          if (CURLE_OK != (e = curl_multi_add_handle(multi_handle, file->u.curl.handle)))
-		      {
-			      if (CURLM_CALL_MULTI_PERFORM != e)
-			      {
-				      url_fclose(file, false, pstate->cur_relname);
-				      elog(ERROR, "internal error: curl_multi_add_handle failed (%d - %s)",
-					        e, curl_easy_strerror(e));
-			      }
-		      }
-            
-    		while (CURLM_CALL_MULTI_PERFORM ==
-    			   (e = curl_multi_perform(multi_handle, &file->u.curl.still_running)));
-
-    		if (e != CURLE_OK)
-    		{
-    			url_fclose(file, false, pstate->cur_relname);
-    			elog(ERROR, "internal error: curl_multi_perform failed (%d - %s)",
-    				 e, curl_easy_strerror(e));
-    		}
-
-    		/* read some bytes to make sure the connection is established */
-    		fill_buffer(file, 1);
-        }
-        else
-        {
-        	/* use empty message */
-            if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_POSTFIELDS, "")))
-        	{
-        		url_fclose(file, false, pstate->cur_relname);
-        		elog(ERROR, "internal error: curl_easy_setopt CURLOPT_POSTFIELDS error (%d - %s)",
-        			 e, curl_easy_strerror(e));
-        	}
-
-            /* post away! */
-            if (CURLE_OK != (e = curl_easy_perform(file->u.curl.handle)))
-        	{
-        		url_fclose(file, false, pstate->cur_relname);
-        		elog(ERROR, "error: %s",
-        			 curl_easy_strerror(e));
-        	}
-        }
-        
-		/* check the connection */
-		if (check_response(file, response_code, response_string))
-		{
-			url_fclose(file, false, pstate->cur_relname);
-			return NULL;
-		}
-
-    }
     else
     {
     	/* we're using a custom protocol */
@@ -1228,47 +457,8 @@ url_fclose(URL_FILE *file, bool failOnError, const char *relname)
 			break;
 
 		case CFTYPE_CURL:
-			
-			/*
-			 * if WET, send a final "I'm done" request from this segment.
-			 */
-			if(file->u.curl.for_write)
-				gp_proto0_write_done(file);
-			
-			if (file->u.curl.x_httpheader)
-			{
-				curl_slist_free_all(file->u.curl.x_httpheader);
-				file->u.curl.x_httpheader = NULL;
-			}
-
-			/* make sure the easy handle is not in the multi handle anymore */
-			if (file->u.curl.handle)
-			{
-				curl_multi_remove_handle(multi_handle, file->u.curl.handle);
-				/* cleanup */
-				curl_easy_cleanup(file->u.curl.handle);
-				file->u.curl.handle = NULL;
-			}
-
-			/* free any allocated buffer space */
-			if (file->u.curl.in.ptr)
-			{
-				free(file->u.curl.in.ptr);
-				file->u.curl.in.ptr = NULL;
-			}
-				
-			if (file->u.curl.out.ptr)
-			{
-				Assert(file->u.curl.for_write);
-				pfree(file->u.curl.out.ptr);
-				file->u.curl.out.ptr = NULL;
-			}
-			
-			file->u.curl.gp_proto = 0;
-			file->u.curl.error = file->u.curl.eof = 0;
-			memset(&file->u.curl.in, 0, sizeof(file->u.curl.in));
-			memset(&file->u.curl.block, 0, sizeof(file->u.curl.block));
-			break;
+		  url_curl_fclose(file, failOnError, relname);
+		  return ret;
 
 		case CFTYPE_CUSTOM:
 			
@@ -1311,8 +501,7 @@ url_feof(URL_FILE *file, int bytesread)
 			break;
 
 		case CFTYPE_CURL:
-			ret = (file->u.curl.eof != 0);
-			break;
+		  return url_curl_feof(file, bytesread);
 
 		case CFTYPE_CUSTOM:
 			ret = (bytesread == 0);
@@ -1352,8 +541,7 @@ bool url_ferror(URL_FILE *file, int bytesread, char *ebuf, int ebuflen)
 			break;
 
 		case CFTYPE_CURL:
-			ret = (file->u.curl.error != 0);
-			break;
+		  return url_curl_ferror(file, bytesread, ebuf, ebuflen);
 
 		case CFTYPE_CUSTOM:
 			ret = (bytesread == -1);
@@ -1368,415 +556,6 @@ bool url_ferror(URL_FILE *file, int bytesread, char *ebuf, int ebuflen)
 	return ret;
 }
 
-/*
- * gp_proto0_read
- *
- * get data from the server and handle it according to PROTO 0. In PROTO 0 we
- * expect the content of the file without any kind of meta info. Simple.
- */
-static size_t gp_proto0_read(char *buf, int bufsz, URL_FILE* file)
-{
-	int 		n = 0;
-	curlctl_t* 	curl = &file->u.curl;
-
-	fill_buffer(file, bufsz);
-
-	/* check if there's data in the buffer - if not fill_buffer()
-	 * either errored or EOF. For proto0, we cannot distinguish
-	 * between error and EOF. */
-	n = curl->in.top - curl->in.bot;
-	if (n == 0 && !curl->still_running)
-		curl->eof = 1;
-
-	if (n > bufsz)
-		n = bufsz;
-
-	/* xfer data to caller */
-	memcpy(buf, curl->in.ptr, n);
-	curl->in.bot += n;
-
-	return n;
-}
-
-/*
- * gp_proto1_read
- *
- * get data from the server and handle it according to PROTO 1. In this protocol
- * each data block is tagged by meta info like this:
- * byte 0: type (can be 'F'ilename, 'O'ffset, 'D'ata, 'E'rror, 'L'inenumber)
- * byte 1-4: length. # bytes of following data block. in network-order.
- * byte 5-X: the block itself.
- */
-static size_t gp_proto1_read(char *buf, int bufsz, URL_FILE *file, CopyState pstate, char *buf2)
-{
-	char type;
-	int  n, len;
-	curlctl_t* curl = &file->u.curl;
-
-	/*
-	 * Loop through and get all types of messages, until we get actual data,
-	 * or until there's no more data. Then quit the loop to process it and
-	 * return it.
-	 */
-	while (curl->block.datalen == 0 && !curl->eof)
-	{
-		/* need 5 bytes, 1 byte type + 4 bytes length */
-		fill_buffer(file, 5);
-		n = curl->in.top - curl->in.bot;
-
-		if (n == 0)
-		{
-			elog(ERROR, "gpfdist error: server closed connection.\n");
-			return -1;
-		}
-
-		if (n < 5)
-		{
-			elog(ERROR, "gpfdist error: incomplete packet - packet len %d\n", n);
-			return -1;
-		}
-
-		/* read type */
-		type = curl->in.ptr[curl->in.bot++];
-
-		/* read len */
-		memcpy(&len, &curl->in.ptr[curl->in.bot], 4);
-		len = ntohl(len);		/* change order */
-		curl->in.bot += 4;
-
-		if (len < 0)
-		{
-			elog(ERROR, "gpfdist error: bad packet type %d len %d",
-				 type, len);
-			return -1;
-		}
-
-		/* elog(NOTICE, "HEADER %c %d, bot %d, top %d", type, len,
-		   curl->in.bot, curl->in.top);
-		*/
-
-		/* Error */
-		if (type == 'E')
-		{
-			fill_buffer(file, len);
-			n = curl->in.top - curl->in.bot;
-
-			if (n > len)
-				n = len;
-
-			if (n > 0)
-			{
-				/*
-				 * cheat a little. swap last char and
-				 * NUL-terminator. then print string (without last
-				 * char) and print last char artificially
-				 */
-				char x = curl->in.ptr[curl->in.bot + n - 1];
-				curl->in.ptr[curl->in.bot + n - 1] = 0;
-				elog(ERROR, "gpfdist error - %s%c", &curl->in.ptr[curl->in.bot], x);
-
-				return -1;
-			}
-
-			elog(ERROR, "gpfdist error: please check gpfdist log messages.");
-
-			return -1;
-		}
-
-		/* Filename */
-		if (type == 'F')
-		{
-			if (buf != buf2)
-			{
-				curl->in.bot -= 5;
-				return 0;
-			}
-			if (len > 256)
-			{
-				elog(ERROR, "gpfdist error: filename too long (%d)", len);
-				return -1;
-			}
-			if (-1 == fill_buffer(file, len))
-			{
-				elog(ERROR, "gpfdist error: stream ends suddenly");
-				return -1;
-			}
-
-			/*
-			 * If SREH is used we now update it with the actual file that the
-			 * gpfdist server is reading. This is because SREH (or the client
-			 * in general) doesn't know which file gpfdist is reading, since
-			 * the original URL may include a wildcard or a directory listing.
-			 */
-			if (pstate->cdbsreh)
-			{
-				char fname[257];
-
-				memcpy(fname, curl->in.ptr + curl->in.bot, len);
-				fname[len] = 0;
-				snprintf(pstate->cdbsreh->filename, sizeof pstate->cdbsreh->filename,"%s [%s]", pstate->filename, fname);
-			}
-
-			curl->in.bot += len;
-			Assert(curl->in.bot <= curl->in.top);
-			continue;
-		}
-
-		/* Offset */
-		if (type == 'O')
-		{
-			if (len != 8)
-			{
-				elog(ERROR, "gpfdist error: offset not of length 8 (%d)", len);
-				return -1;
-			}
-			if (-1 == fill_buffer(file, len))
-			{
-				elog(ERROR, "gpfdist error: stream ends suddenly");
-				return -1;
-			}
-
-			curl->in.bot += 8;
-			Assert(curl->in.bot <= curl->in.top);
-			continue;
-		}
-
-		/* Line number */
-		if (type == 'L')
-		{
-			int64 line_number;
-
-			if (len != 8)
-			{
-				elog(ERROR, "gpfdist error: line number not of length 8 (%d)", len);
-				return -1;
-			}
-			if (-1 == fill_buffer(file, len))
-			{
-				elog(ERROR, "gpfdist error: stream ends suddenly");
-				return -1;
-			}
-
-			/*
-			 * update the line number of the first line we're about to get from
-			 * gpfdist. pstate will update the following lines when processing
-			 * the data
-			 */
-			memcpy(&line_number, curl->in.ptr + curl->in.bot, len);
-			line_number = local_ntohll(line_number);
-			pstate->cur_lineno = line_number ? line_number - 1 : INT64_MIN;
-			curl->in.bot += 8;
-			Assert(curl->in.bot <= curl->in.top);
-			continue;
-		}
-
-		/* Data */
-		if (type == 'D')
-		{
-			curl->block.datalen = len;
-			curl->eof = (len == 0);
-			// elog(NOTICE, "D %d", curl->block.datalen);
-			break;
-		}
-
-		elog(ERROR, "gpfdist error: unknown meta type %d", type);
-		return -1;
-	}
-
-	/* read data block */
-	if (bufsz > curl->block.datalen)
-		bufsz = curl->block.datalen;
-
-	fill_buffer(file, bufsz);
-	n = curl->in.top - curl->in.bot;
-
-	/* if gpfdist closed connection prematurely or died catch it here */
-	if (n == 0 && !curl->eof)
-	{
-		curl->error = 1;
-		
-		if(!curl->still_running)
-			elog(ERROR, "gpfdist server closed connection.\n");
-	}
-
-	if (n > bufsz)
-		n = bufsz;
-
-	memcpy(buf, curl->in.ptr + curl->in.bot, n);
-	curl->in.bot += n;
-	curl->block.datalen -= n;
-	// elog(NOTICE, "returning %d bytes, %d bytes left \n", n, curl->block.datalen);
-	return n;
-}
-
-/*
- * gp_proto0_write
- * 
- * use curl to write data to a the remote gpfdist server. We use
- * a push model with a POST request. 
- * 
- */
-static void gp_proto0_write(URL_FILE *file, CopyState pstate)
-{
-	curlctl_t*	curl = &file->u.curl;
-	char*		buf = curl->out.ptr;
-	int			nbytes = curl->out.top;
-	int 		e;
-	int 		response_code;
-	const char*	response_string;
-
-	if (nbytes == 0)
-		return;
-	
-	/* post binary data */  
-    if (CURLE_OK != (e = curl_easy_setopt(curl->handle, CURLOPT_POSTFIELDS, buf)))
-		elog(ERROR, "internal error: curl_easy_setopt CURLOPT_POSTFIELDS error (%d - %s)",
-			 e, curl_easy_strerror(e));
-
-	 /* set the size of the postfields data */  
-    if (CURLE_OK != (e = curl_easy_setopt(curl->handle, CURLOPT_POSTFIELDSIZE, nbytes)))
-		elog(ERROR, "internal error: curl_easy_setopt CURLOPT_POSTFIELDSIZE error (%d - %s)",
-			 e, curl_easy_strerror(e));
-
-    /* post away! */
-    if (CURLE_OK != (e = curl_easy_perform(curl->handle)))
-		elog(ERROR, "%s error (%d - %s)",
-			 file->u.curl.curl_url,
-			 e, curl_easy_strerror(e));
-    
-	/* check the response from server */
-	if (check_response(file, &response_code, &response_string))
-		elog(ERROR, "error while writing data to gpfdist on %s (code %d, msg %s)",
-				file->u.curl.curl_url, response_code, response_string);
-
-}
-
-/*
- * Send an empty POST request, with an added X-GP-DONE header.
- */
-static void gp_proto0_write_done(URL_FILE *file)
-{
-    int 	e;
-
-	set_httpheader(file, "X-GP-DONE", "1");
-
-	/* use empty message */
-	if (CURLE_OK != (e = curl_easy_setopt(file->u.curl.handle, CURLOPT_POSTFIELDS, "")))
-	{
-		elog(ERROR, "internal error: curl_easy_setopt CURLOPT_POSTFIELDS %s error (%d - %s)",
-			 file->u.curl.curl_url,
-			 e, curl_easy_strerror(e));
-	}
-
-    /* post away! */
-    if (CURLE_OK != (e = curl_easy_perform(file->u.curl.handle)))
-	{
-		elog(ERROR, "%s error: %s",
-			 file->u.curl.curl_url,
-			 curl_easy_strerror(e));
-	}
-
-}
-
-static size_t curl_fread(char *buf, int bufsz, URL_FILE* file, CopyState pstate)
-{
-	curlctl_t*	curl = &file->u.curl;
-	char*		p = buf;
-	char*		q = buf + bufsz;
-	int 		n;
-	const int 	gp_proto = curl->gp_proto;
-
-	if (gp_proto != 0 && gp_proto != 1)
-	{
-		elog(ERROR, "unknown gp protocol %d", curl->gp_proto);
-		return 0;
-	}
-
-	for (; p < q; p += n)
-	{
-		if (gp_proto == 0)
-			n = gp_proto0_read(p, q - p, file);
-		else
-			n = gp_proto1_read(p, q - p, file, pstate, buf);
-
-		//elog(NOTICE, "curl_fread %d bytes", n);
-
-		if (n <= 0)
-			break;
-	}
-
-	return p - buf;
-}
-
-static size_t curl_fwrite(char *buf, int nbytes, URL_FILE* file, CopyState pstate)
-{
-	curlctl_t*	curl = &file->u.curl;
-	
-	if (curl->gp_proto != 0 && curl->gp_proto != 1)
-	{
-		elog(ERROR, "unknown gp protocol %d", curl->gp_proto);
-		return 0;
-	}
-
-	/*
-	 * allocate data buffer if not done already
-	 */
-	if(!curl->out.ptr)
-	{
-		const int bufsize = 64 * 1024 * sizeof(char);
-		MemoryContext oldcontext = CurrentMemoryContext;
-		
-		MemoryContextSwitchTo(CurTransactionContext); /* TODO: is there a better cxt to use? */
-		curl->out.ptr = (char *)palloc(bufsize);
-		curl->out.max = bufsize;
-		curl->out.bot = curl->out.top = 0;
-		MemoryContextSwitchTo(oldcontext);
-	}
-	
-	/*
-	 * if buffer is full (current item can't fit) - write it out to
-	 * the server. if item still doesn't fit after we emptied the
-	 * buffer, make more room.
-	 */
-	if (curl->out.top + nbytes >= curl->out.max)
-	{
-		/* item doesn't fit */
-		if (curl->out.top > 0)
-		{
-			/* write out existing data, empty the buffer */
-			gp_proto0_write(file, pstate);
-			curl->out.top = 0;
-		}
-		
-		/* does it still not fit? enlarge buffer */
-		if (curl->out.top + nbytes >= curl->out.max)
-		{
-			int 	n = nbytes + 1024;
-			char*	newbuf;
-			MemoryContext oldcontext = CurrentMemoryContext;
-
-			MemoryContextSwitchTo(CurTransactionContext); /* TODO: is there a better cxt to use? */
-			newbuf = repalloc(curl->out.ptr, n);
-			MemoryContextSwitchTo(oldcontext);
-
-			if (!newbuf)
-				elog(ERROR, "out of memory (curl_fwrite)");
-
-			curl->out.ptr = newbuf;
-			curl->out.max = n;
-
-			Assert(nbytes < curl->out.max);
-		}
-	}
-
-	/* copy buffer into curl->buf */
-	memcpy(curl->out.ptr + curl->out.top, buf, nbytes);
-	curl->out.top += nbytes;
-	
-	return nbytes;
-}
-
-
 size_t
 url_fread(void *ptr, size_t size, size_t nmemb, URL_FILE *file, CopyState pstate, ExternalSelectDesc desc, List **splits)
 {
@@ -1815,7 +594,7 @@ url_fread(void *ptr, size_t size, size_t nmemb, URL_FILE *file, CopyState pstate
 		case CFTYPE_CURL:
 
 			/* get data (up to nmemb * size) from the http/gpfdist server */
-			n = curl_fread(ptr, nmemb * size, file, pstate);
+			n = url_curl_fread(ptr, nmemb * size, file, pstate);
 
 			/* number of items - nb correct op - checked with glibc code*/
 			want = n / size;
@@ -1857,7 +636,7 @@ url_fwrite(void *ptr, size_t size, size_t nmemb, URL_FILE *file, CopyState pstat
 		case CFTYPE_CURL:
 			
 			/* write data to the gpfdist server via curl */
-			n = curl_fwrite(ptr, nmemb * size, file, pstate);
+			n = url_curl_fwrite(ptr, nmemb * size, file, pstate);
 			want = n / size;
 			break;
 		
@@ -1895,7 +674,7 @@ url_fflush(URL_FILE *file, CopyState pstate)
 			break;
 
 		case CFTYPE_CURL:
-			gp_proto0_write(file, pstate);
+		  url_curl_fflush(file, pstate);
 			break;
 
 		default: /* unknown or unsupported type */
@@ -1923,20 +702,7 @@ url_rewind(URL_FILE *file, const char *relname)
 			break;
 
 		case CFTYPE_CURL:
-			/* halt transaction */
-			curl_multi_remove_handle(multi_handle, file->u.curl.handle);
-
-			/* restart */
-			curl_multi_add_handle(multi_handle, file->u.curl.handle);
-
-			/* ditch buffer - write will recreate - resets stream pos*/
-			if (file->u.curl.in.ptr)
-				free(file->u.curl.in.ptr);
-
-			file->u.curl.gp_proto = 0;
-			file->u.curl.error = file->u.curl.eof = 0;
-			memset(&file->u.curl.in, 0, sizeof(file->u.curl.in));
-			memset(&file->u.curl.block, 0, sizeof(file->u.curl.block));
+		  url_curl_rewind(file, relname);
 			break;
 
 		case CFTYPE_CUSTOM:
diff --git a/src/backend/access/external/url_curl.c b/src/backend/access/external/url_curl.c
new file mode 100644
index 0000000..422f65e
--- /dev/null
+++ b/src/backend/access/external/url_curl.c
@@ -0,0 +1,1336 @@
+/*
+ * 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.
+ */
+
+/*-------------------------------------------------------------------------
+ *
+ * url_curl.c
+ *    Core support for opening external relations via a URL with curl
+ *
+ */
+
+#include "postgres.h"
+
+#include "access/url.h"
+
+#include <arpa/inet.h>
+
+#include <curl/curl.h>
+
+#include "cdb/cdbsreh.h"
+#include "cdb/cdbutil.h"
+#include "cdb/cdbvars.h"
+#include "miscadmin.h"
+#include "utils/guc.h"
+#include "utils/uri.h"
+
+#if BYTE_ORDER == BIG_ENDIAN
+#define local_htonll(n) (n)
+#define local_ntohll(n) (n)
+#else
+#define local_htonll(n) ((((uint64)htonl(n)) << 32LL) | htonl((n) >> 32LL))
+#define local_ntohll(n) \
+  ((((uint64)ntohl(n)) << 32LL) | (uint32)ntohl(((uint64)n) >> 32LL))
+#endif
+
+#define HOST_NAME_SIZE 100
+#define FDIST_TIMEOUT 408
+#define MAX_TRY_WAIT_TIME 64
+
+/* we use a global one for convenience */
+static CURLM *multi_handle = 0;
+
+/*
+ * A helper macro, to call curl_easy_setopt(), and ereport() if it fails.
+ */
+#define CURL_EASY_SETOPT(h, opt, val)                                        \
+  do {                                                                       \
+    int e;                                                                   \
+    if ((e = curl_easy_setopt(h, opt, val)) != CURLE_OK)                     \
+      elog(ERROR, "internal error: curl_easy_setopt \"%s\" error (%d - %s)", \
+           CppAsString(opt), e, curl_easy_strerror(e));                      \
+  } while (0)
+
+/*
+ * header_callback
+ *
+ * when a header arrives from the server curl calls this routine. In here we
+ * extract the information we are interested in from the header, and store it
+ * in the passed in callback argument (URL_FILE *) which lives in our
+ * application.
+ */
+static size_t header_callback(void *ptr_, size_t size, size_t nmemb,
+                              void *userp) {
+  URL_FILE *url = (URL_FILE *)userp;
+  char *ptr = ptr_;
+  int len = size * nmemb;
+  int i;
+  char buf[20];
+
+  Assert(size == 1);
+
+  /*
+   * parse the http response line (code and message) from
+   * the http header that we get. Basically it's the whole
+   * first line (e.g: "HTTP/1.0 400 time out"). We do this
+   * in order to capture any error message that comes from
+   * gpfdist, and later use it to report the error string in
+   * check_response() to the database user.
+   */
+  if (url->u.curl.http_response == 0) {
+    int n = nmemb;
+    char *p;
+
+    if (n > 0 && 0 != (p = palloc(n + 1))) {
+      memcpy(p, ptr, n);
+      p[n] = 0;
+
+      if (n > 0 && (p[n - 1] == '\r' || p[n - 1] == '\n')) p[--n] = 0;
+
+      if (n > 0 && (p[n - 1] == '\r' || p[n - 1] == '\n')) p[--n] = 0;
+
+      url->u.curl.http_response = p;
+    }
+  }
+
+  /*
+   * extract the GP-PROTO value from the HTTP header.
+   */
+  if (len > 10 && *ptr == 'X' && 0 == strncmp("X-GP-PROTO", ptr, 10)) {
+    ptr += 10;
+    len -= 10;
+
+    while (len > 0 && (*ptr == ' ' || *ptr == '\t')) {
+      ptr++;
+      len--;
+    }
+
+    if (len > 0 && *ptr == ':') {
+      ptr++;
+      len--;
+
+      while (len > 0 && (*ptr == ' ' || *ptr == '\t')) {
+        ptr++;
+        len--;
+      }
+
+      for (i = 0; i < sizeof(buf) - 1 && i < len; i++) buf[i] = ptr[i];
+
+      buf[i] = 0;
+      url->u.curl.gp_proto = strtol(buf, 0, 0);
+    }
+  }
+
+  return size * nmemb;
+}
+
+/*
+ * write_callback
+ *
+ * when data arrives from gpfdist server and curl is ready to write it
+ * to our application, it calls this routine. In here we will store the
+ * data in the application variable (URL_FILE *)file which is the passed
+ * in the forth argument as a part of the callback settings.
+ *
+ * we return the number of bytes written to the application buffer
+ */
+static size_t write_callback(char *buffer, size_t size, size_t nitems,
+                             void *userp) {
+  URL_FILE *file = (URL_FILE *)userp;
+  curlctl_t *curl = &file->u.curl;
+  const int nbytes = size * nitems;
+  int n;
+
+  /*
+   * if insufficient space in buffer make more space
+   */
+  if (curl->in.top + nbytes >= curl->in.max) {
+    /* compact ? */
+    if (curl->in.bot) {
+      n = curl->in.top - curl->in.bot;
+      memmove(curl->in.ptr, curl->in.ptr + curl->in.bot, n);
+      curl->in.bot = 0;
+      curl->in.top = n;
+    }
+
+    /* if still insufficient space in buffer, enlarge it */
+    if (curl->in.top + nbytes >= curl->in.max) {
+      char *newbuf;
+
+      n = curl->in.top - curl->in.bot + nbytes + 1024;
+      newbuf = repalloc(curl->in.ptr, n);
+
+      curl->in.ptr = newbuf;
+      curl->in.max = n;
+
+      Assert(curl->in.top + nbytes < curl->in.max);
+    }
+  }
+
+  /* enough space. copy buffer into curl->buf */
+  memcpy(curl->in.ptr + curl->in.top, buffer, nbytes);
+  curl->in.top += nbytes;
+
+  return nbytes;
+}
+
+static char *local_strstr(const char *str1, const char *str2) {
+  char *cp = (char *)str1;
+  char *s1, *s2;
+
+  if (!*str2) return ((char *)str1);
+
+  while (*cp) {
+    s1 = cp;
+    s2 = (char *)str2;
+
+    while (*s1 && (*s1 == *s2)) s1++, s2++;
+
+    if (!*s2) return (cp);
+
+    cp++;
+  }
+
+  return (NULL);
+}
+
+/*
+ * This function purpose is to make sure that the URL string contains a
+ * numerical IP address.  The input URL is in the parameter url. The output
+ * result URL is in the output parameter - buf.  When parameter - url already
+ * contains a numerical ip, then output parameter - buf will be a copy of url.
+ * For this case calling getDnsAddress method inside make_url, will serve the
+ * purpose of IP validation.  But when parameter - url will contain a domain
+ * name, then the domain name substring will be changed to a numerical ip
+ * address in the buf output parameter.
+ *
+ * Returns the length of the converted URL string, excluding null-terminator.
+ */
+static int make_url(const char *url, char *buf, bool is_ipv6) {
+  char *authority_start = local_strstr(url, "//");
+  char *authority_end;
+  char *hostname_start;
+  char *hostname_end;
+  char hostname[HOST_NAME_SIZE];
+  char *hostip = NULL;
+  char portstr[9];
+  int len;
+  char *p;
+  int port = 80; /* default for http */
+  bool domain_resolved_to_ipv6 = false;
+
+  if (!authority_start) elog(ERROR, "illegal url '%s'", url);
+
+  authority_start += 2;
+  authority_end = strchr(authority_start, '/');
+  if (!authority_end) authority_end = authority_start + strlen(authority_start);
+
+  hostname_start = strchr(authority_start, '@');
+  if (!(hostname_start && hostname_start < authority_end))
+    hostname_start = authority_start;
+
+  if (is_ipv6) /* IPV6 */
+  {
+    int len;
+
+    hostname_end = strchr(hostname_start, ']');
+    if (hostname_end == NULL)
+      ereport(ERROR, (errcode(ERRCODE_INVALID_NAME),
+                      errmsg("unexpected IPv6 format %s", url)));
+    hostname_end += 1;
+
+    if (hostname_end[0] == ':') {
+      /* port number exists in this url. get it */
+      len = authority_end - hostname_end;
+      if (len > 8)
+        ereport(ERROR,
+                (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+                 errmsg("<port> substring size must not exceed 8 characters")));
+
+      memcpy(portstr, hostname_end + 1, len);
+      portstr[len] = '\0';
+      port = atoi(portstr);
+    }
+
+    /* skippping the brackets */
+    hostname_end -= 1;
+    hostname_start += 1;
+  } else {
+    hostname_end = strchr(hostname_start, ':');
+    if (!(hostname_end && hostname_end < authority_end)) {
+      hostname_end = authority_end;
+    } else {
+      /* port number exists in this url. get it */
+      int len = authority_end - hostname_end;
+      if (len > 8)
+        ereport(ERROR,
+                (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+                 errmsg("<port> substring size must not exceed 8 characters")));
+
+      memcpy(portstr, hostname_end + 1, len);
+      portstr[len] = '\0';
+      port = atoi(portstr);
+    }
+  }
+
+  if (!port)
+    ereport(ERROR, (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+                    errmsg("<port> substring must contain only digits")));
+
+  if (hostname_end - hostname_start >= sizeof(hostname))
+    elog(ERROR, "hostname too long for url '%s'", url);
+
+  memcpy(hostname, hostname_start, hostname_end - hostname_start);
+  hostname[hostname_end - hostname_start] = 0;
+
+  hostip = getDnsAddress(hostname, port, ERROR);
+
+  /*
+   * test for the case where the URL originaly contained a domain name
+   * (so is_ipv6 was set to false) but the DNS resolution in getDnsAddress
+   * returned an IPv6 address so know we also have to put the square
+   * brackets [..] in the URL string.
+   */
+  if (strchr(hostip, ':') != NULL && !is_ipv6) domain_resolved_to_ipv6 = true;
+
+  if (!buf) {
+    int len = strlen(url) - strlen(hostname) + strlen(hostip);
+    if (domain_resolved_to_ipv6) len += 2; /* for the square brackets */
+    return len;
+  }
+
+  p = buf;
+  len = hostname_start - url;
+  strncpy(p, url, len);
+  p += len;
+  url += len;
+
+  len = strlen(hostname);
+  url += len;
+
+  len = strlen(hostip);
+  if (domain_resolved_to_ipv6) {
+    *p = '[';
+    p++;
+  }
+  strncpy(p, hostip, len);
+  p += len;
+  if (domain_resolved_to_ipv6) {
+    *p = ']';
+    p++;
+  }
+
+  strcpy(p, url);
+  p += strlen(url);
+
+  return p - buf;
+}
+
+static void extract_http_domain(char *i_path, char *o_domain, int dlen) {
+  int domsz, cpsz;
+  char *p_st = (char *)local_strstr(i_path, "//");
+  p_st = p_st + 2;
+  char *p_en = strchr(p_st, '/');
+
+  domsz = p_en - p_st;
+  cpsz = (domsz < dlen) ? domsz : dlen;
+  memcpy(o_domain, p_st, cpsz);
+}
+
+static bool url_has_ipv6_format(char *url) {
+  bool is6 = false;
+  char *ipv6 = local_strstr(url, "://[");
+
+  if (ipv6) ipv6 = strchr(ipv6, ']');
+  if (ipv6) is6 = true;
+
+  return is6;
+}
+
+static void set_httpheader(URL_FILE *fcurl, const char *name,
+                           const char *value) {
+  struct curl_slist *new_httpheader;
+  char tmp[1024];
+
+  if (strlen(name) + strlen(value) + 5 > sizeof(tmp))
+    elog(ERROR, "set_httpheader name/value is too long. name = %s, value=%s",
+         name, value);
+
+  snprintf(tmp, sizeof(tmp), "%s: %s", name, value);
+
+  new_httpheader = curl_slist_append(fcurl->u.curl.x_httpheader, tmp);
+  if (new_httpheader == NULL)
+    elog(ERROR, "could not set curl HTTP header \"%s\" to \"%s\"", name, value);
+
+  fcurl->u.curl.x_httpheader = new_httpheader;
+}
+
+static void replace_httpheader(URL_FILE *file, const char *name,
+                               const char *value) {
+  struct curl_slist *new_httpheader;
+  char tmp[1024];
+
+  if (strlen(name) + strlen(value) + 5 > sizeof(tmp))
+    elog(ERROR,
+         "replace_httpheader name/value is too long. name = %s, value=%s", name,
+         value);
+
+  sprintf(tmp, "%s: %s", name, value);
+
+  /* Find existing header, if any */
+  struct curl_slist *p = file->u.curl.x_httpheader;
+  while (p != NULL) {
+    if (!strncmp(name, p->data, strlen(name))) {
+      /*
+       * NOTE: p->data is not palloc'd! It is originally allocated
+       * by curl_slist_append, so use plain malloc/free here as well.
+       */
+      char *dupdata = strdup(tmp);
+
+      if (dupdata == NULL) elog(ERROR, "out of memory");
+
+      free(p->data);
+      p->data = dupdata;
+      return;
+    }
+    p = p->next;
+  }
+
+  /* No existing header, add a new one */
+
+  new_httpheader = curl_slist_append(file->u.curl.x_httpheader, tmp);
+  if (new_httpheader == NULL)
+    elog(ERROR, "could not append HTTP header \"%s\"", name);
+  file->u.curl.x_httpheader = new_httpheader;
+}
+
+// callback for request /gpfdist/status for debugging purpose.
+static size_t log_http_body(char *buffer, size_t size, size_t nitems,
+                            void *userp) {
+  char body[256] = {0};
+  int nbytes = size * nitems;
+  int len = sizeof(body) - 1 > nbytes ? nbytes : sizeof(body) - 1;
+
+  memcpy(body, buffer, len);
+
+  elog(LOG, "gpfdist/status: %s", body);
+
+  return nbytes;
+}
+
+// GET /gpfdist/status to get gpfdist status.
+static void get_gpfdist_status(URL_FILE *file) {
+  CURL *status_handle = NULL;
+  char status_url[256];
+  char domain[HOST_NAME_SIZE] = {0};
+  CURLcode e;
+
+  extract_http_domain(file->url, domain, HOST_NAME_SIZE);
+  snprintf(status_url, sizeof(status_url), "http://%s/gpfdist/status", domain);
+
+  do {
+    if (!(status_handle = curl_easy_init())) {
+      elog(LOG, "internal error: get_gpfdist_status.curl_easy_init failed");
+      break;
+    }
+    if (CURLE_OK !=
+        (e = curl_easy_setopt(status_handle, CURLOPT_TIMEOUT, 60L))) {
+      elog(LOG,
+           "internal error: get_gpfdist_status.curl_easy_setopt "
+           "CURLOPT_TIMEOUT error (%d - %s)",
+           e, curl_easy_strerror(e));
+      break;
+    }
+    if (CURLE_OK !=
+        (e = curl_easy_setopt(status_handle, CURLOPT_URL, status_url))) {
+      elog(LOG,
+           "internal error: get_gpfdist_status.curl_easy_setopt CURLOPT_URL "
+           "error (%d - %s)",
+           e, curl_easy_strerror(e));
+      break;
+    }
+    if (CURLE_OK != (e = curl_easy_setopt(status_handle, CURLOPT_WRITEFUNCTION,
+                                          log_http_body))) {
+      elog(LOG,
+           "internal error: get_gpfdist_status.curl_easy_setopt "
+           "CURLOPT_WRITEFUNCTION error (%d - %s)",
+           e, curl_easy_strerror(e));
+      break;
+    }
+    if (CURLE_OK != (e = curl_easy_perform(status_handle))) {
+      elog(LOG, "send status request failed: %s", curl_easy_strerror(e));
+    }
+  } while (0);
+
+  curl_easy_cleanup(status_handle);
+}
+
+/*
+ * fill_buffer
+ *
+ * Attempt to fill the read buffer up to requested number of bytes.
+ * We first check if we already have the number of bytes that we
+ * want already in the buffer (from write_callback), and we do
+ * a select on the socket only if we don't have enough.
+ *
+ * return 0 if successful; raises ERROR otherwise.
+ */
+static int fill_buffer(URL_FILE *file, int want) {
+  fd_set fdread;
+  fd_set fdwrite;
+  fd_set fdexcep;
+  int maxfd = 0;
+  int nfds = 0, e = 0;
+  int timeout_count = 0;
+  curlctl_t *curl = &file->u.curl;
+
+  /* attempt to fill buffer */
+  while (curl->still_running && curl->in.top - curl->in.bot < want) {
+    FD_ZERO(&fdread);
+    FD_ZERO(&fdwrite);
+    FD_ZERO(&fdexcep);
+
+    CHECK_FOR_INTERRUPTS();
+
+    /* set a suitable timeout to fail on */
+    struct timeval timeout;
+    timeout.tv_sec = 5;
+    timeout.tv_usec = 0;
+
+    /* get file descriptors from the transfers */
+    if (0 != (e = curl_multi_fdset(multi_handle, &fdread, &fdwrite, &fdexcep,
+                                   &maxfd))) {
+      elog(ERROR, "internal error: curl_multi_fdset failed (%d - %s)", e,
+           curl_easy_strerror(e));
+    }
+
+    /* When libcurl returns -1 in max_fd, it is because libcurl currently does
+     * something that isn't possible for your application to monitor with a
+     * socket and unfortunately you can then not know exactly when the current
+     * action is completed using select(). You then need to wait a while before
+     * you proceed and call curl_multi_perform anyway. How long to wait? Unless
+     * curl_multi_timeout gives you a lower number, we suggest 100 milliseconds
+     * or so, but you may want to test it out in your own particular conditions
+     * to find a suitable value.*/
+    if (maxfd == -1) {
+      pg_usleep(100);
+      nfds = 1;
+    } else {
+      nfds = select(maxfd + 1, &fdread, &fdwrite, &fdexcep, &timeout);
+    }
+    if (nfds == -1) {
+      if (errno == EINTR || errno == EAGAIN) {
+        elog(DEBUG2, "select failed on curl_multi_fdset (maxfd %d) (%d - %s)",
+             maxfd, errno, strerror(errno));
+        continue;
+      }
+      elog(ERROR,
+           "internal error: select failed on curl_multi_fdset (maxfd %d) (%d - "
+           "%s)",
+           maxfd, errno, strerror(errno));
+    } else if (nfds == 0) {
+      // timeout
+      timeout_count++;
+
+      if (timeout_count % 12 == 0) {
+        elog(LOG,
+             "segment has not received data from gpfdist for about 1 minute, "
+             "waiting for %d bytes.",
+             (want - (curl->in.top - curl->in.bot)));
+      }
+
+      if (readable_external_table_timeout != 0 &&
+          timeout_count * 5 > readable_external_table_timeout) {
+        elog(LOG,
+             "bot = %d, top = %d, want = %d, maxfd = %d, nfds = %d, e = %d, "
+             "still_running = %d, for_write = %d, error = %d, eof = %d, "
+             "datalen = %d",
+             curl->in.bot, curl->in.top, want, maxfd, nfds, e,
+             curl->still_running, curl->for_write, curl->error, curl->eof,
+             curl->block.datalen);
+        get_gpfdist_status(file);
+        ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE),
+                        errmsg("segment has not received data from gpfdist for "
+                               "long time, cancelling the query.")));
+        break;
+      }
+    } else if (nfds > 0) {
+      /* timeout or readable/writable sockets */
+      /* note we *could* be more efficient and not wait for
+       * CURLM_CALL_MULTI_PERFORM to clear here and check it on re-entry
+       * but that gets messy */
+      while (CURLM_CALL_MULTI_PERFORM ==
+             (e = curl_multi_perform(multi_handle, &curl->still_running)))
+        ;
+
+      if (e != 0) {
+        elog(ERROR, "internal error: curl_multi_perform failed (%d - %s)", e,
+             curl_easy_strerror(e));
+      }
+    } else {
+      elog(ERROR, "select return unexpected result");
+    }
+  }
+
+  if (curl->still_running == 0) {
+    elog(LOG,
+         "quit fill_buffer due to still_running = 0, bot = %d, top = %d, want "
+         "= %d, "
+         "for_write = %d, error = %d, eof = %d, datalen = %d, maxfd = %d, nfds "
+         "= %d, e = %d",
+         curl->in.bot, curl->in.top, want, curl->for_write, curl->error,
+         curl->eof, curl->block.datalen, maxfd, nfds, e);
+  }
+
+  return 0;
+}
+
+/*
+ * check_response
+ *
+ * If got an HTTP response with an error code from the server (gpfdist), report
+ * the error code and message it to the database user and abort operation.
+ */
+static int check_response(URL_FILE *file, int *rc, char **response_string) {
+  long response_code;
+  char *effective_url = NULL;
+  CURL *curl = file->u.curl.handle;
+  char buffer[30];
+
+  /* get the response code from curl */
+  if (curl_easy_getinfo(curl, CURLINFO_RESPONSE_CODE, &response_code) !=
+      CURLE_OK) {
+    *rc = 500;
+    *response_string = pstrdup("curl_easy_getinfo failed");
+    return -1;
+  }
+  *rc = response_code;
+  snprintf(buffer, sizeof buffer, "Response Code=%d", (int)response_code);
+  *response_string = pstrdup(buffer);
+
+  if (curl_easy_getinfo(curl, CURLINFO_EFFECTIVE_URL, &effective_url) !=
+      CURLE_OK)
+    return -1;
+  if (effective_url == NULL) effective_url = "";
+
+  if (!(200 <= response_code && response_code < 300)) {
+    if (response_code == 0) {
+      long oserrno = 0;
+      static char connmsg[64];
+
+      /* get the os level errno, and string representation of it */
+      if (curl_easy_getinfo(curl, CURLINFO_OS_ERRNO, &oserrno) == CURLE_OK) {
+        if (oserrno != 0)
+          snprintf(connmsg, sizeof connmsg, "error code = %d (%s)",
+                   (int)oserrno, strerror((int)oserrno));
+      }
+
+      ereport(
+          ERROR,
+          (errcode(ERRCODE_CONNECTION_FAILURE),
+           errmsg("connection with gpfdist failed for \"%s\", effective url: "
+                  "\"%s\". %s",
+                  file->url, effective_url, (oserrno != 0 ? connmsg : ""))));
+    } else if (response_code ==
+               FDIST_TIMEOUT)  // gpfdist server return timeout code
+    {
+      return FDIST_TIMEOUT;
+    } else {
+      /* we need to sleep 1 sec to avoid this condition:
+         1- seg X gets an error message from gpfdist
+         2- seg Y gets a 500 error
+         3- seg Y report error before seg X, and error message
+         in seg X is thrown away.
+      */
+      pg_usleep(1000000);
+
+      ereport(ERROR,
+              (errcode(ERRCODE_CONNECTION_FAILURE),
+               errmsg("http response code %ld from gpfdist (%s): %s",
+                      response_code, file->url,
+                      file->u.curl.http_response ? file->u.curl.http_response
+                                                 : "?")));
+    }
+  }
+
+  return 0;
+}
+
+/**
+ * Send curl request and check response.
+ * If failed, will retry multiple times.
+ * Return true if succeed, false otherwise.
+ */
+static void gp_curl_easy_perform_backoff_and_check_response(URL_FILE *file) {
+  int response_code;
+  char *response_string = NULL;
+
+  /* retry in case server return timeout error */
+  unsigned int wait_time = 1;
+  unsigned int retry_count = 0;
+  /* retry at most twice(300 seconds * 2) when CURLE_OPERATION_TIMEDOUT happens
+   */
+  unsigned int timeout_count = 0;
+
+  while (true) {
+    /*
+     * Use backoff policy to call curl_easy_perform to fix following error
+     * when work load is high:
+     *  - 'could not connect to server'
+     *  - gpfdist return timeout (HTTP 408)
+     * By default it will wait at most 127 seconds before abort.
+     * 1 + 2 + 4 + 8 + 16 + 32 + 64 = 127
+     */
+    CURLcode e = curl_easy_perform(file->u.curl.handle);
+    if (CURLE_OK != e) {
+      elog(WARNING, "%s error (%d - %s)", file->u.curl.curl_url, e,
+           curl_easy_strerror(e));
+      if (CURLE_OPERATION_TIMEDOUT == e) {
+        timeout_count++;
+      }
+    } else {
+      /* check the response from server */
+      response_code = check_response(file, &response_code, &response_string);
+      switch (response_code) {
+        case 0:
+          /* Success! */
+          return;
+
+        case FDIST_TIMEOUT:
+          break;
+
+        default:
+          ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE),
+                          errmsg("error while getting response from gpfdist on "
+                                 "%s (code %d, msg %s)",
+                                 file->u.curl.curl_url, response_code,
+                                 response_string)));
+      }
+      if (response_string) pfree(response_string);
+      response_string = NULL;
+    }
+
+    if (wait_time > MAX_TRY_WAIT_TIME || timeout_count >= 2) {
+      ereport(
+          ERROR,
+          (errcode(ERRCODE_CONNECTION_FAILURE),
+           errmsg("error when writing data to gpfdist %s, quit after %d tries",
+                  file->u.curl.curl_url, retry_count + 1)));
+    } else {
+      elog(
+          WARNING,
+          "failed to send request to gpfdist (%s), will retry after %d seconds",
+          file->u.curl.curl_url, wait_time);
+      unsigned int for_wait = 0;
+      while (for_wait++ < wait_time) {
+        pg_usleep(1000000);
+        CHECK_FOR_INTERRUPTS();
+      }
+      wait_time = wait_time + wait_time;
+      retry_count++;
+    }
+  }
+}
+
+/*
+ * Send an empty POST request, with an added X-GP-DONE header.
+ */
+static void gp_proto0_write_done(URL_FILE *file) {
+  set_httpheader(file, "X-GP-DONE", "1");
+
+  /* use empty message */
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_POSTFIELDS, "");
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_POSTFIELDSIZE, 0);
+
+  /* post away! */
+  gp_curl_easy_perform_backoff_and_check_response(file);
+}
+
+/*
+ * gp_proto0_read
+ *
+ * get data from the server and handle it according to PROTO 0. In PROTO 0 we
+ * expect the content of the file without any kind of meta info. Simple.
+ */
+static size_t gp_proto0_read(char *buf, int bufsz, URL_FILE *file) {
+  int n = 0;
+  curlctl_t *curl = &file->u.curl;
+
+  fill_buffer(file, bufsz);
+
+  /* check if there's data in the buffer - if not fill_buffer()
+   * either errored or EOF. For proto0, we cannot distinguish
+   * between error and EOF. */
+  n = curl->in.top - curl->in.bot;
+  if (n == 0 && !curl->still_running) curl->eof = 1;
+
+  if (n > bufsz) n = bufsz;
+
+  /* xfer data to caller */
+  memcpy(buf, curl->in.ptr, n);
+  curl->in.bot += n;
+
+  return n;
+}
+
+/*
+ * gp_proto1_read
+ *
+ * get data from the server and handle it according to PROTO 1. In this protocol
+ * each data block is tagged by meta info like this:
+ * byte 0: type (can be 'F'ilename, 'O'ffset, 'D'ata, 'E'rror, 'L'inenumber)
+ * byte 1-4: length. # bytes of following data block. in network-order.
+ * byte 5-X: the block itself.
+ */
+static size_t gp_proto1_read(char *buf, int bufsz, URL_FILE *file,
+                             CopyState pstate, char *buf2) {
+  char type;
+  int n, len;
+  curlctl_t *curl = &file->u.curl;
+
+  /*
+   * Loop through and get all types of messages, until we get actual data,
+   * or until there's no more data. Then quit the loop to process it and
+   * return it.
+   */
+  while (curl->block.datalen == 0 && !curl->eof) {
+    /* need 5 bytes, 1 byte type + 4 bytes length */
+    fill_buffer(file, 5);
+    n = curl->in.top - curl->in.bot;
+
+    if (n == 0)
+      ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE),
+                      errmsg("gpfdist error: server closed connection")));
+
+    if (n < 5)
+      ereport(ERROR,
+              (errcode(ERRCODE_CONNECTION_FAILURE),
+               errmsg("gpfdist error: incomplete packet - packet len %d", n)));
+
+    /* read type */
+    type = curl->in.ptr[curl->in.bot++];
+
+    /* read len */
+    memcpy(&len, &curl->in.ptr[curl->in.bot], 4);
+    len = ntohl(len); /* change order */
+    curl->in.bot += 4;
+
+    if (len < 0)
+      elog(ERROR, "gpfdist error: bad packet type %d len %d", type, len);
+
+    /* Error */
+    if (type == 'E') {
+      fill_buffer(file, len);
+      n = curl->in.top - curl->in.bot;
+
+      if (n > len) n = len;
+
+      if (n > 0) {
+        /*
+         * cheat a little. swap last char and
+         * NUL-terminator. then print string (without last
+         * char) and print last char artificially
+         */
+        char x = curl->in.ptr[curl->in.bot + n - 1];
+        curl->in.ptr[curl->in.bot + n - 1] = 0;
+        ereport(ERROR, (errcode(ERRCODE_DATA_EXCEPTION),
+                        errmsg("gpfdist error - %s%c",
+                               &curl->in.ptr[curl->in.bot], x)));
+      }
+
+      elog(ERROR, "gpfdist error: please check gpfdist log messages.");
+    }
+
+    /* Filename */
+    if (type == 'F') {
+      if (buf != buf2) {
+        curl->in.bot -= 5;
+        return 0;
+      }
+      if (len > 256) elog(ERROR, "gpfdist error: filename too long (%d)", len);
+
+      if (-1 == fill_buffer(file, len))
+        elog(ERROR, "gpfdist error: stream ends suddenly");
+
+      /*
+       * If SREH is used we now update it with the actual file that the
+       * gpfdist server is reading. This is because SREH (or the client
+       * in general) doesn't know which file gpfdist is reading, since
+       * the original URL may include a wildcard or a directory listing.
+       */
+      if (pstate->cdbsreh) {
+        char fname[257];
+
+        memcpy(fname, curl->in.ptr + curl->in.bot, len);
+        fname[len] = 0;
+        snprintf(pstate->cdbsreh->filename, sizeof pstate->cdbsreh->filename,
+                 "%s [%s]", pstate->filename, fname);
+      }
+
+      curl->in.bot += len;
+      Assert(curl->in.bot <= curl->in.top);
+      continue;
+    }
+
+    /* Offset */
+    if (type == 'O') {
+      if (len != 8)
+        elog(ERROR, "gpfdist error: offset not of length 8 (%d)", len);
+
+      if (-1 == fill_buffer(file, len))
+        elog(ERROR, "gpfdist error: stream ends suddenly");
+
+      curl->in.bot += 8;
+      Assert(curl->in.bot <= curl->in.top);
+      continue;
+    }
+
+    /* Line number */
+    if (type == 'L') {
+      int64 line_number;
+
+      if (len != 8)
+        elog(ERROR, "gpfdist error: line number not of length 8 (%d)", len);
+
+      if (-1 == fill_buffer(file, len))
+        elog(ERROR, "gpfdist error: stream ends suddenly");
+
+      /*
+       * update the line number of the first line we're about to get from
+       * gpfdist. pstate will update the following lines when processing
+       * the data
+       */
+      memcpy(&line_number, curl->in.ptr + curl->in.bot, len);
+      line_number = local_ntohll(line_number);
+      pstate->cur_lineno = line_number ? line_number - 1 : INT64_MIN;
+      curl->in.bot += 8;
+      Assert(curl->in.bot <= curl->in.top);
+      continue;
+    }
+
+    /* Data */
+    if (type == 'D') {
+      curl->block.datalen = len;
+      curl->eof = (len == 0);
+      break;
+    }
+
+    elog(ERROR, "gpfdist error: unknown meta type %d", type);
+  }
+
+  /* read data block */
+  if (bufsz > curl->block.datalen) bufsz = curl->block.datalen;
+
+  fill_buffer(file, bufsz);
+  n = curl->in.top - curl->in.bot;
+
+  /* if gpfdist closed connection prematurely or died catch it here */
+  if (n == 0 && !curl->eof) {
+    curl->error = 1;
+
+    if (!curl->still_running)
+      ereport(
+          ERROR,
+          (errcode(ERRCODE_CONNECTION_FAILURE),
+           errmsg("gpfdist server closed connection"),
+           errhint(
+               "The root cause is likely to be an overload of the ETL host or "
+               "a temporary network glitch between the database and the ETL "
+               "host "
+               "causing the connection between the gpfdist and database to "
+               "disconnect.")));
+  }
+
+  if (n > bufsz) n = bufsz;
+
+  memcpy(buf, curl->in.ptr + curl->in.bot, n);
+  curl->in.bot += n;
+  curl->block.datalen -= n;
+  return n;
+}
+
+static size_t curl_fread(char *buf, int bufsz, URL_FILE *file,
+                         CopyState pstate) {
+  curlctl_t *curl = &file->u.curl;
+  char *p = buf;
+  char *q = buf + bufsz;
+  int n;
+  const int gp_proto = curl->gp_proto;
+
+  if (gp_proto != 0 && gp_proto != 1)
+    elog(ERROR, "unknown gp protocol %d", curl->gp_proto);
+
+  for (; p < q; p += n) {
+    if (gp_proto == 0)
+      n = gp_proto0_read(p, q - p, file);
+    else
+      n = gp_proto1_read(p, q - p, file, pstate, buf);
+
+    if (n <= 0) break;
+  }
+
+  return p - buf;
+}
+
+/*
+ * gp_proto0_write
+ *
+ * use curl to write data to a the remote gpfdist server. We use
+ * a push model with a POST request.
+ *
+ */
+static void gp_proto0_write(URL_FILE *file, CopyState pstate) {
+  curlctl_t *curl = &file->u.curl;
+  char *buf = curl->out.ptr;
+  int nbytes = curl->out.top;
+  if (nbytes == 0) return;
+  /* post binary data */
+  CURL_EASY_SETOPT(curl->handle, CURLOPT_POSTFIELDS, buf);
+
+  /* set the size of the postfields data */
+  CURL_EASY_SETOPT(curl->handle, CURLOPT_POSTFIELDSIZE, nbytes);
+
+  /* set sequence number */
+  char seq[128] = {0};
+  snprintf(seq, sizeof(seq), INT64_FORMAT, file->u.curl.seq_number);
+
+  replace_httpheader(file, "X-GP-SEQ", seq);
+
+  gp_curl_easy_perform_backoff_and_check_response(file);
+}
+
+static size_t curl_fwrite(char *buf, int nbytes, URL_FILE *file,
+                          CopyState pstate) {
+  curlctl_t *curl = &file->u.curl;
+
+  if (!curl->for_write)
+    elog(ERROR, "cannot write to a read-mode external table");
+
+  if (curl->gp_proto != 0 && curl->gp_proto != 1)
+    elog(ERROR, "unknown gp protocol %d", curl->gp_proto);
+
+  /*
+   * if buffer is full (current item can't fit) - write it out to
+   * the server. if item still doesn't fit after we emptied the
+   * buffer, make more room.
+   */
+  if (curl->out.top + nbytes >= curl->out.max) {
+    /* item doesn't fit */
+    if (curl->out.top > 0) {
+      /* write out existing data, empty the buffer */
+      gp_proto0_write(file, pstate);
+      curl->out.top = 0;
+    }
+
+    /* does it still not fit? enlarge buffer */
+    if (curl->out.top + nbytes >= curl->out.max) {
+      int n = nbytes + 1024;
+      char *newbuf;
+
+      newbuf = repalloc(curl->out.ptr, n);
+
+      if (!newbuf) elog(ERROR, "out of memory (curl_fwrite)");
+
+      curl->out.ptr = newbuf;
+      curl->out.max = n;
+
+      Assert(nbytes < curl->out.max);
+    }
+  }
+
+  /* copy buffer into file->buf */
+  memcpy(curl->out.ptr + curl->out.top, buf, nbytes);
+  curl->out.top += nbytes;
+
+  return nbytes;
+}
+
+URL_FILE *url_curl_fopen(char *url, bool forwrite, extvar_t *ev,
+                         CopyState pstate) {
+  Assert(IS_HTTP_URI(url) || IS_GPFDIST_URI(url));
+
+  URL_FILE *file = NULL;
+
+  bool is_ipv6 = url_has_ipv6_format(url);
+  int sz = make_url(url, NULL, is_ipv6);
+  if (sz < 0) elog(ERROR, "illegal URL: %s", url);
+
+  file = (URL_FILE *)palloc0(sizeof(URL_FILE));
+  file->type = CFTYPE_CURL;
+  file->url = pstrdup(url);
+  file->u.curl.for_write = forwrite;
+
+  file->u.curl.curl_url = (char *)palloc0(sz + 1);
+  make_url(file->url, file->u.curl.curl_url, is_ipv6);
+  /*
+   * We need to call is_url_ipv6 for the case where inside make_url function
+   * a domain name was transformed to an IPv6 address.
+   */
+  if (!is_ipv6) is_ipv6 = url_has_ipv6_format(file->u.curl.curl_url);
+
+  if (IS_GPFDIST_URI(file->u.curl.curl_url)) {
+    /* replace gpfdist:// with http:// or gpfdists:// with https://
+     * by overriding 'dist' with 'http' */
+    unsigned int tmp_len = strlen(file->u.curl.curl_url) + 1;
+    memmove(file->u.curl.curl_url, file->u.curl.curl_url + 3, tmp_len - 3);
+    memcpy(file->u.curl.curl_url, "http", 4);
+    pstate->header_line = 0;
+  }
+
+  /* initialize a curl session and get a libcurl handle for it */
+  if (!(file->u.curl.handle = curl_easy_init()))
+    elog(ERROR, "internal error: curl_easy_init failed");
+
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_URL, file->u.curl.curl_url);
+
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_VERBOSE, 0L /* FALSE */);
+
+  /* set callback for each header received from server */
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_HEADERFUNCTION,
+                   header_callback);
+
+  /* 'file' is the application variable that gets passed to header_callback */
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_WRITEHEADER, file);
+
+  /* set callback for each data block arriving from server to be written to
+   * application */
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_WRITEFUNCTION, write_callback);
+
+  /* 'file' is the application variable that gets passed to write_callback */
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_WRITEDATA, file);
+
+  int ip_mode;
+  if (!is_ipv6)
+    ip_mode = CURL_IPRESOLVE_V4;
+  else
+    ip_mode = CURL_IPRESOLVE_V6;
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_IPRESOLVE, ip_mode);
+
+  /*
+   * set up a linked list of http headers. start with common headers
+   * needed for read and write operations, and continue below with
+   * more specifics
+   */
+  Assert(file->u.curl.x_httpheader == NULL);
+
+  /*
+   * support multihomed http use cases. see MPP-11874
+   */
+  if (IS_HTTP_URI(url)) {
+    char domain[HOST_NAME_SIZE] = {0};
+
+    extract_http_domain(file->url, domain, HOST_NAME_SIZE);
+    set_httpheader(file, "Host", domain);
+  }
+
+  set_httpheader(file, "X-GP-XID", ev->GP_XID);
+  set_httpheader(file, "X-GP-CID", ev->GP_CID);
+  set_httpheader(file, "X-GP-SN", ev->GP_SN);
+  set_httpheader(file, "X-GP-SEGMENT-ID", ev->GP_SEGMENT_ID);
+  set_httpheader(file, "X-GP-SEGMENT-COUNT", ev->GP_SEGMENT_COUNT);
+
+  if (forwrite) {
+    // TIMEOUT for POST only, GET is single HTTP request,
+    // probablity take long time.
+    CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_TIMEOUT, 300L);
+
+    /*init sequence number*/
+    file->u.curl.seq_number = 1;
+
+    /* write specific headers */
+    set_httpheader(file, "X-GP-PROTO", "0");
+    set_httpheader(file, "X-GP-SEQ", "1");
+    set_httpheader(file, "Content-Type", "text/xml");
+  } else {
+    /* read specific - (TODO: unclear why some of these are needed) */
+    set_httpheader(file, "X-GP-PROTO", "1");
+    set_httpheader(file, "X-GP-MASTER_HOST", ev->GP_MASTER_HOST);
+    set_httpheader(file, "X-GP-MASTER_PORT", ev->GP_MASTER_PORT);
+    set_httpheader(file, "X-GP-CSVOPT", ev->GP_CSVOPT);
+    set_httpheader(file, "X-GP_SEG_PG_CONF", ev->GP_SEG_PG_CONF);
+    set_httpheader(file, "X-GP_SEG_DATADIR", ev->GP_SEG_DATADIR);
+    set_httpheader(file, "X-GP-DATABASE", ev->GP_DATABASE);
+    set_httpheader(file, "X-GP-USER", ev->GP_USER);
+    set_httpheader(file, "X-GP-SEG-PORT", ev->GP_SEG_PORT);
+    set_httpheader(file, "X-GP-SESSION-ID", ev->GP_SESSION_ID);
+  }
+
+  {
+    /*
+     * MPP-13031
+     * copy #transform fragment, if present, into X-GP-TRANSFORM header
+     */
+    char *p = local_strstr(file->url, "#transform=");
+    if (p && p[11]) set_httpheader(file, "X-GP-TRANSFORM", p + 11);
+  }
+
+  CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_HTTPHEADER,
+                   file->u.curl.x_httpheader);
+
+  if (!multi_handle) {
+    if (!(multi_handle = curl_multi_init()))
+      elog(ERROR, "internal error: curl_multi_init failed");
+  }
+
+  /* Allocate input and output buffers. */
+  file->u.curl.in.ptr = palloc(1024); /* 1 kB buffer initially */
+  file->u.curl.in.max = 1024;
+  file->u.curl.in.bot = file->u.curl.in.top = 0;
+
+  if (forwrite) {
+    int bufsize = writable_external_table_bufsize * 1024;
+    file->u.curl.out.ptr = (char *)palloc(bufsize);
+    file->u.curl.out.max = bufsize;
+    file->u.curl.out.bot = file->u.curl.out.top = 0;
+  }
+
+  /*
+   * lets check our connection.
+   * start the fetch if we're SELECTing (GET request), or write an
+   * empty message if we're INSERTing (POST request)
+   */
+  if (!forwrite) {
+    int e;
+    int response_code;
+    char *response_string;
+
+    if (CURLE_OK !=
+        (e = curl_multi_add_handle(multi_handle, file->u.curl.handle))) {
+      if (CURLM_CALL_MULTI_PERFORM != e)
+        elog(ERROR, "internal error: curl_multi_add_handle failed (%d - %s)", e,
+             curl_easy_strerror(e));
+    }
+
+    while (CURLM_CALL_MULTI_PERFORM ==
+           (e = curl_multi_perform(multi_handle, &file->u.curl.still_running)))
+      ;
+
+    if (e != CURLE_OK)
+      elog(ERROR, "internal error: curl_multi_perform failed (%d - %s)", e,
+           curl_easy_strerror(e));
+
+    /* read some bytes to make sure the connection is established */
+    fill_buffer(file, 1);
+
+    /* check the connection for GET request */
+    if (check_response(file, &response_code, &response_string))
+      ereport(ERROR,
+              (errcode(ERRCODE_CONNECTION_FAILURE),
+               errmsg("could not open \"%s\" for reading", file->url),
+               errdetail("Unexpected response from gpfdist server: %d - %s",
+                         response_code, response_string)));
+  } else {
+    /* use empty message */
+    CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_POSTFIELDS, "");
+    CURL_EASY_SETOPT(file->u.curl.handle, CURLOPT_POSTFIELDSIZE, 0);
+
+    /* post away and check response, retry if failed (timeout or * connect
+     * error) */
+    gp_curl_easy_perform_backoff_and_check_response(file);
+    file->u.curl.seq_number++;
+  }
+
+  return file;
+}
+
+void url_curl_fclose(URL_FILE *file, bool failOnError, const char *relname) {
+  /*
+   * if WET, send a final "I'm done" request from this segment.
+   */
+  if (file->u.curl.for_write && file->u.curl.handle != NULL)
+    gp_proto0_write_done(file);
+
+  if (file->u.curl.x_httpheader) {
+    curl_slist_free_all(file->u.curl.x_httpheader);
+    file->u.curl.x_httpheader = NULL;
+  }
+
+  /* make sure the easy handle is not in the multi handle anymore */
+  if (file->u.curl.handle) {
+    curl_multi_remove_handle(multi_handle, file->u.curl.handle);
+    /* cleanup */
+    curl_easy_cleanup(file->u.curl.handle);
+    file->u.curl.handle = NULL;
+  }
+
+  /* free any allocated buffer space */
+  if (file->u.curl.in.ptr) {
+    pfree(file->u.curl.in.ptr);
+    file->u.curl.in.ptr = NULL;
+  }
+
+  if (file->u.curl.curl_url) {
+    pfree(file->u.curl.curl_url);
+    file->u.curl.curl_url = NULL;
+  }
+
+  if (file->u.curl.out.ptr) {
+    Assert(file->u.curl.for_write);
+    pfree(file->u.curl.out.ptr);
+    file->u.curl.out.ptr = NULL;
+  }
+
+  file->u.curl.gp_proto = 0;
+  file->u.curl.error = file->u.curl.eof = 0;
+  memset(&file->u.curl.in, 0, sizeof(file->u.curl.in));
+  memset(&file->u.curl.block, 0, sizeof(file->u.curl.block));
+
+  pfree(file->url);
+
+  pfree(file);
+}
+
+bool url_curl_feof(URL_FILE *file, int bytesread) {
+  return (file->u.curl.eof != 0);
+}
+
+bool url_curl_ferror(URL_FILE *file, int bytesread, char *ebuf, int ebuflen) {
+  return (file->u.curl.error != 0);
+}
+
+size_t url_curl_fread(void *ptr, size_t size, URL_FILE *file,
+                      CopyState pstate) {
+  /* get data (up size) from the http/gpfdist server */
+  return curl_fread(ptr, size, file, pstate);
+}
+
+size_t url_curl_fwrite(void *ptr, size_t size, URL_FILE *file,
+                       CopyState pstate) {
+  /* write data to the gpfdist server via curl */
+  return curl_fwrite(ptr, size, file, pstate);
+}
+
+void url_curl_fflush(URL_FILE *file, CopyState pstate) {
+  gp_proto0_write(file, pstate);
+}
+
+void url_curl_rewind(URL_FILE *file, const char *relname) {
+  /* halt transaction */
+  curl_multi_remove_handle(multi_handle, file->u.curl.handle);
+
+  /* restart */
+  curl_multi_add_handle(multi_handle, file->u.curl.handle);
+
+  /* ditch buffer - write will recreate - resets stream pos*/
+  if (file->u.curl.in.ptr) pfree(file->u.curl.in.ptr);
+
+  file->u.curl.gp_proto = 0;
+  file->u.curl.error = file->u.curl.eof = 0;
+  memset(&file->u.curl.in, 0, sizeof(file->u.curl.in));
+  memset(&file->u.curl.block, 0, sizeof(file->u.curl.block));
+}
diff --git a/src/backend/utils/misc/fstream/gfile.c b/src/backend/utils/misc/fstream/gfile.c
index 21b9697..6889e6e 100644
--- a/src/backend/utils/misc/fstream/gfile.c
+++ b/src/backend/utils/misc/fstream/gfile.c
@@ -42,6 +42,7 @@
 
 #include <errno.h>
 #include <fcntl.h>
+#include <sys/file.h>   /* for flock */
 #include <sys/stat.h>
 
 
@@ -707,6 +708,10 @@ int gfile_open(gfile_t* fd, const char* fpath, int flags, int* response_code, co
 #define S_IFDIR  _S_IFDIR
 #define S_ISDIR(m) (((m) & S_IFMT) == S_IFDIR)
 #endif
+#if !defined(S_ISFIFO)
+#define S_IFIFO _S_IFIFO
+#define S_ISFIFO(m) (((m) & S_IFMT) == S_IFIFO)
+#endif
 #define strcasecmp stricmp
 #endif 
 
@@ -886,6 +891,28 @@ int gfile_open(gfile_t* fd, const char* fpath, int flags, int* response_code, co
 		return 1;
 	}
 
+#if !defined(WIN32) && !defined(_AIX)
+  if (!is_win_pipe && (flags == GFILE_OPEN_FOR_READ))
+  {
+    /* Restrict only one reader session for each PIPE */
+    if (S_ISFIFO(sta.st_mode))
+    {
+      if (flock (fd->fd.filefd, LOCK_EX | LOCK_NB) != 0)
+      {
+        fd->held_pipe_lock = FALSE;
+        gfile_printf_then_putc_newline("gfile %s is a pipe", fpath);
+        *response_code = 404;
+        *response_string = "Multiple reader to a pipe is forbidden.";
+        return 1;
+      }
+      else
+      {
+        fd->held_pipe_lock = TRUE;
+      }
+    }
+  }
+#endif
+
 	/*
 	 * prepare to use the appropriate i/o routines 
 	 */
@@ -995,17 +1022,19 @@ gfile_close(gfile_t*fd)
 
 		if (!fd->is_win_pipe)
 		{
-			int i;
+      if (fd->held_pipe_lock) {
+#ifndef WIN32
+        flock(fd->fd.filefd, LOCK_UN);
+#endif
+      }
 
-			do
-			{
-				//fsync(fd->fd.filefd);
-				i = close(fd->fd.filefd);
-			}
-			while (i < 0 && errno == EINTR);
+      int i;
+      do {
+        // fsync(fd->fd.filefd);
+        i = close(fd->fd.filefd);
+      } while (i < 0 && errno == EINTR);
 
-			if (e == 0)
-				e = i;
+      if (e == 0) e = i;
 		}
 
 #ifdef GPFXDIST
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 6577b4c..15b6eb1 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -790,6 +790,10 @@ char   *acl_type;
 int    rps_addr_port;
 int    rps_check_local_interval;
 
+// curl url external table related guc
+int writable_external_table_bufsize = 64;
+int readable_external_table_timeout = 0;
+
 /*
  * Displayable names for context types (enum GucContext)
  *
@@ -5307,6 +5311,28 @@ static struct config_int ConfigureNamesInt[] =
 		UNINITIALIZED_GP_IDENTITY_VALUE, INT_MIN, INT_MAX, NULL, NULL
 	},
 
+  {
+    {"readable_external_table_timeout", PGC_USERSET, EXTERNAL_TABLES,
+      gettext_noop("Cancel the query if no data read within N seconds."),
+      gettext_noop("A value of 0 turns off the timeout."),
+      GUC_UNIT_S | GUC_NOT_IN_SAMPLE
+    },
+    &readable_external_table_timeout,
+    0, 0, INT_MAX,
+    NULL, NULL, NULL
+  },
+
+  {
+    {"writable_external_table_bufsize", PGC_USERSET, EXTERNAL_TABLES,
+      gettext_noop("Buffer size in kilobytes for writable external table before writing data to gpfdist."),
+      gettext_noop("Valid value is between 32K and 128M: [32, 131072]."),
+      GUC_UNIT_KB | GUC_NOT_IN_SAMPLE
+    },
+    &writable_external_table_bufsize,
+    64, 32, 131072,
+    NULL, NULL, NULL
+  },
+
 	{
 		{"gp_max_csv_line_length", PGC_USERSET, EXTERNAL_TABLES,
 			gettext_noop("Maximum allowed length of a csv input data row in bytes"),
diff --git a/src/bin/gpfdist/Makefile b/src/bin/gpfdist/Makefile
index ef20efe..ee6bca0 100644
--- a/src/bin/gpfdist/Makefile
+++ b/src/bin/gpfdist/Makefile
@@ -28,7 +28,7 @@ CFLAGS := -fprofile-arcs -ftest-coverage $(CFLAGS)
 endif
 
 code_dir     = $(CURDIR)/src/gpfdist/
-GPFDISTFILES = fstream.c gfile.c gpfdist.c transform.c
+GPFDISTFILES = fstream.c gfile.c gpfdist.c transform.c gpfdist_helper.c
 OBJS         = $(subst .c,.o,$(GPFDISTFILES))
 
 SHELL=bash
diff --git a/src/bin/gpfdist/src/gpfdist/glob.c b/src/bin/gpfdist/src/gpfdist/glob.c
index ff0af6d..9fab8b8 100644
--- a/src/bin/gpfdist/src/gpfdist/glob.c
+++ b/src/bin/gpfdist/src/gpfdist/glob.c
@@ -13,8 +13,10 @@
  * 2. Redistributions in binary form must reproduce the above copyright
  *    notice, this list of conditions and the following disclaimer in the
  *    documentation and/or other materials provided with the distribution.
- * 3. BSD Advertising Clause omitted per the July 22, 1999 licensing change
- *    ftp://ftp.cs.berkeley.edu/pub/4bsd/README.Impt.License.Change
+ * 3. All advertising materials mentioning features or use of this software
+ *    must display the following acknowledgement:
+ *	This product includes software developed by the University of
+ *	California, Berkeley and its contributors.
  * 4. Neither the name of the University nor the names of its contributors
  *    may be used to endorse or promote products derived from this software
  *    without specific prior written permission.
@@ -132,7 +134,6 @@ typedef unsigned int u_int;
 typedef unsigned char u_char;
 #define ARG_MAX (256 * 1024)
 #define lstat(path, sb) stat((path), (sb))
-extern size_t strlcpy(char *dst, const char *src, size_t siz);
 #else
 typedef u_short Char;
 #endif
@@ -855,11 +856,10 @@ g_opendir(str, pglob)
 	register Char *str;
 	glob_t *pglob;
 {
-	char buf[MAXPATHLEN];
+	char buf[MAXPATHLEN] = ".";
 
-	if (!*str)
-		strlcpy(buf, ".", sizeof buf);
-	else {
+	if (*str)
+	{
 		if (g_Ctoc(str, buf, sizeof(buf)))
 			return(NULL);
 	}
@@ -876,12 +876,33 @@ g_lstat(fn, sb, pglob)
 	struct stat *sb;
 	glob_t *pglob;
 {
+#ifdef WIN32
+	/*
+	 * Behavior of stat() is not stable across C Runtime versions.
+	 * If linked to system basic msvcrt.dll it calls FindFirstFile().
+	 * If linked with normal CRT comes with Visual Studio or Redistribute
+	 * packages, it calls CreateFile(). CreateFile() is problematic here,
+	 * if path is a pipe, it will open the pipe once then close, causing
+	 * the other side to connect to the wrong pipe. Skip stat() if the
+	 * name is pipe and pretend there is one.
+	 */
+	if (wcsstr(fn, L"\\\\.\\pipe") == fn)
+	{
+		memset(sb, 0, sizeof(struct stat));
+		sb->st_dev = -1;
+		sb->st_rdev = -1;
+		sb->st_nlink = 1;
+		sb->st_mode = _S_IFIFO;
+		return 0;
+	}
+#endif
 	char buf[MAXPATHLEN];
 
 	if (g_Ctoc(fn, buf, sizeof(buf)))
 		return(-1);
 	if (pglob->gl_flags & GLOB_ALTDIRFUNC)
 		return((*pglob->gl_lstat)(buf, sb));
+
 	return(lstat(buf, sb));
 }
 
@@ -891,6 +912,26 @@ g_stat(fn, sb, pglob)
 	struct stat *sb;
 	glob_t *pglob;
 {
+#ifdef WIN32
+	/*
+	 * Behavior of stat() is not stable across C Runtime versions.
+	 * If linked to system basic msvcrt.dll it calls FindFirstFile().
+	 * If linked with normal CRT comes with Visual Studio or Redistribute
+	 * packages, it calls CreateFile(). CreateFile() is problematic here,
+	 * if path is a pipe, it will open the pipe once then close, causing
+	 * the other side to connect to the wrong pipe. Skip stat() if the
+	 * name is pipe and pretend there is one.
+	 */
+	if (wcsstr(fn, L"\\\\.\\pipe") == fn)
+	{
+		memset(sb, 0, sizeof(struct stat));
+		sb->st_dev = -1;
+		sb->st_rdev = -1;
+		sb->st_nlink = 1;
+		sb->st_mode = _S_IFIFO;
+		return 0;
+	}
+#endif
 	char buf[MAXPATHLEN];
 
 	if (g_Ctoc(fn, buf, sizeof(buf)))
diff --git a/src/bin/gpfdist/src/gpfdist/gpfdist.c b/src/bin/gpfdist/src/gpfdist/gpfdist.c
index 5060fa0..fed83b8 100644
--- a/src/bin/gpfdist/src/gpfdist/gpfdist.c
+++ b/src/bin/gpfdist/src/gpfdist/gpfdist.c
@@ -1,22 +1,3 @@
-/*
- * 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.
- */
-
 #ifdef WIN32
 /* exclude transformation features on windows for now */
 #undef GPFXDIST
@@ -25,12 +6,12 @@
 #endif
 #endif
 
-#include <apr_getopt.h>
 #include <apr_env.h>
 #include <apr_file_info.h>
+#include <apr_general.h>
+#include <apr_getopt.h>
 #include <apr_hash.h>
 #include <apr_pools.h>
-#include <apr_signal.h>
 #include <apr_strings.h>
 #include <apr_time.h>
 #include <event.h>
@@ -38,78 +19,85 @@
 #include <signal.h>
 #include <stdio.h>
 #include <stdlib.h>
+#ifndef WIN32
 #include <strings.h>
+#endif
 #ifdef GPFXDIST
-#include <regex.h>
 #include <gpfxdist.h>
+#include <regex.h>
 #endif
 #include <fstream/fstream.h>
 
 #ifndef WIN32
-#include <unistd.h>
-#include <sys/socket.h>
-#include <netinet/in.h>
-#include <netdb.h>
 #include <arpa/inet.h>
+#include <netdb.h>
+#include <netinet/in.h>
+#include <pthread.h>
+#include <sys/ioctl.h>
+#include <sys/socket.h>
+#include <unistd.h>
 #define SOCKET int
 #ifndef closesocket
-#define closesocket(x)   close(x)
+#define closesocket(x) close(x)
 #endif
 #else
 #define WIN32_LEAN_AND_MEAN
+#include <io.h>
+#include <windows.h>
 #include <winsock2.h>
 #include <ws2tcpip.h>
-#include <windows.h>
-#include <io.h>
 #define SHUT_WR SD_SEND
-#define SHUT_RD SD_RECEIVE
 #define socklen_t int
 #ifndef ECONNRESET
-#define ECONNRESET   WSAECONNRESET
+#define ECONNRESET WSAECONNRESET
 #endif
 
 #endif
 
-#include <openssl/ssl.h>
-#include <openssl/rand.h>
+#include <pg_config.h>
+#include "gpfdist_helper.h"
+#ifdef USE_SSL
 #include <openssl/err.h>
+#include <openssl/rand.h>
+#include <openssl/ssl.h>
+#endif
 
 /*  A data block */
 typedef struct blockhdr_t blockhdr_t;
-struct blockhdr_t
-{
-	char 	hbyte[293];
-	int 	hbot, htop;
+struct blockhdr_t {
+  char hbyte[293];
+  int hbot, htop;
 };
 
 /*
  * Data that is sent from server to client
  */
 typedef struct block_t block_t;
-struct block_t
-{
-	blockhdr_t 	hdr;
-	int 		bot, top;
-	char*      	data;
+struct block_t {
+  blockhdr_t hdr;
+  int bot, top;
+  char* data;
 };
 
-#define FLINE		__FILE__":"APR_STRINGIFY(__LINE__)
+/*  Get session id for this request */
+#define GET_SID(r) ((r->sid))
 
-int gprint(const char* fmt, ...);
-void gfatal(const char* fline, const char* fmt, ...);
-int gwarning(const char* fline, const char* fmt, ...);
+static long REQUEST_SEQ = 0; /*  sequence number for request */
+static long SESSION_SEQ = 0; /*  sequence number for session */
 
-static void base16_decode(char* data);
+static bool base16_decode(char* data);
 
+#ifdef USE_SSL
 /* SSL additions */
-#define SSL_RENEGOTIATE_TIMEOUT_SEC	(600) /* 10 minutes */
+#define SSL_RENEGOTIATE_TIMEOUT_SEC (600) /* 10 minutes */
 const char* const CertificateFilename = "server.crt";
 const char* const PrivateKeyFilename = "server.key";
 const char* const TrustedCaFilename = "root.crt";
-static SSL_CTX *initialize_ctx(void);
-static void handle_ssl_error(SOCKET sock, BIO *sbio, SSL *ssl);
+static SSL_CTX* initialize_ctx(void);
+static void handle_ssl_error(SOCKET sock, BIO* sbio, SSL* ssl);
 static void flush_ssl_buffer(int fd, short event, void* arg);
 /* SSL end */
+#endif
 
 /**************
 
@@ -119,9 +107,9 @@ static void flush_ssl_buffer(int fd, short event, void* arg);
  X-GP-XID   - transaction ID
  X-GP-CID   - command ID
  X-GP-SN    - session ID
- X-GP-PROTO - protocol number
+ X-GP-PROTO - protocol number, report error if not provided:
 
- X-GP-PROTO = 0 or not provided:
+ X-GP-PROTO = 0
  return the content of the file without any kind of meta info
 
  X-GP-PROTO = 1
@@ -137,222 +125,201 @@ static void flush_ssl_buffer(int fd, short event, void* arg);
  **************/
 
 typedef struct gnet_request_t gnet_request_t;
-struct gnet_request_t
-{
-	int 	argc;
-	char** 	argv;
-	int 	hc;
-	char* 	hname[50];
-	char* 	hvalue[50];
+struct gnet_request_t {
+  int argc;
+  char** argv;
+  int hc;
+  char* hname[50];
+  char* hvalue[50];
 };
 
 static gnet_request_t* gnet_parse_request(const char* buf, int* len,
-										  apr_pool_t* pool);
+                                          apr_pool_t* pool);
 static char* gstring_trim(char* s);
 static void percent_encoding_to_char(char* p, char* pp, char* path);
 
-#ifndef GP_VERSION
-#define GP_VERSION unknown
-#endif
-
-#define GP_VERSIONX APR_STRINGIFY(GP_VERSION)
-
 /* CR-2723 */
-#define GPFDIST_MAX_LINE_LOWER_LIMIT (32*1024)
+#define GPFDIST_MAX_LINE_LOWER_LIMIT (32 * 1024)
 #ifdef GPFXDIST
-#define GPFDIST_MAX_LINE_UPPER_LIMIT (256*1024*1024)
-#define GPFDIST_MAX_LINE_MESSAGE     "Error: -m max row length must be between 32KB and 256MB"
+#define GPFDIST_MAX_LINE_UPPER_LIMIT (256 * 1024 * 1024)
+#define GPFDIST_MAX_LINE_MESSAGE \
+  "Error: -m max row length must be between 32KB and 256MB"
 #else
-#define GPFDIST_MAX_LINE_UPPER_LIMIT (1024*1024)
-#define GPFDIST_MAX_LINE_MESSAGE     "Error: -m max row length must be between 32KB and 1MB"
+#define GPFDIST_MAX_LINE_UPPER_LIMIT (1024 * 1024)
+#define GPFDIST_MAX_LINE_MESSAGE \
+  "Error: -m max row length must be between 32KB and 1MB"
 #endif
 
-
 /*	Struct of command line options */
-static struct
-{
-	int			p; /* port */
-	int			last_port;
-	int			v; /* verbose */
-	int			V; /* very verbose */
-	const char* d; /* directory */
-	const char* l; /* log filename */
-	const char*	f; /* forced filename */
-	int			g; /* gp_proto (0 or 1) (internal, not documented) */
-	int			t; /* timeout in seconds */
-	const char *b; /* IP address to bind (internal, not documented) */
-	int			m; /* max data line len */
-	int			S; /* use O_SYNC when opening files for write  */
-	int			z; /* listen queue size (hidden option currently for debuging) */
-	const char* c; /* config file */
-	struct transform* trlist; /* transforms from config file */
-	const char* ssl; /* path to certificates in case we use gpfdist with ssl */
-	int 		sslclean; /* Defines the time to wait [sec] untill cleanup the SSL resources (internal, not documented) */
-} opt = { 8080, 8080, 0, 0, ".", 0, 0, -1, 5, 0, 32768, 0, 256, 0, 0, 0, 5 };
-
-#if APR_IS_BIGENDIAN
-#define local_htonll(n)  (n)
-#define local_ntohll(n)  (n)
-#else
-#define local_htonll(n)  ((((apr_uint64_t) htonl(n)) << 32LL) | htonl((n) >> 32LL))
-#define local_ntohll(n)  ((((apr_uint64_t) ntohl(n)) << 32LL) | (apr_uint32_t) ntohl(((apr_uint64_t)n) >> 32LL))
+static struct {
+  int p; /* port */
+  int last_port;
+  int v; /* verbose */
+  int V; /* very verbose */
+  int s;
+  const char* d; /* directory */
+  const char* l; /* log filename */
+  const char* f; /* forced filename */
+  int g;         /* gp_proto (0 or 1) (internal, not documented) */
+  int t;         /* timeout in seconds */
+  const char* b; /* IP address to bind (internal, not documented) */
+  int m;         /* max data line len */
+  int S;         /* use O_SYNC when opening files for write  */
+  int z;         /* listen queue size (hidden option currently for debugging) */
+  const char* c; /* config file */
+  struct transform* trlist; /* transforms from config file */
+  const char* ssl; /* path to certificates in case we use gpfdist with ssl */
+  int w;           /* The time used for session timeout in seconds */
+} opt = {8080, 8080, 0, 0, 0, ".", 0, 0, -1, 5, 0, 32768, 0, 256, 0, 0, 0, 0};
+
+typedef union address {
+  struct sockaddr sa;
+  struct sockaddr_in sa_in;
+  struct sockaddr_in6 sa_in6;
+  struct sockaddr_storage sa_stor;
+} address_t;
+
+/*  Global control block */
+
+static struct {
+  apr_pool_t* pool;
+  int listen_sock_count;
+  SOCKET listen_socks[6];
+  struct event listen_events[6];
+  struct event signal_event;
+  struct {
+    int gen;
+    apr_hash_t* tab;
+  } session;
+  apr_int64_t read_bytes;
+  apr_int64_t total_bytes;
+  int total_sessions;
+#ifdef USE_SSL
+  BIO* bio_err;        /* for SSL */
+  SSL_CTX* server_ctx; /* for SSL */
 #endif
-
-/* if VERBOSE mode, print log information */
-#define TR(x)  while (opt.V){printf x;break;}
-
-/*
- * block_fill_header
- *
- * Prepare a block header for sending to the client. It includes various meta
- * data information such as filename, initial linenumber, etc. This will only
- * get used in PROTO-1. We store this header in block_t->hdr (a blockhdr_t)
- * and PROTO-0 never uses it.
- */
-static void block_fill_header(block_t* b,
-							  const struct fstream_filename_and_offset* fos)
-{
-	blockhdr_t*		h = &b->hdr;
-	apr_int32_t 	len;
-	apr_int64_t 	len8;
-	char*			p = h->hbyte;
-	int 			fname_len = strlen(fos->fname);
-
-	h->hbot = 0;
-
-	/* FILENAME: 'F' + len + fname */
-	*p++ = 'F';
-	len = htonl(fname_len);
-	memcpy(p, &len, 4);
-	p += 4;
-	memcpy(p, fos->fname, fname_len);
-	p += fname_len;
-	TR(("F %u %s\n", (unsigned int)ntohl(len), fos->fname));
-
-	/* OFFSET: 'O' + len + foff */
-	*p++ = 'O';
-	len = htonl(8);
-	len8 = local_htonll(fos->foff);
-	memcpy(p, &len, 4);
-	p += 4;
-	memcpy(p, &len8, 8);
-	p += 8;
-	TR(("O %llu\n",(unsigned long long) local_ntohll(len8)));
-
-	/* LINENUMBER: 'L' + len + linenumber */
-	*p++ = 'L';
-	len8 = local_htonll(fos->line_number);
-	memcpy(p, &len, 4);
-	p += 4;
-	memcpy(p, &len8, 8);
-	p += 8;
-	TR(("L %llu\n", (unsigned long long)local_ntohll(len8)));
-
-	/* DATA: 'D' + len */
-	*p++ = 'D';
-	len = htonl(b->top-b->bot);
-	memcpy(p, &len, 4);
-	p += 4;
-	TR(("D %u\n", (unsigned int)ntohl(len)));
-	h->htop = p - h->hbyte;
-	if (h->htop > sizeof(h->hbyte))
-		gfatal(FLINE, "assert failed, h->htop = %d, max = %d\n", h->htop,
-				sizeof(h->hbyte));
-	TR(("header size: %d\n",h->htop-h->hbot));
-}
-
-/*  Global constrol block */
-static struct
-{
-	apr_pool_t* 	pool;
-	int				listen_sock_count;
-	SOCKET 			listen_socks[6];
-	struct event 	listen_events[6];
-	struct
-	{
-		int 		gen;
-		apr_hash_t* tab;
-	} session;
-	apr_int64_t 	read_bytes;
-	apr_int64_t 	total_bytes;
-	int 			total_sessions;
-	BIO 			*bio_err;	/* for SSL */
-	SSL_CTX 		*server_ctx;/* for SSL */
+  int wdtimer; /* Kill gpfdist after k seconds of inactivity. 0 to disable. */
 } gcb;
 
 /*  A session */
 typedef struct session_t session_t;
-struct session_t
-{
-	apr_pool_t* 	pool;
-	const char* 	key;
-	const char* 	tid;
-	const char* 	path;			/* path requested */
-	fstream_t* 		fstream;
-	int 			is_error;		/* error flag */
-	int 			nrequest;		/* # requests attached to this session */
-	int				is_get;     	/* true for GET, false for POST */
-	int*			active_segids;	/* array indexed by segid. used for write operations
-									   to indicate which segdbs are writing and when each
-									   is done (sent a final request) */
-	int				maxsegs; 		/* same as request->totalsegs. length of active_segs arr */
-	apr_time_t		mtime; 			/* time when nrequest was modified */
+struct session_t {
+  long id;
+  apr_pool_t* pool;
+  const char* key;
+  const char* tid;
+  const char* path; /* path requested */
+  fstream_t* fstream;
+  int is_error;       /* error flag */
+  int nrequest;       /* # requests attached to this session */
+  int is_get;         /* true for GET, false for POST */
+  int* active_segids; /* array indexed by segid. used for write operations
+                                                 to indicate which segdbs are
+                         writing and when each is done (sent a final request) */
+  apr_int64_t*
+      seq_segs; /* array indexed by segid. used for write operations to record
+                   the sequence number of data which has been written to disk*/
+  int maxsegs;  /* same as request->totalsegs. length of active_segs arr */
+  apr_time_t mtime;  /* time when nrequest was modified */
+  struct timeval tm; /* timeout for struct event */
+  struct event ev;   /* event we are watching for this session*/
+  apr_hash_t* requests;
 };
 
 /*  An http request */
 typedef struct request_t request_t;
-struct request_t
-{
-	SOCKET 			sock; 		/* the socket */
-	apr_pool_t* 	pool; 		/* memory pool container */
-	struct timeval 	tm; 		/* timeout for struct event */
-	struct event 	ev; 		/* event we are watching for this request */
-	const char* 	peer; 		/* peer IP:port string */
-	const char* 	path; 		/* path to file */
-	const char* 	tid; 		/* transaction id */
-	const char* 	csvopt; 	/* options for csv file */
+struct request_t {
+  long id;         /* request id (auto increment) */
+  long sid;        /* session id (auto increment) */
+  long bytes;      /* bytes sent to TCP or receive from TCP */
+  apr_time_t last; /* last timestamp for send/receive data */
+  apr_int64_t seq; /* sequence number */
+
+  unsigned short port;
+  SOCKET sock;        /* the socket */
+  apr_pool_t* pool;   /* memory pool container */
+  struct timeval tm;  /* timeout for struct event */
+  struct event ev;    /* event we are watching for this request */
+  const char* peer;   /* peer IP:port string */
+  const char* path;   /* path to file */
+  const char* tid;    /* transaction id */
+  const char* csvopt; /* options for csv file */
 
 #ifdef GPFXDIST
-	struct
-	{
-		const char* name;		/* requested transformation */
-		char*		command;	/* command associated with transform */
-		int			paths;		/* 1 if filename passed to transform should contain paths to data files */
-		const char* errfilename; /* name of temporary file holding stderr to send to server */
-		apr_file_t* errfile;	/* temporary file holding stderr to send to server */
-	} trans;
-#endif
-
-	session_t* 		session; 	/* the session this request is attached to */
-	int 			gp_proto; 	/* the protocol to use, sent from client */
-	int				is_get;     /* true for GET, false for POST */
-	int				is_final;	/* the final POST request. a signal from client to end session */
-	int				segid;		/* the segment id of the segdb with the request */
-	int				totalsegs;	/* the total number of segdbs */
-
-	struct
-	{
-		char* 	hbuf; 		/* buffer for raw incoming HTTP request */
-		int 	hbuftop; 	/* # bytes used in hbuf */
-		int 	hbufmax; 	/* size of hbuf[] */
-		gnet_request_t* req; /* a parsed HTTP request, NULL if still incomplete. */
-		int		davailable;	/* number of data bytes available to consume */
-		char*	dbuf;		/* buffer for raw data from a POST request */
-		int 	dbuftop; 	/* # bytes used in dbuf */
-		int 	dbufmax; 	/* size of dbuf[] */
-	} in;
-
-	block_t	outblock;	/* next block to send out */
-	char*           line_delim_str;
-	int             line_delim_length;
-	
-	/* SSL related */
-	BIO			*io;		/* for the i.o. */
-	BIO			*sbio;		/* for the accept */
-	BIO			*ssl_bio;
-	SSL			*ssl;
+  struct {
+    const char* name; /* requested transformation */
+    char* command;    /* command associated with transform */
+    int paths; /* 1 if filename passed to transform should contain paths to data
+                  files */
+    const char* errfilename; /* name of temporary file holding stderr to send to
+                                server */
+    apr_file_t* errfile; /* temporary file holding stderr to send to server */
+  } trans;
+#endif
+
+  session_t* session; /* the session this request is attached to */
+  int gp_proto;       /* the protocol to use, sent from client */
+  int is_get;         /* true for GET, false for POST */
+  int is_final;  /* the final POST request. a signal from client to end session
+                  */
+  int segid;     /* the segment id of the segdb with the request */
+  int totalsegs; /* the total number of segdbs */
+
+  struct {
+    char* hbuf;          /* buffer for raw incoming HTTP request */
+    int hbuftop;         /* # bytes used in hbuf */
+    int hbufmax;         /* size of hbuf[] */
+    gnet_request_t* req; /* a parsed HTTP request, NULL if still incomplete. */
+    int davailable;      /* number of data bytes available to consume */
+    char* dbuf;          /* buffer for raw data from a POST request */
+    int dbuftop;         /* # bytes used in dbuf */
+    int dbufmax;         /* size of dbuf[] */
+  } in;
+
+  block_t outblock; /* next block to send out */
+  char* line_delim_str;
+  int line_delim_length;
+
+#ifdef USE_SSL
+  /* SSL related */
+  BIO* io;   /* for the i.o. */
+  BIO* sbio; /* for the accept */
+  BIO* ssl_bio;
+  SSL* ssl;
+#endif
 };
 
+#if APR_IS_BIGENDIAN
+#define local_htonll(n) (n)
+#define local_ntohll(n) (n)
+#else
+#define local_htonll(n) \
+  ((((apr_uint64_t)htonl(n)) << 32LL) | htonl((n) >> 32LL))
+#define local_ntohll(n)                 \
+  ((((apr_uint64_t)ntohl(n)) << 32LL) | \
+   (apr_uint32_t)ntohl(((apr_uint64_t)n) >> 32LL))
+#endif
+
+#define NO_SEQ 0
+#define OPEN_SEQ 1
+
+static int ggetpid();
+static void log_gpfdist_status();
+static void log_request_header(const request_t* r);
+
+static void gprint(const request_t* r, const char* fmt, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 2, 3)));
+static void gprintln(const request_t* r, const char* fmt, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 2, 3)));
+static void gprintlnif(const request_t* r, const char* fmt, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 2, 3)));
+static void gfatal(const request_t* r, const char* fmt, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 2, 3)));
+static void gwarning(const request_t* r, const char* fmt, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 2, 3)));
+static void gdebug(const request_t* r, const char* fmt, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 2, 3)));
+
 /* send gp-proto==1 ctl info */
 static void gp1_send_eof(request_t* r);
 static void gp1_send_errmsg(request_t* r, const char* msg);
@@ -360,442 +327,676 @@ static void gp1_send_errmsg(request_t* r, const char* msg);
 static void gp1_send_errfile(request_t* r, apr_file_t* errfile);
 #endif
 
+static char* datetime_now(void);
+static char* datetime(apr_time_t t);
 static int setup_read(request_t* r);
 static int setup_write(request_t* r);
+static void setup_do_close(request_t* r);
 static int session_attach(request_t* r);
 static void session_detach(request_t* r);
 static void session_end(session_t* s, int error);
 static void session_free(session_t* s);
 static void session_active_segs_dump(session_t* session);
 static int session_active_segs_isempty(session_t* session);
-static int request_validate(request_t *r);
-static int request_set_path(request_t *r, const char* d, char* p, char* pp, char* path);
-static int request_parse_gp_headers(request_t *r, int opt_g);
+static int request_validate(request_t* r);
+static int request_set_path(request_t* r, const char* d, char* p, char* pp,
+                            char* path);
+static int request_parse_gp_headers(request_t* r, int opt_g);
+static void free_session_cb(int fd, short event, void* arg);
 #ifdef GPFXDIST
-static int request_set_transform(request_t *r);
-#endif
-static void handle_post_request(request_t *r, int header_end);
-static void handle_get_request(request_t *r);
-
-static int gpfdist_socket_send(const request_t *r, const void *buf, const size_t buflen);
-static int gpfdist_SSL_send(const request_t *r, const void *buf, const size_t buflen);
-static int (*gpfdist_send)(const request_t *r, const void *buf, const size_t buflen); /* function pointer */
-static int gpfdist_socket_receive(const request_t *r, void *buf, const size_t buflen);
-static int gpfdist_SSL_receive(const request_t *r, void *buf, const size_t buflen);
-static int (*gpfdist_receive)(const request_t *r, void *buf, const size_t buflen); /* function pointer */
-static void free_SSL_resources(const request_t *r);
+static int request_set_transform(request_t* r);
+#endif
+static void handle_post_request(request_t* r, int header_end);
+static void handle_get_request(request_t* r);
+
+static int gpfdist_socket_send(const request_t* r, const void* buf,
+                               const size_t buflen);
+static int (*gpfdist_send)(const request_t* r, const void* buf,
+                           const size_t buflen); /* function pointer */
+static int gpfdist_socket_receive(const request_t* r, void* buf,
+                                  const size_t buflen);
+static int (*gpfdist_receive)(const request_t* r, void* buf,
+                              const size_t buflen); /* function pointer */
+static void request_cleanup(request_t* r);
+#ifdef USE_SSL
+static int gpfdist_SSL_send(const request_t* r, const void* buf,
+                            const size_t buflen);
+static int gpfdist_SSL_receive(const request_t* r, void* buf,
+                               const size_t buflen);
+static void free_SSL_resources(const request_t* r);
 static void setup_flush_ssl_buffer(request_t* r);
-static void request_cleanup(request_t *r);
-static void request_cleanup_and_free_SSL_resources(int fd, short event, void* arg);
+static void request_cleanup_and_free_SSL_resources(request_t* r);
+#endif
+static int local_send(request_t* r, const char* buf, int buflen);
+
+static int get_unsent_bytes(request_t* r);
+
+static void* palloc_safe(request_t* r, apr_pool_t* pool, apr_size_t size,
+                         const char* fmt, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 4, 5)));
+static void* pcalloc_safe(request_t* r, apr_pool_t* pool, apr_size_t size,
+                          const char* fmt, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 4, 5)));
+
+static void process_term_signal(int sig, short event, void* arg);
+int gpfdist_init(int argc, const char* const argv[]);
+int gpfdist_run(void);
+
+static void delay_watchdog_timer(void);
+#ifndef WIN32
+static apr_time_t shutdown_time;
+static void* watchdog_thread(void*);
+#endif
+
+/*
+ * block_fill_header
+ *
+ * Prepare a block header for sending to the client. It includes various meta
+ * data information such as filename, initial linenumber, etc. This will only
+ * get used in PROTO-1. We store this header in block_t->hdr (a blockhdr_t)
+ * and PROTO-0 never uses it.
+ */
+static void block_fill_header(const request_t* r, block_t* b,
+                              const struct fstream_filename_and_offset* fos) {
+  blockhdr_t* h = &b->hdr;
+  apr_int32_t len;
+  apr_int64_t len8;
+  char* p = h->hbyte;
+  int fname_len = strlen(fos->fname);
+
+  h->hbot = 0;
+
+  /* FILENAME: 'F' + len + fname */
+  *p++ = 'F';
+  len = htonl(fname_len);
+  memcpy(p, &len, 4);
+  p += 4;
+  memcpy(p, fos->fname, fname_len);
+  p += fname_len;
+  gdebug(r, "F %u %s", (unsigned int)ntohl(len), fos->fname);
+
+  /* OFFSET: 'O' + len + foff */
+  *p++ = 'O';
+  len = htonl(8);
+  len8 = local_htonll(fos->foff);
+  memcpy(p, &len, 4);
+  p += 4;
+  memcpy(p, &len8, 8);
+  p += 8;
+#ifndef WIN32
+  gdebug(r, "O %llu", (unsigned long long)local_ntohll(len8));
+#else
+  gdebug(r, "O %lu", (unsigned long)local_ntohll(len8));
+#endif
+
+  /* LINENUMBER: 'L' + len + linenumber */
+  *p++ = 'L';
+  len8 = local_htonll(fos->line_number);
+  memcpy(p, &len, 4);
+  p += 4;
+  memcpy(p, &len8, 8);
+  p += 8;
+#ifndef WIN32
+  gdebug(r, "L %llu", (unsigned long long)local_ntohll(len8));
+#else
+  gdebug(r, "L %lu", (unsigned long)local_ntohll(len8));
+#endif
+
+  /* DATA: 'D' + len */
+  *p++ = 'D';
+  len = htonl(b->top - b->bot);
+  memcpy(p, &len, 4);
+  p += 4;
+  gdebug(r, "D %u", (unsigned int)ntohl(len));
+  h->htop = p - h->hbyte;
+  if (h->htop > sizeof(h->hbyte))
+    gfatal(NULL, "assert failed, h->htop = %d, max = %d", h->htop,
+           (int)sizeof(h->hbyte));
+  gdebug(r, "header size: %d", h->htop - h->hbot);
+}
+
+static unsigned short get_client_port(address_t* clientInformation) {
+  // check the family version of client IP address, so you
+  // can know where to cast, either to sockaddr_in or sockaddr_in6
+  // and then grab the port after casting
+  struct sockaddr* sa = (struct sockaddr*)clientInformation;
+  if (sa->sa_family == AF_INET) {
+    struct sockaddr_in* ipv4 = (struct sockaddr_in*)clientInformation;
+    return ipv4->sin_port;
+  } else {
+    struct sockaddr_in6* ipv6 = (struct sockaddr_in6*)clientInformation;
+    return ipv6->sin6_port;
+  }
+}
 
 /* Print usage */
-static void usage_error(const char* msg, int print_usage)
-{
-	if (print_usage)
-	{
-		char* GPHOME = 0;
-		FILE* fp = 0;
-
-		if (gcb.pool && apr_env_get(&GPHOME, "GPHOME", gcb.pool))
-			GPHOME = 0;
-
-		if (GPHOME)
-		{
-			char* path = apr_psprintf(gcb.pool, "%s/docs/cli_help/gpfdist_help",
-					GPHOME);
-			if (path)
-				fp = fopen(path, "r");
-		}
-
-		if (fp)
-		{
-			int i;
-			while ((i = getc(fp)) != EOF)
-				putchar(i);
-			fclose(fp);
-		}
-		else
-		{
-			fprintf(stderr,
-					"gpfdist -- file distribution web server\n\n"
-						"usage: gpfdist [--ssl <certificates_directory>] [-d <directory>] [-p <http(s)_port>] [-l <log_file>] [-t <timeout>] [-v | -V] [-m <maxlen>]"
+static void usage_error(const char* msg, int print_usage) {
+  if (print_usage) {
+    char* GPHOME = 0;
+    FILE* fp = 0;
+
+    if (gcb.pool && apr_env_get(&GPHOME, "GPHOME", gcb.pool)) GPHOME = 0;
+
+    if (GPHOME) {
+      char* path =
+          apr_psprintf(gcb.pool, "%s/docs/cli_help/gpfdist_help", GPHOME);
+      if (path) fp = fopen(path, "r");
+    }
+
+    if (fp) {
+      int i;
+      while ((i = getc(fp)) != EOF) putchar(i);
+      fclose(fp);
+    } else {
+      fprintf(stderr,
+              "gpfdist -- file distribution web server\n\n"
+              "usage: gpfdist [--ssl <certificates_directory>] [-d "
+              "<directory>] [-p <http(s)_port>] [-l <log_file>] [-t <timeout>] "
+              "[-v | -V | -s] [-m <maxlen>] [-w <timeout>]"
 #ifdef GPFXDIST
-					    "[-c file]"
-#endif
-					    "\n\n"
-						"       gpfdist [-? | --help] | --version\n\n"
-						"        -?, --help : print this screen\n"
-						"        -v         : verbose mode\n"
-						"        -V         : more verbose\n"
-						"        -p port    : port to serve HTTP(S), default is 8080\n"
-						"        -d dir     : serve files under the specified directory,  default is '.'\n"
-						"        -l logfn   : log filename\n"
-						"        -t tm      : timeout in seconds \n"
-						"        -m maxlen  : max data row length expected, in bytes. default is 32768\n"
-						"        --ssl dir  : start HTTPS server. Use the certificates from the specified directory\n"
+              "[-c file]"
+#endif
+              "\n\n"
+              "       gpfdist [-? | --help] | --version\n\n"
+              "        -?, --help : print this screen\n"
+              "        -v         : verbose mode\n"
+              "        -V         : more verbose\n"
+              "        -s         : simplified minimum log\n"
+#ifdef USE_SSL
+              "        -p port    : port to serve HTTP(S), default is 8080\n"
+#else
+              "        -p port    : port to serve HTTP, default is 8080\n"
+#endif
+              "        -d dir     : serve files under the specified directory, "
+              " default is '.'\n"
+              "        -l logfn   : log filename\n"
+              "        -t tm      : timeout in seconds \n"
+              "        -m maxlen  : max data row length expected, in bytes. "
+              "default is 32768\n"
+#ifdef USE_SSL
+              "        --ssl dir  : start HTTPS server. Use the certificates "
+              "from the specified directory\n"
+#endif
 #ifdef GPFXDIST
-					    "        -c file    : configuration file for transformations\n"
+              "        -c file    : configuration file for transformations\n"
 #endif
-						"        --version  : print version information\n\n");
-		}
-	}
+              "        --version  : print version information\n"
+              "        -w timeout : timeout in seconds before close target "
+              "file\n\n");
+    }
+  }
 
-	if (msg)
-		fprintf(stderr, "%s\n", msg);
+  if (msg) fprintf(stderr, "%s\n", msg);
 
-	exit(msg ? 1 : 0);
+  exit(msg ? 1 : 0);
 }
 
-static void print_version(void)
-{
-	printf("gpfdist version \"%s\"\n", GP_VERSIONX);
-	exit(0);
+static void print_version(void) {
+  printf("gpfdist version \"%s\"\n", GP_VERSION);
+  exit(0);
 }
 
-static void print_q_x_h_are_gone(void)
-{
-	fprintf(stderr, "The -q, -h and -x options are gone.  Please specify these as in this example:\n");
-	fprintf(stderr, "create external table a (a int) location ('gpfdist://...') format 'csv' (escape as '\"' quote as '\"' header);\n");
-	exit(1);
+static void print_q_x_h_are_gone(void) {
+  fprintf(stderr,
+          "The -q, -h and -x options are gone.  Please specify these as in "
+          "this example:\n");
+  fprintf(stderr,
+          "create external table a (a int) location ('gpfdist://...') format "
+          "'csv' (escape as '\"' quote as '\"' header);\n");
+  exit(1);
 }
 
 /* Parse command line */
 static void parse_command_line(int argc, const char* const argv[],
-		apr_pool_t* pool)
-{
-	apr_getopt_t* 	os;
-	const char* 	arg;
-	char 			apr_errbuf[256];
-	int				status;
-	int 			ch;
-	int 			e;
-
-	char* current_directory = NULL;
-
-	static const apr_getopt_option_t option[] =
-	{
-	/* long-option, short-option, has-arg flag, description */
-	{ "help", '?', 0, "print help screen" },
-	{ NULL, 'V', 0, "very verbose" },
-	{ NULL, 'v', 0, "verbose mode" },
-	{ NULL, 'p', 1, "which port to serve HTTP(S)" },
-	{ NULL, 'P', 1, "last port of range of ports to serve HTTP(S)" },
-	{ NULL, 'd', 1, "serve files under this directory" },
-	{ NULL, 'f', 1, "internal - force file to be given file name" },
-	{ NULL, 'b', 1, "internal - bind to ip4 address" },
-	{ NULL, 'q', 0, "gone" },
-	{ NULL, 'x', 0, "gone" },
-	{ NULL, 'h', 0, "gone" },
-	{ NULL, 'l', 1, "log filename" },
-	{ NULL, 't', 5, "timeout in seconds" },
-	{ NULL, 'g', 1, "internal - gp_proto number" },
-	{ NULL, 'm', 1, "max data row length expected" },
-	{ NULL, 'S', 0, "use O_SYNC when opening files for write" },
-	{ NULL, 'z', 1, "internal - queue size for listen call" },
-	{ "ssl", 257, 1, "ssl - certificates files under this directory" },
-	{ "sslclean", 258, 1, "Defines the time to wait [sec] untill cleanup the SSL resources" },
+                               apr_pool_t* pool) {
+  apr_getopt_t* os;
+  const char* arg;
+  char apr_errbuf[256];
+  int status;
+  int ch;
+  int e;
+
+  char* current_directory = NULL;
+
+  static const apr_getopt_option_t option[] = {
+      /* long-option, short-option, has-arg flag, description */
+      {"help", '?', 0, "print help screen"},
+      {NULL, 'V', 0, "very verbose"},
+      {NULL, 'v', 0, "verbose mode"},
+      {NULL, 's', 0, "simplified log without request header"},
+      {NULL, 'p', 1, "which port to serve HTTP(S)"},
+      {NULL, 'P', 1, "last port of range of ports to serve HTTP(S)"},
+      {NULL, 'd', 1, "serve files under this directory"},
+      {NULL, 'f', 1, "internal - force file to be given file name"},
+      {NULL, 'b', 1, "internal - bind to ip4 address"},
+      {NULL, 'q', 0, "gone"},
+      {NULL, 'x', 0, "gone"},
+      {NULL, 'h', 0, "gone"},
+      {NULL, 'l', 1, "log filename"},
+      {NULL, 't', 5, "timeout in seconds"},
+      {NULL, 'g', 1, "internal - gp_proto number"},
+      {NULL, 'm', 1, "max data row length expected"},
+      {NULL, 'S', 0, "use O_SYNC when opening files for write"},
+      {NULL, 'z', 1, "internal - queue size for listen call"},
+      {"ssl", 257, 1, "ssl - certificates files under this directory"},
 #ifdef GPFXDIST
-	{ NULL, 'c', 1, "transform configuration file" },
-#endif
-	{ "version", 256, 0, "print version number" },
-	{ 0 } };
-
-	status = apr_getopt_init(&os, pool, argc, argv);
-
-	if (status != APR_SUCCESS)
-		gfatal(FLINE, "apt_getopt_init failed: %s",
-			   apr_strerror(status, apr_errbuf, sizeof(apr_errbuf)));
-
-	while (APR_SUCCESS == (e = apr_getopt_long(os, option, &ch, &arg)))
-	{
-		switch (ch)
-		{
-		case '?':
-			usage_error(0, 1);
-			break;
-		case 'v':
-			opt.v = 1;
-			break;
-		case 'V':
-			opt.v = opt.V = 1;
-			break;
-		case 'h':
-			print_q_x_h_are_gone();
-			break;
-		case 'd':
-			opt.d = arg;
-			break;
-		case 'f':
-			opt.f = arg;
-			break;
-		case 'q':
-			print_q_x_h_are_gone();
-			break;
-		case 'x':
-			print_q_x_h_are_gone();
-			break;
-		case 'p':
-			opt.last_port = opt.p = atoi(arg);
-			break;
-		case 'P':
-			opt.last_port = atoi(arg);
-			break;
-		case 'l':
-			opt.l = arg;
-			break;
-		case 't':
-			opt.t = atoi(arg);
-			break;
-		case 'g':
-			opt.g = atoi(arg);
-			break;
-		case 'b':
-			opt.b = arg;
-			break;
-		case 'm':
-			opt.m = atoi(arg);
-			break;
-		case 'S':
-			opt.S = 1;
-			break;
-		case 'z':
-			opt.z = atoi(arg);
-			break;
-		case 'c':
-			opt.c = arg;
-			break;
-		case 257:
-			opt.ssl = arg;
-			break;
-		case 258:
-			opt.sslclean = atoi(arg);
-			break;
-		case 256:
-			print_version();
-			break;
-		}
-	}
-
-	if (e != APR_EOF)
-		usage_error("Error: illegal arguments", 1);
-
-	if (!(0 < opt.p && opt.p < (1 << 16)))
-		usage_error("Error: please specify a valid port number for -p switch", 0);
-
-	if (-1 != opt.g)
-	{
-		if (!(0 == opt.g || 1 == opt.g))
-			usage_error("Error: please specify 0 or 1 for -g switch (note: this is internal)", 0);
-	}
-
-	if (! ((2 <= opt.t && opt.t <= 30) || opt.t == 0))
-		usage_error("Error: -t timeout must be between 2 and 30, or 0 for no timeout", 0);
-
-	/* validate max row length */
-    if (! ((GPFDIST_MAX_LINE_LOWER_LIMIT <= opt.m) && (opt.m <= GPFDIST_MAX_LINE_UPPER_LIMIT)))
-    	usage_error(GPFDIST_MAX_LINE_MESSAGE, 0);
-
-    if (! (16 <= opt.z && opt.t <= 512) )
-		usage_error("Error: -z listen queue size must be between 16 and 512 (default is 256)", 0);
-
-    /* get current directory, for ssl directory validation */
-    if (0 != apr_filepath_get(&current_directory, APR_FILEPATH_NATIVE, pool))
-		usage_error(apr_psprintf(pool, "Error: cannot access directory '.'\n"
-									   "Please run gpfdist from a different location"), 0);
-
-	/* validate opt.d */
-	{
-		char* p = gstring_trim(apr_pstrdup(pool, opt.d));
-
-		/* collapse // */
-		while (p[0] == '/' && p[1] == '/')
-			p++;
-
-		/* disallow / */
-		if (0 == strcmp(p, "/"))
-			usage_error("Security Error:  You cannot specify the root"
-						" directory (/) as the source files directory.", 0);
-
-		/* strip ending / */
-		while (p[0] && p[strlen(p) - 1] == '/')
-			p[strlen(p) - 1] = 0;
-		opt.d = p;
-
-		if (0 == strlen(opt.d))
-			opt.d = ".";
-
-		/* check that the dir exists */
-		if (0 != apr_filepath_set(opt.d, pool))
-			usage_error(apr_psprintf(pool, "Error: cannot access directory '%s'\n"
-				"Please specify a valid directory for -d switch", opt.d), 0);
-
-		if (0 != apr_filepath_get(&p, APR_FILEPATH_NATIVE, pool))
-			usage_error(apr_psprintf(pool, "Error: cannot access directory '%s'\n"
-				"Please specify a valid directory for -d switch", opt.d), 0);
-		opt.d = p;
-	}
-
-	/* validate opt.l */
-	if (opt.l)
-	{
-		FILE *f;
-
-		char* p = gstring_trim(apr_pstrdup(pool, opt.l));
-
-		/* collapse // */
-		while (p[0] == '/' && p[1] == '/')
-			p++;
-
-		/* disallow / */
-		if (0 == strcmp(p, "/"))
-			usage_error("Security Error: You cannot specify the root"
-						" directory (/) as the log file directory.", 0);
-
-		/* strip ending / */
-		while (p[0] && p[strlen(p) - 1] == '/')
-			p[strlen(p) - 1] = 0;
-		opt.l = p;
-
-		if (0 == strlen(opt.l))
-			opt.l = ".";
-
-		/* check that the file exists */
-		f = fopen(opt.l, "a");
-		if (!f)
-		{
-			fprintf(stderr, "unable to create log file %s: %s\n",
-					opt.l, strerror(errno));
-			exit(1);
-		}
-		fclose(f);
-
-	}
-
-	/* validate opt.ssl */
-	if (opt.ssl)
-	{
-		char* p = gstring_trim(apr_pstrdup(pool, opt.ssl));
-
-		/* collapse // */
-		while (p[0] == '/' && p[1] == '/')
-			p++;
-
-		/* disallow / */
-		if (0 == strcmp(p, "/"))
-			usage_error("Security Error: You cannot specify the root" 
-						" directory (/) as the certificates directory", 0);
-
-		/* strip ending / */
-		while (p[0] && p[strlen(p) - 1] == '/')
-			p[strlen(p) - 1] = 0;
-		opt.ssl = p;
-
-		/* change current directory to original one (after -d changed it) */
-		if (0 != apr_filepath_set(current_directory, pool))
-				usage_error(apr_psprintf(pool, "Error: cannot access directory '%s'\n"
-											   "Please run gpfdist from a different location", current_directory), 0);
-		/* check that the dir exists */
-		if ( (0 != apr_filepath_set(opt.ssl, pool)) || (0 != apr_filepath_get(&p, APR_FILEPATH_NATIVE, pool)) )
-			usage_error(apr_psprintf(pool, "Error: cannot access directory '%s'\n"
-				"Please specify a valid directory for --ssl switch", opt.ssl), 0);
-		opt.ssl = p;
-	}
-
-	/* Validate opt.sslclean*/
-	if ( (opt.sslclean < 0) || (opt.sslclean > 300) )
-		usage_error("Error: -sslclean timeout must be between 0 and 300 [sec] (default is 5[sec])", 0);
+      {NULL, 'c', 1, "transform configuration file"},
+#endif
+      {"version", 256, 0, "print version number"},
+      {NULL, 'w', 1, "wait for session timeout in seconds"},
+      {0}};
+
+  status = apr_getopt_init(&os, pool, argc, argv);
+
+  if (status != APR_SUCCESS)
+    gfatal(NULL, "apt_getopt_init failed: %s",
+           apr_strerror(status, apr_errbuf, sizeof(apr_errbuf)));
+
+  while (APR_SUCCESS == (e = apr_getopt_long(os, option, &ch, &arg))) {
+    switch (ch) {
+      case '?':
+        usage_error(0, 1);
+        break;
+      case 'v':
+        opt.v = 1;
+        break;
+      case 'V':
+        opt.v = opt.V = 1;
+        break;
+      case 's':
+        opt.s = 1;
+        break;
+      case 'h':
+        print_q_x_h_are_gone();
+        break;
+      case 'd':
+        opt.d = arg;
+        break;
+      case 'f':
+        opt.f = arg;
+        break;
+      case 'q':
+        print_q_x_h_are_gone();
+        break;
+      case 'x':
+        print_q_x_h_are_gone();
+        break;
+      case 'p':
+        opt.last_port = opt.p = atoi(arg);
+        break;
+      case 'P':
+        opt.last_port = atoi(arg);
+        break;
+      case 'l':
+        opt.l = arg;
+        break;
+      case 't':
+        opt.t = atoi(arg);
+        break;
+      case 'g':
+        opt.g = atoi(arg);
+        break;
+      case 'b':
+        opt.b = arg;
+        break;
+      case 'm':
+        opt.m = atoi(arg);
+        break;
+      case 'S':
+        opt.S = 1;
+        break;
+      case 'z':
+        opt.z = atoi(arg);
+        break;
+      case 'c':
+        opt.c = arg;
+        break;
+#ifdef USE_SSL
+      case 257:
+        opt.ssl = arg;
+        break;
+#else
+      case 257:
+        usage_error("SSL is not supported by this build", 0);
+        break;
+#endif
+      case 256:
+        print_version();
+        break;
+      case 'w':
+        opt.w = atoi(arg);
+        break;
+    }
+  }
+
+  if (e != APR_EOF) usage_error("Error: illegal arguments", 1);
+
+  if (!(0 < opt.p && opt.p < (1 << 16)))
+    usage_error("Error: please specify a valid port number for -p switch", 0);
+
+  if (-1 != opt.g) {
+    if (!(0 == opt.g || 1 == opt.g))
+      usage_error(
+          "Error: please specify 0 or 1 for -g switch (note: this is internal)",
+          0);
+  }
+
+  if (!is_valid_timeout(opt.t))
+    usage_error(
+        "Error: -t timeout must be between 2 and 7200, or 0 for no timeout", 0);
+
+  if (!is_valid_session_timeout(opt.w))
+    usage_error(
+        "Error: -w timeout must be between 1 and 7200, or 0 for no timeout", 0);
+
+  /* validate max row length */
+  if (!((GPFDIST_MAX_LINE_LOWER_LIMIT <= opt.m) &&
+        (opt.m <= GPFDIST_MAX_LINE_UPPER_LIMIT)))
+    usage_error(GPFDIST_MAX_LINE_MESSAGE, 0);
+
+  if (!is_valid_listen_queue_size(opt.z))
+    usage_error(
+        "Error: -z listen queue size must be between 16 and 512 (default is "
+        "256)",
+        0);
+
+  /* get current directory, for ssl directory validation */
+  if (0 != apr_filepath_get(&current_directory, APR_FILEPATH_NATIVE, pool))
+    usage_error(apr_psprintf(pool,
+                             "Error: cannot access directory '.'\n"
+                             "Please run gpfdist from a different location"),
+                0);
+
+  /* validate opt.d */
+  {
+    char* p = gstring_trim(apr_pstrdup(pool, opt.d));
+
+    /* collapse // */
+    while (p[0] == '/' && p[1] == '/') p++;
+
+    /* disallow / */
+    if (0 == strcmp(p, "/"))
+      usage_error(
+          "Security Error:  You cannot specify the root"
+          " directory (/) as the source files directory.",
+          0);
+
+    /* strip ending / */
+    while (p[0] && p[strlen(p) - 1] == '/') p[strlen(p) - 1] = 0;
+    opt.d = p;
+
+    if (0 == strlen(opt.d)) opt.d = ".";
+
+    /* check that the dir exists */
+    if (0 != apr_filepath_set(opt.d, pool))
+      usage_error(apr_psprintf(pool,
+                               "Error: cannot access directory '%s'\n"
+                               "Please specify a valid directory for -d switch",
+                               opt.d),
+                  0);
+
+    if (0 != apr_filepath_get(&p, APR_FILEPATH_NATIVE, pool))
+      usage_error(apr_psprintf(pool,
+                               "Error: cannot access directory '%s'\n"
+                               "Please specify a valid directory for -d switch",
+                               opt.d),
+                  0);
+    opt.d = p;
+  }
+
+  /* validate opt.l */
+  if (opt.l) {
+    FILE* f;
+
+    char* p = gstring_trim(apr_pstrdup(pool, opt.l));
+
+    /* collapse // */
+    while (p[0] == '/' && p[1] == '/') p++;
+
+    /* disallow / */
+    if (0 == strcmp(p, "/"))
+      usage_error(
+          "Security Error: You cannot specify the root"
+          " directory (/) as the log file directory.",
+          0);
+
+    /* strip ending / */
+    while (p[0] && p[strlen(p) - 1] == '/') p[strlen(p) - 1] = 0;
+    opt.l = p;
+
+    if (0 == strlen(opt.l)) opt.l = ".";
+
+    /* check that the file exists */
+    f = fopen(opt.l, "a");
+    if (!f) {
+      fprintf(stderr, "unable to create log file %s: %s\n", opt.l,
+              strerror(errno));
+      exit(1);
+    }
+    fclose(f);
+  }
+
+#ifdef USE_SSL
+  /* validate opt.ssl */
+  if (opt.ssl) {
+    char* p = gstring_trim(apr_pstrdup(pool, opt.ssl));
+
+    /* collapse // */
+    while (p[0] == '/' && p[1] == '/') p++;
+
+    /* disallow / */
+    if (0 == strcmp(p, "/"))
+      usage_error(
+          "Security Error: You cannot specify the root"
+          " directory (/) as the certificates directory",
+          0);
+
+    /* strip ending / */
+    while (p[0] && p[strlen(p) - 1] == '/') p[strlen(p) - 1] = 0;
+    opt.ssl = p;
+
+    /* change current directory to original one (after -d changed it) */
+    if (0 != apr_filepath_set(current_directory, pool))
+      usage_error(apr_psprintf(pool,
+                               "Error: cannot access directory '%s'\n"
+                               "Please run gpfdist from a different location",
+                               current_directory),
+                  0);
+    /* check that the dir exists */
+    if ((0 != apr_filepath_set(opt.ssl, pool)) ||
+        (0 != apr_filepath_get(&p, APR_FILEPATH_NATIVE, pool)))
+      usage_error(
+          apr_psprintf(pool,
+                       "Error: cannot access directory '%s'\n"
+                       "Please specify a valid directory for --ssl switch",
+                       opt.ssl),
+          0);
+    opt.ssl = p;
+  }
+#endif
 
 #ifdef GPFXDIST
-    /* validate opt.c */
-    if (opt.c)
-    {
-        extern int transform_config(const char* filename, struct transform** trlistp, int verbose);
-
-		if (transform_config(opt.c, &opt.trlist, opt.V))
-		{
-			/* transform_config has already printed a message to stderr on failure */
-			exit(1);
-        }
+  /* validate opt.c */
+  if (opt.c) {
+    extern int transform_config(const char* filename,
+                                struct transform** trlistp, int verbose);
+
+    if (transform_config(opt.c, &opt.trlist, opt.V)) {
+      /* transform_config has already printed a message to stderr on failure */
+      exit(1);
     }
+  }
 #endif
 
-	/* there should not be any more args left */
-	if (os->ind != argc)
-		usage_error("Error: illegal arguments", 1);
+  /* there should not be any more args left */
+  if (os->ind != argc) usage_error("Error: illegal arguments", 1);
 }
 
 /* http error codes used by gpfdist */
 #define FDIST_OK 200
 #define FDIST_BAD_REQUEST 400
+#define FDIST_TIMEOUT 408
 #define FDIST_INTERNAL_ERROR 500
 
 /* send an error response */
-static void http_error(request_t* r, int code, const char* msg)
-{
-	char buf[1024];
-	int n;
-	gprint("%s - %d %s\n", r->peer, code, msg);
-	n = apr_snprintf(buf, sizeof(buf), "HTTP/1.0 %d %s\r\n"
-		"Content-length: 0\r\n"
-		"Expires: 0\r\n"
-		"X-GPFDIST-VERSION: " GP_VERSIONX "\r\n"
-		"Cache-Control: no-cache\r\n"
-		"Connection: close\r\n\r\n", code, msg);
-
-	gpfdist_send(r, buf, n);
+static void http_error(request_t* r, int code, const char* msg) {
+  char buf[1024];
+  int n;
+  gwarning(r, "HTTP ERROR: %s - %d %s\n", r->peer, code, msg);
+  n = apr_snprintf(buf, sizeof(buf),
+                   "HTTP/1.0 %d %s\r\n"
+                   "Content-length: 0\r\n"
+                   "Expires: 0\r\n"
+                   "X-GPFDIST-VERSION: " GP_VERSION
+                   "\r\n"
+                   "Cache-Control: no-cache\r\n"
+                   "Connection: close\r\n\r\n",
+                   code, msg);
+
+  local_send(r, buf, n);
 }
 
 /* send an empty response */
-static void http_empty(request_t* r)
-{
-	static const char buf[] = "HTTP/1.0 200 ok\r\n"
-		"Content-type: text/plain\r\n"
-		"Content-length: 0\r\n"
-		"Expires: 0\r\n"
-		"X-GPFDIST-VERSION: " GP_VERSIONX "\r\n"
-		"Cache-Control: no-cache\r\n"
-		"Connection: close\r\n\r\n";
-	gprint("%s %s %s - OK\n", r->peer, r->in.req->argv[0], r->in.req->argv[1]);
-	gpfdist_send(r, buf, sizeof buf -1);
+static void http_empty(request_t* r) {
+  static const char buf[] =
+      "HTTP/1.0 200 ok\r\n"
+      "Content-type: text/plain\r\n"
+      "Content-length: 0\r\n"
+      "Expires: 0\r\n"
+      "X-GPFDIST-VERSION: " GP_VERSION
+      "\r\n"
+      "Cache-Control: no-cache\r\n"
+      "Connection: close\r\n\r\n";
+  gprintln(r, "HTTP EMPTY: %s %s %s - OK", r->peer, r->in.req->argv[0],
+           r->in.req->argv[1]);
+  local_send(r, buf, sizeof buf - 1);
 }
 
 /* send a Continue response */
-static void http_continue(request_t* r)
-{
-	static const char buf[] = "HTTP/1.1 100 Continue\r\n\r\n";
+static void http_continue(request_t* r) {
+  static const char buf[] = "HTTP/1.1 100 Continue\r\n\r\n";
 
-	gprint("%s %s %s - Continue\n", r->peer, r->in.req->argv[0], r->in.req->argv[1]);
+  gprintlnif(r, "%s %s %s - Continue", r->peer, r->in.req->argv[0],
+             r->in.req->argv[1]);
 
-	gpfdist_send(r, buf, sizeof buf -1);
+  local_send(r, buf, sizeof buf - 1);
 }
 
-
 /* send an OK response */
-static apr_status_t http_ok(request_t* r)
-{
-	const char* fmt = "HTTP/1.0 200 ok\r\n"
-		"Content-type: text/plain\r\n"
-		"Expires: 0\r\n"
-		"X-GPFDIST-VERSION: " GP_VERSIONX "\r\n"
-		"X-GP-PROTO: %d\r\n"
-		"Cache-Control: no-cache\r\n"
-		"Connection: close\r\n\r\n";
-	char buf[1024];
-	int m, n;
-
-	n = apr_snprintf(buf, sizeof(buf), fmt, r->gp_proto);
-	if (n >= sizeof(buf) - 1)
-		gfatal(FLINE, "internal error - buffer overflow during http_ok");
-
-	m = gpfdist_send(r, buf, n);
-	if (m != n)
-	{
-		gprint("%s - socket error\n", r->peer);
-		return APR_EGENERAL;
-	}
-	gprint("%s %s %s - OK\n", r->peer, r->in.req->argv[0], r->in.req->argv[1]);
-
-	return 0;
+static apr_status_t http_ok(request_t* r) {
+  const char* fmt =
+      "HTTP/1.0 200 ok\r\n"
+      "Content-type: text/plain\r\n"
+      "Expires: 0\r\n"
+      "X-GPFDIST-VERSION: " GP_VERSION
+      "\r\n"
+      "X-GP-PROTO: %d\r\n"
+      "Cache-Control: no-cache\r\n"
+      "Connection: close\r\n\r\n";
+  char buf[1024];
+  int m, n;
+
+  n = apr_snprintf(buf, sizeof(buf), fmt, r->gp_proto);
+  if (n >= sizeof(buf) - 1)
+    gfatal(r, "internal error - buffer overflow during http_ok");
+
+  m = local_send(r, buf, n);
+  if (m != n) {
+    gprintln(r, "%s - socket error\n", r->peer);
+    return APR_EGENERAL;
+  }
+  gprintlnif(r, "%s %s %s - OK", r->peer, r->in.req->argv[0],
+             r->in.req->argv[1]);
+
+  return 0;
+}
+
+static void log_gpfdist_status() {
+  char buf[1024];
+  int i;
+
+  int num_sessions = apr_hash_count(gcb.session.tab);
+  gprint(NULL, "---------------------------------------\n");
+  gprint(NULL, "STATUS: total session(s) %d\n", num_sessions);
+
+  apr_hash_index_t* hi = apr_hash_first(gcb.pool, gcb.session.tab);
+  for (i = 0; hi && i < num_sessions; i++, hi = apr_hash_next(hi)) {
+    void* entry;
+    apr_hash_this(hi, 0, 0, &entry);
+    session_t* s = (session_t*)entry;
+    if (s == NULL) {
+      gprint(NULL, "session %d: NULL\n", i);
+      continue;
+    }
+    const char* ferror =
+        (s->fstream == NULL ? NULL : fstream_get_error(s->fstream));
+    gprint(NULL,
+           "session %d: tid=%s, fs_error=%s, is_error=%d, nrequest=%d "
+           "is_get=%d, maxsegs=%d\n",
+           i, s->tid, (ferror == NULL ? "N/A" : ferror), s->is_error,
+           s->nrequest, s->is_get, s->maxsegs);
+    session_active_segs_dump(s);
+  }
+
+  printf("session: [\r\n");
+
+  hi = apr_hash_first(gcb.pool, gcb.session.tab);
+  for (i = 0; hi && i < num_sessions; i++, hi = apr_hash_next(hi)) {
+    void* entry;
+    apr_hash_this(hi, 0, 0, &entry);
+    session_t* s = (session_t*)entry;
+    if (s == NULL) {
+      continue;
+    }
+    (void)apr_snprintf(buf, sizeof(buf),
+                       "\t%s :{\r\n"
+                       "\t\tnrequest: %d\r\n"
+                       "\t\tis_get: %d\r\n"
+                       "\t\tpath: %s\r\n"
+                       "\t\trequest: [\r\n",
+                       s->tid, s->nrequest, s->is_get, s->path);
+
+    printf("%s\n", buf);
+
+    int j;
+    int num_requests = apr_hash_count(s->requests);
+    apr_hash_index_t* hj = apr_hash_first(s->pool, s->requests);
+    for (j = 0; hj && j < num_requests; j++, hj = apr_hash_next(hj)) {
+      void* entry;
+      apr_hash_this(hj, 0, 0, &entry);
+      request_t* r = (request_t*)entry;
+      if (r == NULL) {
+        continue;
+      }
+      apr_snprintf(buf, sizeof(buf),
+                   "\t\t\t%ld : {\r\n"
+                   "\t\t\t\tbytes: %ld\r\n"
+                   "\t\t\t\tunsent_bytes: %d\r\n"
+                   "\t\t\t\tlast: %s\r\n"
+#ifdef WIN32
+                   "\t\t\t\tseq: %ld\r\n"
+#else
+                   "\t\t\t\tseq: %" APR_INT64_T_FMT
+                   "\r\n"
+#endif
+                   "\t\t\t\tis_final: %d\r\n"
+                   "\t\t\t\tsegid: %d\r\n"
+                   "\t\t\t}\r\n",
+                   r->id, r->bytes, get_unsent_bytes(r), datetime(r->last),
+#ifdef WIN32
+                   (long)r->seq,
+#else
+                   r->seq,
+#endif
+                   r->is_final, r->segid);
+
+      printf("%s\n", buf);
+    }
+    printf("\t\t]\r\n\t}\r\n");
+  }
+  printf("]\r\n");
+
+  gprint(NULL, "---------------------------------------\n");
 }
 
 /*
@@ -806,36 +1007,81 @@ static apr_status_t http_ok(request_t* r)
  * debugging purposes. For more information see do_read, search for
  * 'gpfdist/status'.
  */
-static apr_status_t send_gpfdist_status(request_t* r)
-{
-	char	buf[1024];
-	int 	m, n;
-
-	n = apr_snprintf(buf, sizeof(buf),	"HTTP/1.0 200 ok\r\n"
-										"Content-type: text/plain\r\n"
-										"Expires: 0\r\n"
-										"X-GPFDIST-VERSION: " GP_VERSIONX "\r\n"
-										"Cache-Control: no-cache\r\n"
-										"Connection: close\r\n\r\n"
-										"read_bytes %"APR_INT64_T_FMT"\r\n"
-										"total_bytes %"APR_INT64_T_FMT"\r\n"
-										"total_sessions %d\r\n",
-										gcb.read_bytes,
-										gcb.total_bytes,
-										gcb.total_sessions);
-
-	if (n >= sizeof buf - 1)
-		gfatal(FLINE, "internal error - buffer overflow during send_gpfdist_status");
-
-	m = gpfdist_send(r, buf, n);
-
-	if (m != n)
-	{
-		gprint("%s - socket error\n", r->peer);
-		return APR_EGENERAL;
-	}
-
-	return 0;
+static apr_status_t send_gpfdist_status(request_t* r) {
+  log_gpfdist_status();
+
+  /*
+   * TODO: return response body is json encoded like:
+   * {
+   *   "request_time": "requst_time 2014-08-13 16:17:13",
+   *   "read_bytes": 1787522407,
+   *   "total_bytes": 3147292500,
+   *   "total_sessions": 2,
+   *   sessions: [
+   *   	"1" : {
+   *   	   	"tid": session->tid,
+   *   	   	"nrequest": session->nrequest,
+   *		   	"is_get": session->is_get,
+   *		   	"path": session->path,
+   *		   	"requests": [
+   *		   		"segid1": {
+   *					"bytes": request->bytes,
+   *					"last": request->last,
+   *					"seq": request->seq,
+   *					"is_final": request->is_final,
+   *					"segid": request->segid,
+   *
+   *		   		},
+   *		   		"segid2": {
+   *
+   *		   		}
+   *		   	]
+   *   	},
+   *   	"2" : {
+   *   	}
+   *   ]
+   * }
+   */
+  char buf[1024];
+  char* time = datetime_now();
+  int n = apr_snprintf(buf, sizeof(buf),
+                       "HTTP/1.0 200 ok\r\n"
+                       "Content-type: text/plain\r\n"
+                       "Expires: 0\r\n"
+                       "X-GPFDIST-VERSION: " GP_VERSION
+                       "\r\n"
+                       "Cache-Control: no-cache\r\n"
+                       "Connection: close\r\n\r\n"
+                       "requst_time %s\r\n"
+#ifdef WIN32
+                       "read_bytes %ld\r\n"
+                       "total_bytes %ld\r\n"
+#else
+                       "read_bytes %" APR_INT64_T_FMT
+                       "\r\n"
+                       "total_bytes %" APR_INT64_T_FMT
+                       "\r\n"
+#endif
+                       "total_sessions %d\r\n",
+                       time,
+#ifdef WIN32
+                       (long)gcb.read_bytes, (long)gcb.total_bytes,
+#else
+                       gcb.read_bytes, gcb.total_bytes,
+#endif
+                       gcb.total_sessions);
+
+  if (n >= sizeof buf - 1)
+    gfatal(r, "internal error - buffer overflow during send_gpfdist_status");
+
+  int m = local_send(r, buf, n);
+
+  if (m != n) {
+    gprint(r, "%s - socket error\n", r->peer);
+    return APR_EGENERAL;
+  }
+
+  return 0;
 }
 
 /*
@@ -846,115 +1092,113 @@ static apr_status_t send_gpfdist_status(request_t* r)
  * Maybe there is a request out there (of the same session), that still
  * has a block to send out.
  */
-static void request_end(request_t* r, int error, const char* errmsg)
-{
-	session_t* s = r->session;
+static void request_end(request_t* r, int error, const char* errmsg) {
+  session_t* s = r->session;
 
 #ifdef GPFXDIST
-	if (r->trans.errfile)
-	{
-		apr_status_t rv;
-
-		/*
-		 * close and then re-open (for reading) the temporary file we've used to capture stderr
-		 */
-		apr_file_flush(r->trans.errfile);
-		apr_file_close(r->trans.errfile);
-		TR(("[%d] request closed stderr file %s\n", r->sock, r->trans.errfilename));
-
-		/*
-		 * send the first 8K of stderr to the server
-		 */
-		if ((rv = apr_file_open(&r->trans.errfile, r->trans.errfilename, APR_READ|APR_BUFFERED, APR_UREAD, r->pool)) == APR_SUCCESS)
-		{
-			gp1_send_errfile(r, r->trans.errfile);
-			apr_file_close(r->trans.errfile);
-		}
-
-		/*
-		 * remove the temp file
-		 */
-		apr_file_remove(r->trans.errfilename, r->pool);
-		TR(("[%d] request removed stderr file %s\n", r->sock, r->trans.errfilename));
-
-		r->trans.errfile = NULL;
-	}
-
-#endif
-
-	if (r->gp_proto == 1)
-	{
-		if (!error)
-			gp1_send_eof(r);
-		else if (errmsg)
-			gp1_send_errmsg(r, errmsg);
-	}
-
-	TR(("[%d] request end\n", r->sock));
-
-	/* If we still have a block outstanding, the session is corrupted. */
-	if (r->outblock.top != r->outblock.bot)
-	{
-		gwarning(FLINE, "request failure resulting in session failure");
-		if (s)
-			session_end(s, 1);
-	}
-	else
-	{
-		/* detach this request from its session */
-		session_detach(r);
-	}
-
-	/* If we still have data in the buffer - flush it */
-	if ( opt.ssl )
-	{
-		flush_ssl_buffer(r->sock, 0, r);
-	}
-	else
-	{
-		request_cleanup(r);
-	}
-}
-
-static int local_send(request_t *r, const char* buf, int buflen)
-{
-	int n = gpfdist_send(r, buf, buflen);
-
-	if (n < 0)
-	{
+  if (r->trans.errfile) {
+    apr_status_t rv;
+
+    /*
+     * close and then re-open (for reading) the temporary file we've used to
+     * capture stderr
+     */
+    apr_file_flush(r->trans.errfile);
+    apr_file_close(r->trans.errfile);
+    gprintln(r, "request closed stderr file %s", r->trans.errfilename);
+
+    /*
+     * send the first 8K of stderr to the server
+     */
+    if ((rv = apr_file_open(&r->trans.errfile, r->trans.errfilename,
+                            APR_READ | APR_BUFFERED, APR_UREAD, r->pool)) ==
+        APR_SUCCESS) {
+      gp1_send_errfile(r, r->trans.errfile);
+      apr_file_close(r->trans.errfile);
+    }
+
+    /*
+     * remove the temp file
+     */
+    apr_file_remove(r->trans.errfilename, r->pool);
+    gprintln(r, "request removed stderr file %s", r->trans.errfilename);
+
+    r->trans.errfile = NULL;
+  }
+
+#endif
+
+  if (r->gp_proto == 1) {
+    if (!error)
+      gp1_send_eof(r);
+    else if (errmsg)
+      gp1_send_errmsg(r, errmsg);
+  }
+
+  gprintlnif(r, "request end");
+
+  /* If we still have a block outstanding, the session is corrupted. */
+  if (r->outblock.top != r->outblock.bot) {
+    gwarning(r,
+             "request failure resulting in session failure: top = %d, bot = %d",
+             r->outblock.top, r->outblock.bot);
+    if (s) session_end(s, 1);
+  } else {
+    /* detach this request from its session */
+    session_detach(r);
+  }
+
+  /* If we still have data in the buffer - flush it */
+#ifdef USE_SSL
+  if (opt.ssl)
+    flush_ssl_buffer(r->sock, 0, r);
+  else
+    request_cleanup(r);
+#else
+  request_cleanup(r);
+#endif
+}
+
+static int local_send(request_t* r, const char* buf, int buflen) {
+  int n = gpfdist_send(r, buf, buflen);
+
+  if (n < 0) {
 #ifdef WIN32
-		int e = WSAGetLastError();
-		int ok = (e == WSAEINTR || e == WSAEWOULDBLOCK);
+    int e = WSAGetLastError();
+    int ok = (e == WSAEINTR || e == WSAEWOULDBLOCK);
 #else
-		int e = errno;
-		int ok = (e == EINTR || e == EAGAIN);
+    int e = errno;
+    int ok = (e == EINTR || e == EAGAIN);
 #endif
-		if ( e == EPIPE )
-		{
-			gprint(FLINE,"gpfdist_send failed - the connection was terminated by the client (%d)\n",e);
-		}
-		return ok ? 0 : -1;
-	}
+    if (e == EPIPE || e == ECONNRESET) {
+      gwarning(r,
+               "gpfdist_send failed - the connection was terminated by the "
+               "client (%d: %s)",
+               e, strerror(e));
+      /* close stream and release fd & flock on pipe file*/
+      if (r->session) session_end(r->session, 0);
+    } else {
+      gdebug(r, "gpfdist_send failed - due to (%d: %s)", e, strerror(e));
+    }
+    return ok ? 0 : -1;
+  }
 
-	return n;
+  return n;
 }
 
-static int local_sendall(request_t* r, const char* buf, int buflen)
-{
-	int oldlen = buflen;
+static int local_sendall(request_t* r, const char* buf, int buflen) {
+  int oldlen = buflen;
 
-	while (buflen)
-	{
-		int i = local_send(r, buf, buflen);
+  while (buflen) {
+    int i = local_send(r, buf, buflen);
 
-		if (i < 0)
-			return i;
+    if (i < 0) return i;
 
-		buf += i;
-		buflen -= i;
-	}
+    buf += i;
+    buflen -= i;
+  }
 
-	return oldlen;
+  return oldlen;
 }
 
 /*
@@ -969,63 +1213,60 @@ static int local_sendall(request_t* r, const char* buf, int buflen)
  * this function creates and sends the meta info according to the passed in
  * arguments. It does not send the block itself (bytes 5-X).
  */
-static int
-gp1_send_header(request_t*r, char letter, int length)
-{
-	char 		hdr[5];
-	const char*	p = hdr;
+static int gp1_send_header(request_t* r, char letter, int length) {
+  char hdr[5];
+  const char* p = hdr;
 
-	hdr[0] = letter;
-	length = htonl(length);
+  hdr[0] = letter;
+  length = htonl(length);
 
-	memcpy(hdr + 1, &length, 4);
+  memcpy(hdr + 1, &length, 4);
 
-	return local_sendall(r, p, 5) < 0 ? -1 : 0;
+  return local_sendall(r, p, 5) < 0 ? -1 : 0;
 }
 
 /*
  * Send a message to the client to indicate EOF - no more data. This is done
  * by sending a 'D' message type (Data) with length 0.
  */
-static void
-gp1_send_eof(request_t* r)
-{
-	if (!gp1_send_header(r, 'D', 0))
-		TR(("[%d] sent EOF\n", r->sock));
+static void gp1_send_eof(request_t* r) {
+  int result = gp1_send_header(r, 'D', 0);
+  gprintln(r, "sent EOF: %s", (result == 0 ? "succeed" : "fail"));
 }
 
 /*
  * Send an error message to the client, using the 'E' message type.
  */
-static void 
-gp1_send_errmsg(request_t* r, const char* errmsg)
-{
-	apr_int32_t len = strlen(errmsg);
-	if (!gp1_send_header(r, 'E', len))
-		local_sendall(r, errmsg, len);
+static void gp1_send_errmsg(request_t* r, const char* errmsg) {
+  gwarning(r, "send error message: %s", errmsg);
+
+  apr_int32_t len = strlen(errmsg);
+  if (!gp1_send_header(r, 'E', len)) {
+    local_sendall(r, errmsg, len);
+  } else {
+    gwarning(r, "failed to send error message");
+  }
 }
 
 #ifdef GPFXDIST
 /*
  * Send the first 8k of the specified file as an error message
  */
-static void gp1_send_errfile(request_t* r, apr_file_t* errfile)
-{
-	char         buf[8192];
-	apr_size_t   nbytes = sizeof(buf);
-	apr_status_t rv;
-
-	if ((rv = apr_file_read(errfile, buf, &nbytes)) == APR_SUCCESS)
-	{
-		if (nbytes > 0)
-		{
-			if (! gp1_send_header(r, 'E', nbytes))
-			{
-				local_sendall(r, buf, nbytes);
-				TR(("[%d] request sent %"APR_SIZE_T_FMT" stderr bytes to server\n", r->sock, nbytes));
-			}
-		}
-	}
+static void gp1_send_errfile(request_t* r, apr_file_t* errfile) {
+  char buf[8192];
+  apr_size_t nbytes = sizeof(buf);
+  apr_status_t rv;
+
+  if ((rv = apr_file_read(errfile, buf, &nbytes)) == APR_SUCCESS) {
+    if (nbytes > 0) {
+      if (!gp1_send_header(r, 'E', nbytes)) {
+        local_sendall(r, buf, nbytes);
+        gdebug(r,
+               "[%d] request sent %" APR_SIZE_T_FMT " stderr bytes to server",
+               r->sock, nbytes);
+      }
+    }
+  }
 }
 #endif
 
@@ -1035,401 +1276,405 @@ static void gp1_send_errfile(request_t* r, apr_file_t* errfile)
  * Get a block out of the session. return error string. This includes a block
  * header (metadata for client such as filename, etc) and the data itself.
  */
-static const char*
-session_get_block(session_t* session, block_t* retblock, char* line_delim_str, int line_delim_length)
-{
-	int 		size;
-	const int 	whole_rows = 1; /* gpfdist must not read data with partial rows */
-	struct fstream_filename_and_offset fos;
+static const char* session_get_block(const request_t* r, block_t* retblock,
+                                     char* line_delim_str,
+                                     int line_delim_length) {
+  int size;
+  const int whole_rows = 1; /* gpfdist must not read data with partial rows */
+  struct fstream_filename_and_offset fos = {0};
+
+  session_t* session = r->session;
 
-	retblock->bot = retblock->top = 0;
+  retblock->bot = retblock->top = 0;
 
-	if (session->is_error || 0 == session->fstream)
-	{
-		session_end(session, 0);
-		return 0;
-	}
+  if (session->is_error || 0 == session->fstream) {
+    gprintln(NULL, "session_get_block: end session is_error: %d",
+             session->is_error);
+    session_end(session, 0);
+    return 0;
+  }
+
+  gcb.read_bytes -= fstream_get_compressed_position(session->fstream);
 
-	gcb.read_bytes -= fstream_get_compressed_position(session->fstream);
+  /* read data from our filestream as a chunk with whole data rows */
 
-	/* read data from our filestream as a chunk with whole data rows */
-	
-	size = fstream_read(session->fstream, retblock->data, opt.m, &fos, whole_rows, line_delim_str, line_delim_length);
+  size = fstream_read(session->fstream, retblock->data, opt.m, &fos, whole_rows,
+                      line_delim_str, line_delim_length);
+  delay_watchdog_timer();
 
-	if (size == 0)
-	{
-		gcb.read_bytes += fstream_get_compressed_size(session->fstream);
-		session_end(session, 0);
-		return 0;
-	}
+  if (size == 0) {
+    gprintln(NULL, "session_get_block: end session due to EOF");
+    gcb.read_bytes += fstream_get_compressed_size(session->fstream);
+    session_end(session, 0);
+    return 0;
+  }
 
-	gcb.read_bytes += fstream_get_compressed_position(session->fstream);
+  gcb.read_bytes += fstream_get_compressed_position(session->fstream);
 
-	if (size < 0)
-	{
-		const char* ferror = fstream_get_error(session->fstream);
-		session_end(session, 1);
-		return ferror;
-	}
+  if (size < 0) {
+    const char* ferror = fstream_get_error(session->fstream);
+    gwarning(NULL, "session_get_block end session due to %s", ferror);
+    session_end(session, 1);
+    return ferror;
+  }
 
-	retblock->top = size;
+  retblock->top = size;
 
-	/* fill the block header with meta data for the client to parse and use */
-	block_fill_header(retblock, &fos);
+  /* fill the block header with meta data for the client to parse and use */
+  block_fill_header(r, retblock, &fos);
 
-	return 0;
+  return 0;
 }
 
 /* finish the session - close the file */
-static void session_end(session_t* session, int error)
-{
-	if (error)
-		session->is_error = error;
+static void session_end(session_t* session, int error) {
+  gprintln(NULL, "session end.");
+
+  if (error) session->is_error = error;
 
-	if (session->fstream)
-	{
-		fstream_close(session->fstream);
-		session->fstream = 0;
-	}
+  if (session->fstream) {
+    fstream_close(session->fstream);
+    session->fstream = 0;
+  }
 }
 
 /* deallocate session, remove from hashtable */
-static void session_free(session_t* session)
-{
-	gprint("free session %s\n", session->key);
+static void session_free(session_t* session) {
+  gprintln(NULL, "free session %s", session->key);
 
-	if (session->fstream)
-	{
-		fstream_close(session->fstream);
-		session->fstream = 0;
-	}
+  if (session->fstream) {
+    fstream_close(session->fstream);
+    session->fstream = 0;
+  }
 
-	apr_hash_set(gcb.session.tab, session->key, APR_HASH_KEY_STRING, 0);
-	apr_pool_destroy(session->pool);
+  event_del(&session->ev);
+
+  apr_hash_set(gcb.session.tab, session->key, APR_HASH_KEY_STRING, 0);
+  apr_pool_destroy(session->pool);
 }
 
 /* detach a request from a session */
-static void session_detach(request_t* r)
-{
-	session_t* session = r->session;
-
-	r->session = 0;
-
-	if (session)
-	{
-		if (session->nrequest <= 0)
-			gfatal(FLINE, "internal error - detaching a request from an empty session");
-
-		session->nrequest--;
-		session->mtime = apr_time_now();
-
-		/* for auto-tid sessions, we can free it now */
-		if (0 == strncmp("auto-tid.", session->tid, 9))
-		{
-			if (session->nrequest != 0)
-				gfatal(FLINE, "internal error - expected an empty auto-tid session"
-							  "but saw %d requests", session->nrequest);
-
-			session_free(session);
-		}
-		else if(!session->is_get &&
-				session->nrequest == 0 &&
-				session_active_segs_isempty(session))
-		{
-			/*
-			 * free the session if this is a POST request and it's
-			 * the last request for this session (we can tell is all
-			 * segments sent a "done" request by calling session_active_isempty.
-			 * (nrequest == 0 test isn't sufficient by itself).
-			 *
-			 * this is needed in order to make sure to close the out file
-			 * when we're done writing. (only in write operations, not in read).
-			 */
-			session_free(session);
-		}
-	}
-}
-
-static void sessions_cleanup(void)
-{
-	apr_hash_index_t*	hi;
-	int 				i, n = 0;
-	void* 				entry;
-	session_t**			session;
-	session_t*			s;
-	int 				numses;
-
-	numses = apr_hash_count(gcb.session.tab);
-
-	if (numses == 0)
-		return;
-
-	if (!(session = malloc(sizeof(session_t *) * numses)))
-		gfatal(FLINE, "out of memory in sessions_cleanup");
-
-	hi = apr_hash_first(gcb.pool, gcb.session.tab);
-
-	for (i = 0; hi && i < numses; i++, hi = apr_hash_next(hi))
-	{
-		apr_hash_this(hi, 0, 0, &entry);
-		s = (session_t*) entry;
-
-		if (s->nrequest == 0 && (s->mtime < apr_time_now() - 300
-				* APR_USEC_PER_SEC))
-		{
-			session[n++] = s;
-		}
-	}
-
-	for (i = 0; i < n; i++)
-	{
-		gprint("remove out-dated session %s\n", session[i]->key);
-		session_free(session[i]);
-		session[i] = 0;
-	}
-
-	free(session);
-}
+static void session_detach(request_t* r) {
+  gprintlnif(r, "detach segment request from session");
 
-/*
- * session_attach
- *
- * attach a request to a session (create the session if not already exists).
- */
-static int session_attach(request_t* r)
-{
-	char key[1024];
-	session_t* session = 0;
-
-	/*
-	 * create the session key (tid:path)
-	 */
-	if (sizeof(key) - 1 == apr_snprintf(key, sizeof(key), "%s:%s",
-										r->tid, r->path))
-	{
-		http_error(r, FDIST_BAD_REQUEST, "path too long");
-		request_end(r, 1, 0);
-		return -1;
-	}
-
-
-	/* check if such session already exists in hashtable */
-	session = apr_hash_get(gcb.session.tab, key, APR_HASH_KEY_STRING);
-
-	if (!session)
-	{
-		/* not in hashtable - create new session */
-
-		fstream_t* 	fstream = 0;
-		apr_pool_t* pool;
-		int 		response_code;
-		const char*	response_string;
-		struct fstream_options fstream_options;
-
-		/* remove any outdated sessions*/
-		sessions_cleanup();
-
-		/*
-		 * this is the special WET "session-end" request. Another similar
-		 * request must have already came in from another segdb and finished
-		 * the session we were at. we don't want to create a new session now,
-		 * so just exit instead
-		 */
-		if (r->is_final)
-		{
-			gprint("got a final write request. skipping session creation\n");
-			http_empty(r);
-			request_end(r, 0, 0);
-			return -1;
-		}
-
-		if (apr_pool_create(&pool, gcb.pool))
-		{
-			gwarning(FLINE, "out of memory");
-			http_error(r, FDIST_INTERNAL_ERROR, "internal error - out of memory");
-			request_end(r, 1, 0);
-			return -1;
-		}
-
-		/* parse csvopt header */
-		memset(&fstream_options, 0, sizeof fstream_options);
-		fstream_options.verbose = opt.v;
-		fstream_options.bufsize = opt.m;
-
-		{
-			int quote = 0;
-			int escape = 0;
-
-			sscanf(r->csvopt, "m%dx%dq%dh%d", &fstream_options.is_csv, &escape,
-					&quote, &fstream_options.header);
-			fstream_options.quote = quote;
-			fstream_options.escape = escape;
-		}
-
-		/* set fstream for read (GET) or write (PUT) */
-		if (r->is_get)
-			fstream_options.forwrite = 0; /* GET request */
-		else
-		{
-			fstream_options.forwrite = 1; /* PUT request */
-			fstream_options.usesync = opt.S;
-		}
+  session_t* session = r->session;
 
-#ifdef GPFXDIST
-		/* set transformation options */
-        if (r->trans.command)
-        {
-            fstream_options.transform = (struct gpfxdist_t*) apr_pcalloc(pool, sizeof(struct gpfxdist_t));
+  r->session = 0;
 
-            if (! fstream_options.transform)
-                    gfatal(FLINE, "out of memory in session_attach");
+  if (session) {
+    if (session->nrequest <= 0)
+      gfatal(r, "internal error - detaching a request from an empty session");
 
-            fstream_options.transform->cmd        = r->trans.command;
-			fstream_options.transform->pass_paths = r->trans.paths;
-            fstream_options.transform->for_write  = fstream_options.forwrite;
-            fstream_options.transform->mp         = pool;
-			fstream_options.transform->errfile    = r->trans.errfile;
-        }
-		gprint("r->path %s\n", r->path);
-#endif
+    session->nrequest--;
+    session->mtime = apr_time_now();
+    apr_hash_set(session->requests, &r->id, sizeof(r->id), NULL);
 
-		/* try opening the fstream */
-		gprint("new session trying to open the data stream\n");
-		fstream = fstream_open(r->path, &fstream_options, &response_code, &response_string);
+    if (session->is_get && session->nrequest == 0) {
+      gprintln(r, "session has finished all segment requests");
+    }
 
-		if (!fstream)
-		{
-			gwarning(FLINE, "reject request from %s, path %s\n", r->peer,
-					r->path);
-			http_error(r, response_code, response_string);
-			request_end(r, 1, 0);
-			apr_pool_destroy(pool);
-			return -1;
-		}
+    /* for auto-tid sessions, we can free it now */
+    if (0 == strncmp("auto-tid.", session->tid, 9)) {
+      if (session->nrequest != 0)
+        gwarning(r,
+                 "internal error - expected an empty auto-tid session but saw "
+                 "%d requests",
+                 session->nrequest);
+
+      session_free(session);
+    } else if (!session->is_get && session->nrequest == 0 &&
+               session_active_segs_isempty(session)) {
+      /*
+       * free the session if this is a POST request and it's
+       * the last request for this session (we can tell is all
+       * segments sent a "done" request by calling session_active_isempty.
+       * (nrequest == 0 test isn't sufficient by itself).
+       *
+       * this is needed in order to make sure to close the out file
+       * when we're done writing. (only in write operations, not in read).
+       */
+#ifdef WIN32
+      if (!fstream_is_win_pipe(session->fstream)) {
+        session_free(session);
+        return;
+      }
+#endif
 
-		gprint("new session successfully opened the data stream\n");
+      if (opt.w == 0) {
+        session_free(session);
+        return;
+      }
 
-		gcb.total_sessions++;
-		gcb.total_bytes += fstream_get_compressed_size(fstream);
+      event_del(&session->ev);
+      evtimer_set(&session->ev, free_session_cb, session);
+      session->tm.tv_sec = opt.w;
+      session->tm.tv_usec = 0;
+      (void)evtimer_add(&session->ev, &session->tm);
+    }
+  }
+}
 
-		/* allocate session */
-		session = apr_palloc(pool, sizeof(session_t));
+static void sessions_cleanup(void) {
+  apr_hash_index_t* hi;
+  int i, n = 0;
+  void* entry;
+  session_t** session;
+  session_t* s;
+  int numses;
 
-		if (session == 0)
-			gfatal(FLINE, "out of memory in session_attach");
+  gprintln(NULL, "remove sessions");
 
-		memset(session, 0, sizeof(*session));
+  numses = apr_hash_count(gcb.session.tab);
 
-		/* allocate active_segdb array (session member) */
-		session->active_segids = (int *) apr_palloc(pool, sizeof(int) * r->totalsegs);
+  if (numses == 0) return;
 
-		if (session->active_segids == 0)
-			gfatal(FLINE, "out of memory when allocating active_segids array");
+  if (!(session = malloc(sizeof(session_t*) * numses)))
+    gfatal(NULL, "out of memory in sessions_cleanup");
 
-		memset(session->active_segids, 0, sizeof(int) * r->totalsegs);
+  hi = apr_hash_first(gcb.pool, gcb.session.tab);
 
-		/* initialize session values */
-		session->tid = apr_pstrdup(pool, r->tid);
-		session->path = apr_pstrdup(pool, r->path);
-		session->key = apr_pstrdup(pool, key);
-		session->fstream = fstream;
-		session->pool = pool;
-		session->is_get = r->is_get;
-		session->active_segids[r->segid] = 1; /* mark this segid as active */
-		session->maxsegs = r->totalsegs;
+  for (i = 0; hi && i < numses; i++, hi = apr_hash_next(hi)) {
+    apr_hash_this(hi, 0, 0, &entry);
+    s = (session_t*)entry;
 
-		if (session->tid == 0 || session->path == 0 || session->key == 0)
-			gfatal(FLINE, "out of memory in session_attach");
+    if (s->nrequest == 0 &&
+        (s->mtime < apr_time_now() - 300 * APR_USEC_PER_SEC)) {
+      session[n++] = s;
+    }
+  }
 
-		/* insert into hashtable */
-		apr_hash_set(gcb.session.tab, session->key, APR_HASH_KEY_STRING, session);
+  for (i = 0; i < n; i++) {
+    gprint(NULL, "remove out-dated session %s\n", session[i]->key);
+    session_free(session[i]);
+    session[i] = 0;
+  }
 
-		gprint("new session (%s, %s)\n", session->path, session->tid);
-	}
+  free(session);
+}
 
-	/* found a session in hashtable*/
+/*
+ * session_attach
+ *
+ * attach a request to a session (create the session if not already exists).
+ */
+static int session_attach(request_t* r) {
+  char key[1024];
+  session_t* session = NULL;
+
+  /*
+   * create the session key (tid:path)
+   */
+  if (sizeof(key) - 1 ==
+      apr_snprintf(key, sizeof(key), "%s:%s", r->tid, r->path)) {
+    http_error(r, FDIST_BAD_REQUEST, "path too long");
+    request_end(r, 1, 0);
+    return -1;
+  }
+
+  /* check if such session already exists in hashtable */
+  session = apr_hash_get(gcb.session.tab, key, APR_HASH_KEY_STRING);
+
+  if (!session) {
+    /* not in hashtable - create new session */
+
+    fstream_t* fstream = 0;
+    apr_pool_t* pool;
+    int response_code;
+    const char* response_string;
+    struct fstream_options fstream_options;
+
+    /* remove any outdated sessions*/
+    sessions_cleanup();
 
-	/* if error, send an error and close */
-	if (session->is_error)
-	{
-		http_error(r, FDIST_INTERNAL_ERROR, "session error");
-		request_end(r, 1, 0);
-		return -1;
-	}
+    /*
+     * this is the special WET "session-end" request. Another similar
+     * request must have already came in from another segdb and finished
+     * the session we were at. we don't want to create a new session now,
+     * so just exit instead
+     */
+    if (r->is_final) {
+      gprintln(r, "got a final write request. skipping session creation");
+      http_empty(r);
+      request_end(r, 0, 0);
+      return -1;
+    }
 
-	/* session already ended. send an empty response, and close. */
-	if (0 == session->fstream)
-	{
-		gprint("session already ended. return empty response (OK)\n");
+    if (apr_pool_create(&pool, gcb.pool)) {
+      gwarning(r, "out of memory");
+      http_error(r, FDIST_INTERNAL_ERROR, "internal error - out of memory");
+      request_end(r, 1, 0);
+      return -1;
+    }
 
-		http_empty(r);
-		request_end(r, 0, 0);
-		return -1;
-	}
+    /* parse csvopt header */
+    memset(&fstream_options, 0, sizeof fstream_options);
+    fstream_options.verbose = opt.v;
+    fstream_options.bufsize = opt.m;
 
-	/*
-	 * disallow mixing GET and POST requests in one session.
-	 * this will protect us from an infinitely running
-	 * INSERT INTO ext_t SELECT FROM ext_t
-	 */
-	if (r->is_get != session->is_get)
-	{
-		http_error(r, FDIST_BAD_REQUEST, "can\'t write to and read from the same "
-										 "gpfdist server simultaneously");
-		request_end(r, 1, 0);
-		return -1;
-	}
+    {
+      int quote = 0;
+      int escape = 0;
+      sscanf(r->csvopt, "m%dx%dq%dh%d", &fstream_options.is_csv, &escape,
+             &quote, &fstream_options.header);
+      fstream_options.quote = quote;
+      fstream_options.escape = escape;
+    }
 
-	gprint("joined session (%s, %s)\n", session->path, session->tid);
+    /* set fstream for read (GET) or write (PUT) */
+    if (r->is_get)
+      fstream_options.forwrite = 0; /* GET request */
+    else {
+      fstream_options.forwrite = 1; /* PUT request */
+      fstream_options.usesync = opt.S;
+    }
 
-	/* one more request for session */
-	session->nrequest++;
-	session->active_segids[r->segid] = !r->is_final;
-	session->mtime = apr_time_now();
-	r->session = session;
+#ifdef GPFXDIST
+    /* set transformation options */
+    if (r->trans.command) {
+      fstream_options.transform =
+          (struct gpfxdist_t*)pcalloc_safe(r, pool, sizeof(struct gpfxdist_t),
+                                           "out of memory in session_attach");
+
+      fstream_options.transform->cmd = r->trans.command;
+      fstream_options.transform->pass_paths = r->trans.paths;
+      fstream_options.transform->for_write = fstream_options.forwrite;
+      fstream_options.transform->mp = pool;
+      fstream_options.transform->errfile = r->trans.errfile;
+    }
+    gprintlnif(r, "r->path %s", r->path);
+#endif
 
-	if(!session->is_get)
-		session_active_segs_dump(session);
+    /* try opening the fstream */
+    gprintlnif(r, "new session trying to open the data stream");
+    fstream = fstream_open(r->path, &fstream_options, &response_code,
+                           &response_string);
+    delay_watchdog_timer();
+
+    if (!fstream) {
+      gwarning(r, "reject request from %s, path %s", r->peer, r->path);
+      http_error(r, response_code, response_string);
+      request_end(r, 1, 0);
+      apr_pool_destroy(pool);
+      return -1;
+    }
 
-	return 0;
+    gprintlnif(r, "new session successfully opened the data stream");
+
+    gcb.total_sessions++;
+    gcb.total_bytes += fstream_get_compressed_size(fstream);
+
+    /* allocate session */
+    session = pcalloc_safe(r, pool, sizeof(session_t),
+                           "out of memory in session_attach");
+
+    /* allocate active_segdb array (session member) */
+    session->active_segids =
+        (int*)pcalloc_safe(r, pool, sizeof(int) * r->totalsegs,
+                           "out of memory when allocating active_segids array");
+
+    /* allocate seq_segs array (session member) */
+    session->seq_segs = (apr_int64_t*)pcalloc_safe(
+        r, pool, sizeof(apr_int64_t) * r->totalsegs,
+        "out of memory when allocating seq_segs array");
+
+    /* initialize session values */
+    session->id = ++SESSION_SEQ;
+    session->tid = apr_pstrdup(pool, r->tid);
+    session->path = apr_pstrdup(pool, r->path);
+    session->key = apr_pstrdup(pool, key);
+    session->fstream = fstream;
+    session->pool = pool;
+    session->is_get = r->is_get;
+    session->active_segids[r->segid] = 1; /* mark this segid as active */
+    session->maxsegs = r->totalsegs;
+    session->requests = apr_hash_make(pool);
+
+    if (session->tid == 0 || session->path == 0 || session->key == 0)
+      gfatal(r, "out of memory in session_attach");
+
+    /* insert into hashtable */
+    apr_hash_set(gcb.session.tab, session->key, APR_HASH_KEY_STRING, session);
+
+    gprintlnif(r, "new session (%ld): (%s, %s)", session->id, session->path,
+               session->tid);
+  }
+
+  /* found a session in hashtable*/
+
+  /* if error, send an error and close */
+  if (session->is_error) {
+    http_error(r, FDIST_INTERNAL_ERROR, "session error");
+    request_end(r, 1, 0);
+    return -1;
+  }
+
+  /* session already ended. send an empty response, and close. */
+  if (NULL == session->fstream) {
+    gprintln(r, "session already ended. return empty response (OK)");
+
+    http_empty(r);
+    request_end(r, 0, 0);
+    return -1;
+  }
+
+  /*
+   * disallow mixing GET and POST requests in one session.
+   * this will protect us from an infinitely running
+   * INSERT INTO ext_t SELECT FROM ext_t
+   */
+  if (r->is_get != session->is_get) {
+    http_error(r, FDIST_BAD_REQUEST,
+               "can\'t write to and read from the same "
+               "gpfdist server simultaneously");
+    request_end(r, 1, 0);
+    return -1;
+  }
+
+  gprintlnif(r, "joined session (%s, %s)", session->path, session->tid);
+
+  /* one more request for session */
+  session->nrequest++;
+  session->active_segids[r->segid] = !r->is_final;
+  session->mtime = apr_time_now();
+  apr_hash_set(session->requests, &r->id, sizeof(r->id), r);
+
+  r->session = session;
+  r->sid = session->id;
+
+  if (!session->is_get) session_active_segs_dump(session);
+
+  return 0;
 }
 
 /*
  * Dump all the segdb ids that currently participate
  * in this session.
  */
-static void session_active_segs_dump(session_t* session)
-{
-	if(opt.v)
-	{
-		int i = 0;
+static void session_active_segs_dump(session_t* session) {
+  if (opt.v) {
+    int i = 0;
 
-		gprint("active segids in session: ");
+    gprint(NULL, "active segids in session: ");
 
-		for (i = 0 ; i < session->maxsegs ; i++)
-		{
-			if(session->active_segids[i] == 1)
-				printf("%d ", i);
-		}
-		printf("\n");
-	}
+    for (i = 0; i < session->maxsegs; i++) {
+      if (session->active_segids[i] == 1) printf("%d ", i);
+    }
+    printf("\n");
+  }
 }
 
 /*
  * Is there any segdb still sending us data? or are
  * all of them done already? if empty all are done.
  */
-static int session_active_segs_isempty(session_t* session)
-{
-	int i = 0;
+static int session_active_segs_isempty(session_t* session) {
+  int i = 0;
 
-	for (i = 0 ; i < session->maxsegs ; i++)
-	{
-		if(session->active_segids[i] == 1)
-			return 0; /* not empty */
-	}
+  for (i = 0; i < session->maxsegs; i++) {
+    if (session->active_segids[i] == 1) return 0; /* not empty */
+  }
 
-	return 1; /* empty */
+  return 1; /* empty */
 }
 
 /*
@@ -1437,106 +1682,124 @@ static int session_active_segs_isempty(session_t* session)
  *
  * Callback when the socket is ready to be written
  */
-void gfile_printf_then_putc_newline(const char *format, ...);
-
-static void do_write(int fd, short event, void* arg)
-{
-	request_t* 	r = (request_t*) arg;
-	int 		n, i;
-	block_t* 	datablock;
-
-	if (fd != r->sock)
-		gfatal(FLINE, "internal error - non matching fd (%d) "
-					  "and socket (%d)", fd, r->sock);
-
-	/* Loop at most 3 blocks or until we choke on the socket */
-	for (i = 0; i < 3; i++)
-	{
-		/* get a block (or find a remaining block) */
-		if (r->outblock.top == r->outblock.bot)
-		{
-			const char* ferror = session_get_block(r->session, &r->outblock, r->line_delim_str, r->line_delim_length);
-
-			if (ferror)
-			{
-				request_end(r, 1, ferror);
-				gfile_printf_then_putc_newline("%s", ferror);
-				return;
-			}
-			if (!r->outblock.top)
-			{
-				request_end(r, 0, 0);
-				return;
-			}
-		}
-
-		datablock = &r->outblock;
-
-		/*
-		 * If PROTO-1: first write out the block header (metadata).
-		 */
-		if (r->gp_proto == 1)
-		{
-			n = datablock->hdr.htop - datablock->hdr.hbot;
-
-			if (n > 0)
-			{
-				n = local_send(r, datablock->hdr.hbyte + datablock->hdr.hbot, n);
-				if (n < 0)
-				{
-					if (errno == EPIPE || errno == ECONNRESET)
-						r->outblock.bot = r->outblock.top;
-					request_end(r, 1, 0);
-					return;
-				}
-
-				TR(("[%d] send header bytes %d .. %d (top %d)\n",
-					fd, datablock->hdr.hbot, datablock->hdr.hbot + n, datablock->hdr.htop));
-
-				datablock->hdr.hbot += n;
-				n = datablock->hdr.htop - datablock->hdr.hbot;
-				if (n > 0)
-					break; /* network chocked */
-			}
-		}
-
-		/*
-		 * write out the block data
-		 */
-		n = datablock->top - datablock->bot;
-		n = local_send(r, datablock->data + datablock->bot, n);
-		if (n < 0)
-		{
-			/*
-			 * EPIPE (or ECONNRESET some computers) indicates remote socket
-			 * intentionally shut down half of the pipe.  If this was because
-			 * of something like "select ... limit 10;", then it is fine that
-			 * we couldn't transmit all the data--the segment didn't want it
-			 * anyway.  If it is because the segment crashed or something like
-			 * that, hopefully we would find out about that in some other way
-			 * anyway, so it is okay if we don't poison the session.
-			 */
-			if (errno == EPIPE || errno == ECONNRESET)
-				r->outblock.bot = r->outblock.top;
-			request_end(r, 1, 0);
-			return;
-		}
-
-		TR(("[%d] send data bytes off buf %d .. %d (top %d)\n",
-			fd, datablock->bot, datablock->bot + n, datablock->top));
-
-		datablock->bot += n;
-
-		if (datablock->top != datablock->bot)
-		{ /* network chocked */
-			TR(("[%d] network full\n", fd));
-			break;
-		}
-	}
-
-	/* Set up for this routine to be called again */
-	if (setup_write(r))
-		request_end(r, 1, 0);
+void gfile_printf_then_putc_newline(const char* format, ...)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 1, 2)));
+
+static void do_write(int fd, short event, void* arg) {
+  request_t* r = (request_t*)arg;
+  int n, i;
+  block_t* datablock;
+
+  if (fd != r->sock)
+    gfatal(r,
+           "internal error - non matching fd (%d) "
+           "and socket (%d)",
+           fd, r->sock);
+
+  /* Loop at most 3 blocks or until we choke on the socket */
+  for (i = 0; i < 3; i++) {
+    /* get a block (or find a remaining block) */
+    if (r->outblock.top == r->outblock.bot) {
+      const char* ferror = session_get_block(r, &r->outblock, r->line_delim_str,
+                                             r->line_delim_length);
+
+      if (ferror) {
+        request_end(r, 1, ferror);
+        gfile_printf_then_putc_newline("ERROR: %s", ferror);
+        return;
+      }
+      if (!r->outblock.top) {
+        request_end(r, 0, 0);
+        return;
+      }
+    }
+
+    datablock = &r->outblock;
+
+    /*
+     * If PROTO-1: first write out the block header (metadata).
+     */
+    if (r->gp_proto == 1) {
+      n = datablock->hdr.htop - datablock->hdr.hbot;
+
+      if (n > 0) {
+        n = local_send(r, datablock->hdr.hbyte + datablock->hdr.hbot, n);
+        if (n < 0) {
+          if (errno == EPIPE || errno == ECONNRESET)
+            r->outblock.bot = r->outblock.top;
+          request_end(r, 1, "gpfdist send block header failure");
+          return;
+        }
+
+        gdebug(r, "send header bytes %d .. %d (top %d)", datablock->hdr.hbot,
+               datablock->hdr.hbot + n, datablock->hdr.htop);
+
+        datablock->hdr.hbot += n;
+        n = datablock->hdr.htop - datablock->hdr.hbot;
+        if (n > 0) break; /* network chocked */
+      }
+    }
+
+    /*
+     * write out the block data
+     */
+    n = datablock->top - datablock->bot;
+    n = local_send(r, datablock->data + datablock->bot, n);
+    if (n < 0) {
+      /*
+       * EPIPE (or ECONNRESET some computers) indicates remote socket
+       * intentionally shut down half of the pipe.  If this was because
+       * of something like "select ... limit 10;", then it is fine that
+       * we couldn't transmit all the data--the segment didn't want it
+       * anyway.  If it is because the segment crashed or something like
+       * that, hopefully we would find out about that in some other way
+       * anyway, so it is okay if we don't poison the session.
+       */
+      if (errno == EPIPE || errno == ECONNRESET)
+        r->outblock.bot = r->outblock.top;
+      request_end(r, 1, "gpfdist send data failure");
+      return;
+    }
+
+    gdebug(r, "send data bytes off buf %d .. %d (top %d)", datablock->bot,
+           datablock->bot + n, datablock->top);
+
+    r->bytes += n;
+    r->last = apr_time_now();
+    datablock->bot += n;
+
+    if (datablock->top != datablock->bot) { /* network chocked */
+      gdebug(r, "network full");
+      break;
+    }
+  }
+
+  /* Set up for this routine to be called again */
+  if (setup_write(r)) request_end(r, 1, 0);
+}
+
+/*
+ * Log request header
+ */
+static void log_request_header(const request_t* r) {
+  int i;
+
+  if (opt.s) {
+    return;
+  }
+
+  /* Hurray, got a request !!! */
+  gprintln(r, "%s requests %s", r->peer,
+           r->in.req->argv[1] ? r->in.req->argv[1] : "(none)");
+
+  /* print the complete request to the log if in verbose mode */
+  gprintln(r, "got a request at port %d:", r->port);
+  for (i = 0; i < r->in.req->argc; i++) printf(" %s", r->in.req->argv[i]);
+  printf("\n");
+
+  gprintln(r, "request headers:");
+  for (i = 0; i < r->in.req->hc; i++)
+    gprintln(r, "%s:%s", r->in.req->hname[i], r->in.req->hvalue[i]);
 }
 
 /*
@@ -1545,301 +1808,302 @@ static void do_write(int fd, short event, void* arg)
  * Callback when a socket is ready to be read. Read the
  * socket for a complete HTTP request.
  */
-static void do_read_request(int fd, short event, void* arg)
-{
-	request_t* 	r = (request_t*) arg;
-	int 		n, i;
-	char*		p = NULL;
-	char*		pp = NULL;
-	char*		path = NULL;
-
-	/* If we timeout, close the request. */
-	if (event & EV_TIMEOUT)
-	{
-		http_error(r, FDIST_BAD_REQUEST, "time out");
-		request_end(r, 1, 0);
-		return;
-	}
-
-	/* Execute only once */
-	if (opt.ssl && !r->io && !r->ssl_bio)
-	{
-		r->io = BIO_new(BIO_f_buffer());
-		r->ssl_bio = BIO_new(BIO_f_ssl());
-		BIO_set_ssl(r->ssl_bio, r->ssl, BIO_CLOSE);
-		BIO_push(r->io, r->ssl_bio);
-
-		/* Set the renegotiate timeout in seconds. 	*/
-		/* When the renegotiate timeout elapses the */
-		/* session is automatically renegotiated	*/
-		BIO_set_ssl_renegotiate_timeout(r->ssl_bio, SSL_RENEGOTIATE_TIMEOUT_SEC);
-	}
-
-	/* how many bytes left in the header buf */
-	n = r->in.hbufmax - r->in.hbuftop;
-	if (n <= 0)
-	{
-		http_error(r, FDIST_INTERNAL_ERROR, "internal error");
-		request_end(r, 1, 0);
-		return;
-	}
-
-	/* read into header buf */
-	n = gpfdist_receive(r, r->in.hbuf + r->in.hbuftop, n);
-
-	if (n < 0)
-	{
+static void do_read_request(int fd, short event, void* arg) {
+  request_t* r = (request_t*)arg;
+  char* p = NULL;
+  char* pp = NULL;
+  char* path = NULL;
+
+  /* If we timeout, close the request. */
+  if (event & EV_TIMEOUT) {
+    gwarning(r, "do_read_request time out");
+    http_error(r, FDIST_TIMEOUT, "time out");
+    request_end(r, 1, 0);
+    return;
+  }
+
+#ifdef USE_SSL
+  /* Execute only once */
+  if (opt.ssl && !r->io && !r->ssl_bio) {
+    r->io = BIO_new(BIO_f_buffer());
+    r->ssl_bio = BIO_new(BIO_f_ssl());
+    BIO_set_ssl(r->ssl_bio, r->ssl, BIO_CLOSE);
+    BIO_push(r->io, r->ssl_bio);
+
+    /* Set the renegotiate timeout in seconds. 	*/
+    /* When the renegotiate timeout elapses the */
+    /* session is automatically renegotiated	*/
+    BIO_set_ssl_renegotiate_timeout(r->ssl_bio, SSL_RENEGOTIATE_TIMEOUT_SEC);
+  }
+#endif
+
+  /* how many bytes left in the header buf */
+  int n = r->in.hbufmax - r->in.hbuftop;
+  if (n <= 0) {
+    gwarning(r, "do_read_request internal error. max: %d, top: %d",
+             r->in.hbufmax, r->in.hbuftop);
+    http_error(r, FDIST_INTERNAL_ERROR, "internal error");
+    request_end(r, 1, 0);
+    return;
+  }
+
+  /* read into header buf */
+  n = gpfdist_receive(r, r->in.hbuf + r->in.hbuftop, n);
+
+  if (n < 0) {
 #ifdef WIN32
-		int e = WSAGetLastError();
-		int ok = (e == WSAEINTR || e == WSAEWOULDBLOCK);
+    int e = WSAGetLastError();
+    int ok = (e == WSAEINTR || e == WSAEWOULDBLOCK);
 #else
-		int e = errno;
-		int ok = (e == EINTR || e == EAGAIN);
-#endif
-		if (!ok)
-		{
-			request_end(r, 1, 0);
-			return;
-		}
-	}
-	else if (n == 0)
-	{
-		/* socket close by peer will return 0 */
-		request_end(r, 1, 0);
-		return;
-	}
-	else
-	{
-		/* check if a complete HTTP request is available in header buf */
-		r->in.hbuftop += n;
-		n = r->in.hbuftop;
-		r->in.req = gnet_parse_request(r->in.hbuf, &n, r->pool);
-		if (!r->in.req && r->in.hbuftop >= r->in.hbufmax)
-		{
-			/* not available, but headerbuf is full - send error and close */
-			http_error(r, FDIST_BAD_REQUEST, "forbidden");
-			request_end(r, 1, 0);
-			return;
-		}
-	}
-
-	/*
-	 * if we don't yet have a complete request, set up this function to be
-	 * called again for
-	 */
-	if (!r->in.req)
-	{
-		if (setup_read(r))
-		{
-			http_error(r, FDIST_INTERNAL_ERROR, "internal error");
-			request_end(r, 1, 0);
-		}
-		return;
-	}
-
-	/* Hurray, got a request !!! */
-	gprint("%s requests %s\n", r->peer, r->in.req->argv[1] ? r->in.req->argv[1]
-			: "(none)");
-
-	/* print the complete request to the log if in verbose mode */
-	if (opt.V)
-	{
-		gprint("[%d] got a request:", r->sock);
-		for (i = 0; i < r->in.req->argc; i++)
-			TR((" %s", r->in.req->argv[i]));
-		TR(("\n"));
-		gprint("request headers:");
-		for (i = 0; i < r->in.req->hc; i++)
-			TR(("%s:%s\n", r->in.req->hname[i], r->in.req->hvalue[i] ));
-	}
-
-	/* check that the request is validly formatted */
-	if(request_validate(r))
-		return;
-
-	/* mark it as a GET or PUT request */
-	if (0 == strcmp("GET", r->in.req->argv[0]))
-		r->is_get = 1;
-
-	/* if GET, we don't need to read from the socket anymore */
-	if (r->is_get)
-		shutdown(r->sock, SHUT_RD);
-
-
-	/* make a copy of the path */
-	path = apr_pstrdup(r->pool, r->in.req->argv[1]);
-
-	/* decode %xx to char */
-	percent_encoding_to_char(p, pp, path);
-
-	/*
-	 * This is a debug hook. We'll get here By creating an external table with
-	 * name(a text) location('gpfdist://<host>:<port>/gpfdist/status').
-	 * Show some state of gpfdist (num sessions, num bytes).
-	 */
-	if (!strcmp(path, "/gpfdist/status"))
-	{
-		send_gpfdist_status(r);
-		request_end(r, 0, 0);
-		return;
-	}
-
-	/*
-	 * set up the requested path
-	 */
-	if (opt.f)
-	{
-		/* we forced in a filename with the hidden -f option. use it */
-		r->path = opt.f;
-	}
-	else
-	{
-		if(request_set_path(r, opt.d, p, pp, path) != 0)
-			return;
-	}
-
-	/* parse gp variables from the request */
-	if(request_parse_gp_headers(r, opt.g) != 0)
-		return;
+    int e = errno;
+    int ok = (e == EINTR || e == EAGAIN);
+#endif
+    gwarning(r, "do_read_request receive failed. errno: %d, msg: %s", errno,
+             strerror(errno));
+    if (!ok) {
+      request_end(r, 1, 0);
+      return;
+    }
+  } else if (n == 0) {
+    /* socket close by peer will return 0 */
+    gwarning(r,
+             "do_read_request receive failed. socket closed by peer. errno: "
+             "%d, msg: %s",
+             errno, strerror(errno));
+    request_end(r, 1, 0);
+    return;
+  } else {
+    /* check if a complete HTTP request is available in header buf */
+    r->in.hbuftop += n;
+    n = r->in.hbuftop;
+    r->in.req = gnet_parse_request(r->in.hbuf, &n, r->pool);
+    if (!r->in.req && r->in.hbuftop >= r->in.hbufmax) {
+      /* not available, but headerbuf is full - send error and close */
+      gwarning(r, "do_read_request bad request");
+      http_error(r, FDIST_BAD_REQUEST, "forbidden");
+      request_end(r, 1, 0);
+      return;
+    }
+  }
+
+  /*
+   * if we don't yet have a complete request, set up this function to be
+   * called again for
+   */
+  if (!r->in.req) {
+    if (setup_read(r)) {
+      gwarning(r, "do_read_request, failed to read a complete request");
+      http_error(r, FDIST_INTERNAL_ERROR, "internal error");
+      request_end(r, 1, 0);
+    }
+    return;
+  }
+
+  /* check that the request is validly formatted */
+  if (request_validate(r)) {
+    log_request_header(r);
+    return;
+  }
+
+  /* mark it as a GET or PUT request */
+  if (0 == strcmp("GET", r->in.req->argv[0])) r->is_get = 1;
+
+  if (r->is_get || opt.V) log_request_header(r);
+
+  /* make a copy of the path */
+  path = apr_pstrdup(r->pool, r->in.req->argv[1]);
+
+  /* decode %xx to char */
+  percent_encoding_to_char(p, pp, path);
+
+  /*
+   * This is a debug hook. We'll get here By creating an external table with
+   * name(a text) location('gpfdist://<host>:<port>/gpfdist/status').
+   * Show some state of gpfdist (num sessions, num bytes).
+   */
+  if (!strcmp(path, "/gpfdist/status")) {
+    send_gpfdist_status(r);
+    request_end(r, 0, 0);
+    return;
+  }
+
+  /*
+   * set up the requested path
+   */
+  if (opt.f) {
+    /* we forced in a filename with the hidden -f option. use it */
+    r->path = opt.f;
+  } else {
+    if (request_set_path(r, opt.d, p, pp, path) != 0) return;
+  }
+
+  /* parse gp variables from the request */
+  if (request_parse_gp_headers(r, opt.g) != 0) return;
 
 #ifdef GPFXDIST
-	/* setup transform */
-	if(request_set_transform(r) != 0)
-		return;
+  /* setup transform */
+  if (request_set_transform(r) != 0) return;
 #endif
 
-	/* Attach the request to a session */
-	if(session_attach(r) != 0)
-		return;
+  /* Attach the request to a session */
+  if (session_attach(r) != 0) return;
 
-	if (r->is_get)
-	{
-		/* handle GET */
-		handle_get_request(r);
-	}
-	else
-	{
-		/* handle PUT */
-		handle_post_request(r, n);
-	}
+  if (r->is_get) {
+    /* handle GET */
+    handle_get_request(r);
+  } else {
+    /* handle PUT */
+    handle_post_request(r, n);
+  }
 }
 
-
 /* Callback when the listen socket is ready to accept connections. */
-static void do_accept(int fd, short event, void* arg)
-{
-	struct sockaddr_storage 	a;
-	socklen_t 			len = sizeof(a);
-	SOCKET 				sock;
-	request_t* 			r;
-	apr_pool_t* 		pool;
-	int 				on = 1;
-	struct linger		linger;
-	BIO 				*sbio = NULL;	/* only for SSL */
-	SSL 				*ssl = NULL;	/* only for SSL */
-	int 				rd;				/* only for SSL */
-
-	/* do the accept */
-	if ((sock = accept(fd, (struct sockaddr*) &a, &len)) < 0)
-	{
-		gwarning(FLINE, "accept failed");
-		goto failure;
-	}
-	if (opt.ssl)
-	{
-		sbio = BIO_new_socket(sock, BIO_NOCLOSE);
-		ssl = SSL_new(gcb.server_ctx);
-		SSL_set_bio(ssl, sbio, sbio);
-		if ( (rd = SSL_accept(ssl) <= 0) )
-		{
-			handle_ssl_error(sock, sbio, ssl);
-			/* Close the socket that was allocated by accept 			*/
-			/* We also must perform this, in case that a user 			*/
-			/* accidentaly connected via gpfdist, instead of gpfdits	*/
-			closesocket(sock);
-			return;
-		}
-
-		gprint("[%d] Using SSL\n", (int)sock);
-	}
-	/* set to non-blocking, and close-on-exec */
+static void do_accept(int fd, short event, void* arg) {
+  address_t a;
+  socklen_t len = sizeof(a);
+  SOCKET sock;
+  request_t* r;
+  apr_pool_t* pool;
+  int on = 1;
+  struct linger linger;
+
+#ifdef USE_SSL
+  BIO* sbio = NULL; /* only for SSL */
+  SSL* ssl = NULL;  /* only for SSL */
+  int rd;           /* only for SSL */
+#endif
+
+  /* do the accept */
+  if ((sock = accept(fd, (struct sockaddr*)&a, &len)) < 0) {
+    gwarning(NULL, "accept failed");
+    goto failure;
+  }
+
+#ifdef USE_SSL
+  if (opt.ssl) {
+    sbio = BIO_new_socket(sock, BIO_NOCLOSE);
+    ssl = SSL_new(gcb.server_ctx);
+    SSL_set_bio(ssl, sbio, sbio);
+    if ((rd = SSL_accept(ssl) <= 0)) {
+      handle_ssl_error(sock, sbio, ssl);
+      /* Close the socket that was allocated by accept 			*/
+      /* We also must perform this, in case that a user */
+      /* accidentaly connected via gpfdist, instead of gpfdits	*/
+      closesocket(sock);
+      return;
+    }
+
+    gprint(NULL, "[%d] Using SSL\n", (int)sock);
+  }
+#endif
+
+  /* set to non-blocking, and close-on-exec */
 #ifdef WIN32
-	{
-		unsigned long nonblocking = 1;
-		ioctlsocket(sock, FIONBIO, &nonblocking);
-	}
+  {
+    unsigned long nonblocking = 1;
+    ioctlsocket(sock, FIONBIO, &nonblocking);
+  }
 #else
-	if (fcntl(sock, F_SETFL, O_NONBLOCK) == -1)
-	{
-		gwarning(FLINE, "fcntl(F_SETFL, O_NONBLOCK) failed");
-		if ( opt.ssl )
-		{
-			handle_ssl_error(sock, sbio, ssl);
-		}
-		closesocket(sock);
-		goto failure;
-	}
-	if (fcntl(sock, F_SETFD, 1) == -1)
-	{
-		gwarning(FLINE, "fcntl(F_SETFD) failed");
-		if ( opt.ssl )
-		{
-			handle_ssl_error(sock, sbio, ssl);
-		}
-		closesocket(sock);
-		goto failure;
-	}
-#endif
-
-	/* set keepalive, reuseaddr, and linger */
-	setsockopt(sock, SOL_SOCKET, SO_KEEPALIVE, (void*) &on, sizeof(on));
-	setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, (void*) &on, sizeof(on));
-	linger.l_onoff = 1;
-	linger.l_linger = 5;
-	setsockopt(sock, SOL_SOCKET, SO_LINGER, (void*) &linger, sizeof(linger));
-
-	/* create a pool container for this socket */
-	if (apr_pool_create(&pool, gcb.pool))
-		gfatal(FLINE, "out of memory in do_accept");
-
-	/* create the request in pool */
-	r = apr_pcalloc(pool, sizeof(request_t));
-	r->pool = pool;
-	r->sock = sock;
-
-    r->outblock.data = apr_palloc(pool, opt.m); /* use the block size specified by -m option */
-
-	r->line_delim_str = "";
-	r->line_delim_length = -1;
-
-	r->in.hbufmax = 1024 * 4; /* 4K for reading the headers */
-	r->in.hbuf = apr_palloc(pool, r->in.hbufmax);
-	r->is_final = 0;	/* initialize */
-	r->ssl = ssl;
-	r->sbio = sbio;
-
-	{
-		char host[128];
-		getnameinfo((struct sockaddr *)&a, len, host, sizeof(host), NULL, 0, NI_NUMERICHOST
-#ifdef NI_NUMERICSERV
-				| NI_NUMERICSERV
+  if (fcntl(sock, F_SETFL, O_NONBLOCK) == -1) {
+    gwarning(NULL, "fcntl(F_SETFL, O_NONBLOCK) failed");
+#ifdef USE_SSL
+    if (opt.ssl) {
+      handle_ssl_error(sock, sbio, ssl);
+    }
+#endif
+    closesocket(sock);
+    goto failure;
+  }
+  if (fcntl(sock, F_SETFD, 1) == -1) {
+    gwarning(NULL, "fcntl(F_SETFD) failed");
+#ifdef USE_SSL
+    if (opt.ssl) {
+      handle_ssl_error(sock, sbio, ssl);
+    }
+#endif
+    closesocket(sock);
+    goto failure;
+  }
+#endif
+
+  /* set keepalive, reuseaddr, and linger */
+  if (setsockopt(sock, SOL_SOCKET, SO_KEEPALIVE, (void*)&on, sizeof(on)) ==
+      -1) {
+    gwarning(NULL, "Setting SO_KEEPALIVE failed");
+    closesocket(sock);
+    goto failure;
+  }
+  if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, (void*)&on, sizeof(on)) ==
+      -1) {
+    gwarning(NULL, "Setting SO_REUSEADDR on socket failed");
+    closesocket(sock);
+    goto failure;
+  }
+  linger.l_onoff = 1;
+  linger.l_linger = 10;
+  if (setsockopt(sock, SOL_SOCKET, SO_LINGER, (void*)&linger, sizeof(linger)) ==
+      -1) {
+    gwarning(NULL, "Setting SO_LINGER on socket failed");
+    closesocket(sock);
+    goto failure;
+  }
+
+  /* create a pool container for this socket */
+  if (apr_pool_create(&pool, gcb.pool))
+    gfatal(NULL, "out of memory in do_accept");
+
+  /* create the request in pool */
+  r = pcalloc_safe(NULL, pool, sizeof(request_t),
+                   "failed to allocated request_t: %d bytes",
+                   (int)sizeof(request_t));
+
+  r->port = ntohs(get_client_port((address_t*)&a));
+  r->id = ++REQUEST_SEQ;
+  r->pool = pool;
+  r->sock = sock;
+
+  event_set(&r->ev, 0, 0, 0, 0);
+
+  /* use the block size specified by -m option */
+  r->outblock.data = palloc_safe(
+      r, pool, opt.m, "out of memory when allocating buffer: %d bytes", opt.m);
+
+  r->line_delim_str = "";
+  r->line_delim_length = -1;
+
+  r->in.hbufmax = 1024 * 4; /* 4K for reading the headers */
+  r->in.hbuf = palloc_safe(r, pool, r->in.hbufmax,
+                           "out of memory when allocating r->in.hbuf: %d",
+                           r->in.hbufmax);
+
+  r->is_final = 0; /* initialize */
+#ifdef USE_SSL
+  r->ssl = ssl;
+  r->sbio = sbio;
 #endif
-				);
-		r->peer = apr_psprintf(r->pool, "%s", host);
-	}
 
+  {
+    char host[128];
+    getnameinfo((struct sockaddr*)&a, len, host, sizeof(host), NULL, 0,
+                NI_NUMERICHOST
+#ifdef NI_NUMERICSERV
+                    | NI_NUMERICSERV
+#endif
+    );
+    r->peer = apr_psprintf(r->pool, "%s", host);
+  }
 
-	/* set up for callback when socket ready for reading the http request */
-	if (setup_read(r))
-	{
-		http_error(r, FDIST_INTERNAL_ERROR, "internal error");
-		request_end(r, 1, 0);
-	}
+  /* set up for callback when socket ready for reading the http request */
+  if (setup_read(r)) {
+    http_error(r, FDIST_INTERNAL_ERROR, "internal error");
+    request_end(r, 1, 0);
+  }
 
-	return;
+  return;
 
-failure: gwarning(FLINE, "accept failed");
-	return;
+failure:
+  gwarning(NULL, "accept failed");
+  return;
 }
 
 /*
@@ -1848,16 +2112,14 @@ failure: gwarning(FLINE, "accept failed");
  * setup the write event to write data to the socket. It uses
  * the callback function 'do_write'.
  */
-static int setup_write(request_t* r)
-{
-	if (r->sock < 0)
-		gfatal(FLINE, "internal error in setup_write - no socket to use");
-	event_del(&r->ev);
-	event_set(&r->ev, r->sock, EV_WRITE, do_write, r);
-	return (event_add(&r->ev, 0));
+static int setup_write(request_t* r) {
+  if (r->sock < 0)
+    gwarning(r, "internal error in setup_write - no socket to use");
+  event_del(&r->ev);
+  event_set(&r->ev, r->sock, EV_WRITE, do_write, r);
+  return (event_add(&r->ev, 0));
 }
 
-
 /*
  * setup_read
  *
@@ -1871,836 +2133,936 @@ static int setup_write(request_t* r)
  * -- if set to true, use the callback function 'do_read_request'.
  * -- if set to false, use the callback function 'do_read_body'.
  */
-static int setup_read(request_t* r)
-{
-	if (r->sock < 0)
-		gfatal(FLINE, "internal error in setup_read - no socket to use");
-
-	event_del(&r->ev);
-	event_set(&r->ev, r->sock, EV_READ, do_read_request, r);
-
-	if(opt.t == 0)
-	{
-		return (event_add(&r->ev, NULL)); /* no timeout */
-	}
-	else
-	{
-		r->tm.tv_sec = opt.t;
-		r->tm.tv_usec = 0;
-		return (event_add(&r->ev, &r->tm));
-	}
-}
-
-static void
-print_listening_address(struct addrinfo *rp)
-{
-    char full_address[220] = {0};
-	
-	if (rp->ai_family == AF_INET)
-	{
+static int setup_read(request_t* r) {
+  if (r->sock < 0)
+    gwarning(r, "internal error in setup_read - no socket to use");
+
+  event_del(&r->ev);
+  event_set(&r->ev, r->sock, EV_READ, do_read_request, r);
+
+  if (opt.t == 0) {
+    return (event_add(&r->ev, NULL)); /* no timeout */
+  } else {
+    r->tm.tv_sec = opt.t;
+    r->tm.tv_usec = 0;
+    return (event_add(&r->ev, &r->tm));
+  }
+}
+
+static void print_listening_address(struct addrinfo* rp) {
+  char full_address[220] = {0};
+
+  if (rp->ai_family == AF_INET) {
 #ifndef WIN32
-		struct sockaddr_in *ain = (struct sockaddr_in*)rp->ai_addr;
-		char stradd[200] = {0};
-		inet_ntop(AF_INET, (const void*)&ain->sin_addr, stradd, 100);
-		sprintf(full_address, "IPV4 socket: %s:%d", stradd, opt.p);
+    struct sockaddr_in* ain = (struct sockaddr_in*)rp->ai_addr;
+    char stradd[200] = {0};
+    inet_ntop(AF_INET, (const void*)&ain->sin_addr, stradd, 100);
+    sprintf(full_address, "IPV4 socket: %s:%d", stradd, opt.p);
 #else
-		/*
-		 * there is no alternative for inet_ntop in windows that works for all Win platforms
-		 * and for IPV6. inet_ntop transform the integer representation of the IP addr. into a string 
-	     */
-		sprintf(full_address, "IPV4 socket: IPv4:%d", opt.p);
+    /*
+     * there is no alternative for inet_ntop in windows that works for all Win
+     * platforms and for IPV6. inet_ntop transform the integer representation of
+     * the IP addr. into a string
+     */
+    sprintf(full_address, "IPV4 socket: IPv4:%d", opt.p);
 #endif
 
-	}
-	else if (rp->ai_family == AF_INET6)
-	{
+  } else if (rp->ai_family == AF_INET6) {
 #ifndef WIN32
-		struct sockaddr_in6 *ain = (struct sockaddr_in6*)rp->ai_addr;
-		char stradd[200] = {0};
-		inet_ntop(AF_INET6, (const void*)&ain->sin6_addr, stradd, 100);
-		sprintf(full_address, "IPV6 socket: [%s]:%d", stradd, opt.p);
+    struct sockaddr_in6* ain = (struct sockaddr_in6*)rp->ai_addr;
+    char stradd[200] = {0};
+    inet_ntop(AF_INET6, (const void*)&ain->sin6_addr, stradd, 100);
+    sprintf(full_address, "IPV6 socket: [%s]:%d", stradd, opt.p);
 #else
-		sprintf(full_address, "IPV6 socket: [IPV6]:%d", opt.p);
+    sprintf(full_address, "IPV6 socket: [IPV6]:%d", opt.p);
 #endif
 
-	}
-	else 
-	{
-		sprintf(full_address, "unknown protocol - %d", rp->ai_family);
-	}
-	
-	gprint("%s\n", full_address);
+  } else {
+    sprintf(full_address, "unknown protocol - %d", rp->ai_family);
+  }
+
+  gprint(NULL, "%s\n", full_address);
 }
 
 /*
  * Search linked list (head) for first element with family (first_family).
  * Moves first matching element to head of the list.
  */
-static struct
-addrinfo* rearrange_addrs(struct addrinfo *head, int first_family)
-{
-	struct addrinfo* iter;
-	struct addrinfo* new_head = head;
-	struct addrinfo* holder = NULL;
-
-	if (head->ai_family == first_family)
-		return head;
-
-	for (iter = head; iter != NULL && iter->ai_next != NULL; iter = iter->ai_next)
-	{
-		if ( iter->ai_next->ai_family == first_family )
-		{
-			holder = iter->ai_next;
-			iter->ai_next = iter->ai_next->ai_next;
-			/*
-			 * we don't break here since if there are more addrinfo structure that belong to first_family
-			 * in the list, we want to remove them all and keep only one in the holder.
-			 * and then we will put the holder in the front
-			 */
-		}
-	}
-
-	if ( holder != NULL )
-	{
-		holder->ai_next = new_head;
-		new_head = holder;
-	}
-
-	return new_head;
-}
-
-
-static void
-print_addrinfo_list(struct addrinfo *head)
-{
-	struct addrinfo *iter;
-	for (iter = head; iter != NULL; iter = iter->ai_next)
-	{ 
-		print_listening_address(iter);
-	}
+static struct addrinfo* rearrange_addrs(struct addrinfo* head,
+                                        int first_family) {
+  struct addrinfo* iter;
+  struct addrinfo* new_head = head;
+  struct addrinfo* holder = NULL;
+
+  if (head->ai_family == first_family) return head;
+
+  for (iter = head; iter != NULL && iter->ai_next != NULL;
+       iter = iter->ai_next) {
+    if (iter->ai_next->ai_family == first_family) {
+      holder = iter->ai_next;
+      iter->ai_next = iter->ai_next->ai_next;
+      /*
+       * we don't break here since if there are more addrinfo structure that
+       * belong to first_family in the list, we want to remove them all and keep
+       * only one in the holder. and then we will put the holder in the front
+       */
+    }
+  }
+
+  if (holder != NULL) {
+    holder->ai_next = new_head;
+    new_head = holder;
+  }
+
+  return new_head;
+}
+
+static void print_addrinfo_list(struct addrinfo* head) {
+  struct addrinfo* iter;
+  for (iter = head; iter != NULL; iter = iter->ai_next) {
+    print_listening_address(iter);
+  }
+}
+
+static void signal_register() {
+  /* when SIGTERM raised invoke process_term_signal */
+  signal_set(&gcb.signal_event, SIGTERM, process_term_signal, 0);
+
+  /* high priority so we accept as fast as possible */
+  if (event_priority_set(&gcb.signal_event, 0))
+    gwarning(NULL, "signal event priority set failed");
+
+  /* start watching this event */
+  if (signal_add(&gcb.signal_event, 0))
+    gfatal(NULL, "cannot set up event on signal register");
 }
 
 /* Create HTTP port and start to receive request */
-static void
-http_setup(void)
-{
-	SOCKET f;
-	int on = 1;
-	struct linger linger;
-	struct addrinfo hints;
-	struct addrinfo *addrs, *rp;
-	int  s;
-	int  i;
-
-	char service[32];
-	const char *hostaddr = NULL;
-
-	if (opt.ssl)
-	{
-		/* Build our SSL context*/
-		gcb.server_ctx 	= initialize_ctx();
-		gpfdist_send 	= gpfdist_SSL_send;
-		gpfdist_receive = gpfdist_SSL_receive;
-	}
-	else
-	{
-		gcb.server_ctx 	= NULL;
-		gpfdist_send 	= gpfdist_socket_send;
-		gpfdist_receive = gpfdist_socket_receive;
-	}
-
-	gcb.listen_sock_count = 0;
-	if (opt.b != NULL && strlen(opt.b) > 1)
-		hostaddr = opt.b;
-
-	/* setup event priority */
-	if (event_priority_init(10))
-		gwarning(FLINE, "event_priority_init failed");
-
-
-	/* Try each possible port from opt.p to opt.last_port */
-	for (;;)
-	{
-		snprintf(service,32,"%d",opt.p);
-		memset(&hints, 0, sizeof(struct addrinfo));
-		hints.ai_family = AF_UNSPEC;	/* Allow IPv4 or IPv6 */
-		hints.ai_socktype = SOCK_STREAM; /* tcp socket */
-		hints.ai_flags = AI_PASSIVE;	/* For wildcard IP address */
-		hints.ai_protocol = 0;			/* Any protocol */
-
-		s = getaddrinfo(hostaddr, service, &hints, &addrs);
-		if (s != 0)
+static void http_setup(void) {
+  SOCKET f;
+  int on = 1;
+  struct linger linger;
+  struct addrinfo hints;
+  struct addrinfo *addrs, *rp;
+  int s;
+  int i;
+
+  char service[32];
+  const char* hostaddr = NULL;
+
+#ifdef USE_SSL
+  if (opt.ssl) {
+    /* Build our SSL context*/
+    gcb.server_ctx = initialize_ctx();
+    gpfdist_send = gpfdist_SSL_send;
+    gpfdist_receive = gpfdist_SSL_receive;
+  } else {
+    gcb.server_ctx = NULL;
+    gpfdist_send = gpfdist_socket_send;
+    gpfdist_receive = gpfdist_socket_receive;
+  }
+#else
+  gpfdist_send = gpfdist_socket_send;
+  gpfdist_receive = gpfdist_socket_receive;
+#endif
+
+  gcb.listen_sock_count = 0;
+  if (opt.b != NULL && strlen(opt.b) > 1) hostaddr = opt.b;
+
+  /* setup event priority */
+  if (event_priority_init(10)) gwarning(NULL, "event_priority_init failed");
+
+  /* Try each possible port from opt.p to opt.last_port */
+  for (;;) {
+    snprintf(service, 32, "%d", opt.p);
+    memset(&hints, 0, sizeof(struct addrinfo));
+    hints.ai_family = AF_UNSPEC;     /* Allow IPv4 or IPv6 */
+    hints.ai_socktype = SOCK_STREAM; /* tcp socket */
+    hints.ai_flags = AI_PASSIVE;     /* For wildcard IP address */
+    hints.ai_protocol = 0;           /* Any protocol */
+
+    s = getaddrinfo(hostaddr, service, &hints, &addrs);
+    if (s != 0)
 #if (!defined(WIN32)) || defined(gai_strerror)
-			gfatal(FLINE,"getaddrinfo says %s",gai_strerror(s));
+      gfatal(NULL, "getaddrinfo says %s", gai_strerror(s));
 #else
-			/* Broken mingw header file from old version of mingw doesn't have gai_strerror */
-			gfatal(FLINE,"getaddrinfo says %d",s);
-#endif
-
-		addrs = rearrange_addrs(addrs, AF_INET6);
-
-		gprint("Before opening listening sockets - following listening sockets are available:\n");
-		print_addrinfo_list(addrs);
-		
-		/*
-		 * getaddrinfo() returns a list of address structures,
-		 * one for each valid address and family we can use.
-		 *
-		 * Try each address until we successfully bind.
-		 * If socket (or bind) fails, we (close the socket
-		 * and) try the next address.  This can happen if
-		 * the system supports IPv6, but IPv6 is disabled from
-		 * working, or if it supports IPv6 and IPv4 is disabled.
-		 */
-		for (rp = addrs; rp != NULL; rp = rp->ai_next)
-		{
-			gprint("Trying to open listening socket:\n");
-			print_listening_address(rp);
-			
-			/*
-			 * getaddrinfo gives us all the parameters for the socket() call
-			 * as well as the parameters for the bind() call.
-			 */
-			f = socket(rp->ai_family, rp->ai_socktype, rp->ai_protocol);
-
-			if (f == -1)
-			{
-				gwarning(FLINE, "Creating the socket failed\n");
-				continue;
-			}
-			
+      /* Broken mingw header file from old version of mingw doesn't have
+       * gai_strerror */
+      gfatal(NULL, "getaddrinfo says %d", s);
+#endif
+
+    addrs = rearrange_addrs(addrs, AF_INET6);
+
+    gprint(NULL,
+           "Before opening listening sockets - following listening sockets are "
+           "available:\n");
+    print_addrinfo_list(addrs);
+
+    /*
+     * getaddrinfo() returns a list of address structures,
+     * one for each valid address and family we can use.
+     *
+     * Try each address until we successfully bind.
+     * If socket (or bind) fails, we (close the socket
+     * and) try the next address.  This can happen if
+     * the system supports IPv6, but IPv6 is disabled from
+     * working, or if it supports IPv6 and IPv4 is disabled.
+     */
+    for (rp = addrs; rp != NULL; rp = rp->ai_next) {
+      gprint(NULL, "Trying to open listening socket:\n");
+      print_listening_address(rp);
+
+      /*
+       * getaddrinfo gives us all the parameters for the socket() call
+       * as well as the parameters for the bind() call.
+       */
+      f = socket(rp->ai_family, rp->ai_socktype, rp->ai_protocol);
+
+      if (f == -1) {
+        gwarning(NULL, "Creating the socket failed\n");
+        continue;
+      }
+
 #ifndef WIN32
-			if (fcntl(f, F_SETFD, 1) == -1)
-				gfatal(FLINE, "cannot create socket - fcntl(F_SETFD) failed");
+      if (fcntl(f, F_SETFD, 1) == -1)
+        gfatal(NULL, "cannot create socket - fcntl(F_SETFD) failed");
 
-			/* For the Windows case, we could use SetHandleInformation to remove
-			 the HANDLE_INHERIT property from fd.
-			 But for our purposes this does not matter,
-			 as by default handles are *not* inherited. */
+        /* For the Windows case, we could use SetHandleInformation to remove
+         the HANDLE_INHERIT property from fd.
+         But for our purposes this does not matter,
+         as by default handles are *not* inherited. */
 
 #endif
-			setsockopt(f, SOL_SOCKET, SO_KEEPALIVE, (void*) &on, sizeof(on));
-
-			/*
-			 * We cannot use SO_REUSEADDR on win32 because it results in different
-			 * behaviour -- it allows multiple servers to bind to the same port,
-			 * resulting in totally unpredictable behaviour. What a silly operating
-			 * system.
-			 */
+      if (setsockopt(f, SOL_SOCKET, SO_KEEPALIVE, (void*)&on, sizeof(on)) ==
+          -1) {
+        closesocket(f);
+        gwarning(NULL, "Setting SO_KEEPALIVE on socket failed");
+        continue;
+      }
+
+      /*
+       * We cannot use SO_REUSEADDR on win32 because it results in different
+       * behaviour -- it allows multiple servers to bind to the same port,
+       * resulting in totally unpredictable behaviour. What a silly operating
+       * system.
+       */
 #ifndef WIN32
-			setsockopt(f, SOL_SOCKET, SO_REUSEADDR, (void*) &on, sizeof(on));
-#endif
-			linger.l_onoff = 1;
-			linger.l_linger = 5;
-			setsockopt(f, SOL_SOCKET, SO_LINGER, (void*) &linger, sizeof(linger));
-
-			if (bind(f, rp->ai_addr, rp->ai_addrlen) != 0)
-			{
-				/*
-				 * EADDRINUSE warning appears only if the -v or -V option is on,
-				 * All the other warnings will appear anyway
-				 * EADDRINUSE is not defined in win32, so all the warnings will always appear.
-				 */
+      if (setsockopt(f, SOL_SOCKET, SO_REUSEADDR, (void*)&on, sizeof(on)) ==
+          -1) {
+        closesocket(f);
+        gwarning(NULL, "Setting SO_REUSEADDR on socket failed");
+        continue;
+      }
+#endif
+      linger.l_onoff = 1;
+      linger.l_linger = 5;
+      if (setsockopt(f, SOL_SOCKET, SO_LINGER, (void*)&linger,
+                     sizeof(linger)) == -1) {
+        closesocket(f);
+        gwarning(NULL, "Setting SO_LINGER on socket failed");
+        continue;
+      }
+
+      if (bind(f, rp->ai_addr, rp->ai_addrlen) != 0) {
+        /*
+         * EADDRINUSE warning appears only if the -v or -V option is on,
+         * All the other warnings will appear anyway
+         * EADDRINUSE is not defined in win32, so all the warnings will always
+         * appear.
+         */
 #ifdef WIN32
-				if ( 1 )
+        if (1)
 #else
-				if ( errno == EADDRINUSE )
-#endif
-				{
-					if ( opt.v )
-					{
-						gwarning(FLINE, "%s (errno = %d), port: %d",
-					               		strerror(errno), errno, opt.p);
-					}
-				}
-				else
-			    {
-					gwarning(FLINE, "%s (errno=%d), port: %d",strerror(errno), errno, opt.p);
-				}
-
-				/* failed on bind, maybe this address family isn't supported */
-				close(f);
-				continue;
-			}
-
-			/* listen with a big queue */
-			if (listen(f, opt.z))
-			{
-				int saved_errno = errno;
-				close(f);
-				gwarning(FLINE, "listen with queue size %d on socket (%d) using port %d failed with error code (%d): %s",
-							  opt.z,
-							  (int)f,
-							  opt.p,
-							  saved_errno,
-							  strerror(saved_errno));
-				continue;
-			}
-			gcb.listen_socks[gcb.listen_sock_count++] = f;
-
-			gprint("Opening listening socket succeeded\n");
-		}
-
-		/* When we get here, we have either succeeded, or tried all address families for this port */
-
-		if (addrs != NULL)
-		{
-			/* don't need this any more */
-			freeaddrinfo(addrs);
-		}
-
-		if (gcb.listen_sock_count > 0)
-			break;
-
-		if (opt.p >= opt.last_port)
-			gfatal(FLINE, "cannot create socket on port %d "
-						  "(last port is %d)", opt.p, opt.last_port);
-
-		opt.p++;
-		if (opt.v)
-			putchar('\n'); /* this is just to beautify the print outs */
-	}
-
-	for (i = 0; i < gcb.listen_sock_count; i++)
-	{
-		/* when this socket is ready, do accept */
-		event_set(&gcb.listen_events[i], gcb.listen_socks[i], EV_READ | EV_PERSIST,
-				  do_accept, 0);
-
-		/* high priority so we accept as fast as possible */
-		if (event_priority_set(&gcb.listen_events[i], 0))
-			gwarning(FLINE, "event_priority_set failed");
-
-		/* start watching this event */
-		if (event_add(&gcb.listen_events[i], 0))
-			gfatal(FLINE, "cannot set up event on listen socket: %s",
-						   strerror(errno));
-	}
-}
-
-void
-process_signal(int sig)
-{
-	if (sig == SIGINT || sig == SIGTERM)
-	{
-		int i;
-		gwarning(FLINE, "signal %d received. gpfdist exits", sig);
-		for (i = 0; i < gcb.listen_sock_count; i++)
-			if (gcb.listen_socks[i] > 0)
-			{
-				closesocket(gcb.listen_socks[i]);
-			}
-		exit(1);
-	}
-}
-
-
-static gnet_request_t*
-gnet_parse_request(const char* buf, int* len, apr_pool_t* pool)
-{
-	int n = *len;
-	int empty, completed;
-	const char* p;
-	char* line;
-	char* last;
-	char* colon;
-	gnet_request_t* req = 0;
-
-	/* find an empty line */
-	*len = 0;
-	empty = 1, completed = 0;
-	for (p = buf; n > 0 && *p; p++, n--)
-	{
-		int ch = *p;
-		/* skip spaces */
-		if (ch == ' ' || ch == '\t' || ch == '\r')
-			continue;
-		if (ch == '\n')
-		{
-			if (!empty)
-			{
-				empty = 1;
-				continue;
-			}
-			p++;
-			completed = 1;
-			break;
-		}
-		empty = 0;
-	}
-	if (!completed)
-		return 0;
-
-	/* we have a complete HTTP-style request (terminated by empty line) */
-	*len = n = p - buf; /* consume it */
-	line = apr_pstrndup(pool, buf, n); /* dup it */
-	req = apr_pcalloc(pool, sizeof(gnet_request_t));
-
-	/* for first line */
-	line = apr_strtok(line, "\n", &last);
-
-	if (!line)
-		line = apr_pstrdup(pool, "");
-
-	line = gstring_trim(line);
-
-	if (0 != apr_tokenize_to_argv(line, &req->argv, pool))
-		return req;
-
-	while (req->argv[req->argc])
-		req->argc++;
-
-	/* for each subsequent lines */
-	while (0 != (line = apr_strtok(0, "\n", &last)))
-	{
-		if (*line == ' ' || *line == '\t')
-		{
-			/* continuation */
-			if (req->hc == 0) /* illegal - missing first header */
-				break;
-
-			line = gstring_trim(line);
-			if (*line == 0) /* empty line */
-				break;
-
-			/* add to previous hvalue */
-			req->hvalue[req->hc - 1] = gstring_trim(apr_pstrcat(pool,
-					req->hvalue[req->hc - 1], " ", line, (char*)NULL));
-			continue;
-		}
-		/* find a colon, and break the line in two */
-		if (!(colon = strchr(line, ':')))
-			colon = line + strlen(line);
-		else
-			*colon++ = 0;
-
-		line = gstring_trim(line);
-		if (*line == 0) /* empty line */
-			break;
-
-		/* save name, value pair */
-		req->hname[req->hc] = line;
-		req->hvalue[req->hc] = gstring_trim(colon);
-		req->hc++;
-
-		if (req->hc >= sizeof(req->hname) / sizeof(req->hname[0]))
-			break;
-	}
-
-	return req;
-}
-
-static char *gstring_trim(char* s)
-{
-	char* p;
-	s += strspn(s, " \t\r\n");
-	for (p = s + strlen(s) - 1; p > s; p--)
-	{
-		if (strchr(" \t\r\n", *p))
-			*p = 0;
-		else
-			break;
-	}
-	return s;
-}
-
-static char* datetime(void)
-{
-	static char 	buf[100];
-	apr_time_exp_t 	t;
-
-	apr_time_exp_lt(&t, apr_time_now());
-
-	sprintf(buf, "%04d-%02d-%02d %02d:%02d:%02d", 1900 + t.tm_year, 1
-			+ t.tm_mon, t.tm_mday, t.tm_hour, t.tm_min, t.tm_sec);
-
-	return buf;
+        if (errno == EADDRINUSE)
+#endif
+        {
+          if (opt.v) {
+            gwarning(NULL, "%s (errno = %d), port: %d", strerror(errno), errno,
+                     opt.p);
+          }
+        } else {
+          gwarning(NULL, "%s (errno=%d), port: %d", strerror(errno), errno,
+                   opt.p);
+        }
+
+        /* failed on bind, maybe this address family isn't supported */
+        closesocket(f);
+        continue;
+      }
+
+      /* listen with a big queue */
+      if (listen(f, opt.z)) {
+        int saved_errno = errno;
+        closesocket(f);
+        gwarning(NULL,
+                 "listen with queue size %d on socket (%d) using port %d "
+                 "failed with error code (%d): %s",
+                 opt.z, (int)f, opt.p, saved_errno, strerror(saved_errno));
+        continue;
+      }
+      gcb.listen_socks[gcb.listen_sock_count++] = f;
+
+      gprint(NULL, "Opening listening socket succeeded\n");
+    }
+
+    /* When we get here, we have either succeeded, or tried all address families
+     * for this port */
+
+    if (addrs != NULL) {
+      /* don't need this any more */
+      freeaddrinfo(addrs);
+    }
+
+    if (gcb.listen_sock_count > 0) break;
+
+    if (opt.p >= opt.last_port)
+      gfatal(NULL,
+             "cannot create socket on port %d "
+             "(last port is %d)",
+             opt.p, opt.last_port);
+
+    opt.p++;
+    if (opt.v) putchar('\n'); /* this is just to beautify the print outs */
+  }
+
+  for (i = 0; i < gcb.listen_sock_count; i++) {
+    /* when this socket is ready, do accept */
+    event_set(&gcb.listen_events[i], gcb.listen_socks[i], EV_READ | EV_PERSIST,
+              do_accept, 0);
+
+    /* only signal process function priority higher than socket handler */
+    if (event_priority_set(&gcb.listen_events[i], 1))
+      gwarning(NULL, "event_priority_set failed");
+
+    /* start watching this event */
+    if (event_add(&gcb.listen_events[i], 0))
+      gfatal(NULL, "cannot set up event on listen socket: %s", strerror(errno));
+  }
 }
 
-int gprint(const char *fmt, ...)
-{
-	va_list ap;
+void process_term_signal(int sig, short event, void* arg) {
+  gwarning(NULL, "signal %d received. gpfdist exits", sig);
+  log_gpfdist_status();
+  fflush(stdout);
 
-	if (opt.v)
-	{
-		va_start(ap, fmt);
-		printf("[%s] ", datetime());
-		vprintf(fmt, ap);
-		va_end(ap);
-	}
-	return 0;
+  int i;
+  for (i = 0; i < gcb.listen_sock_count; i++)
+    if (gcb.listen_socks[i] > 0) {
+      closesocket(gcb.listen_socks[i]);
+    }
+  exit(1);
 }
 
-void gfatal(const char* fline, const char *fmt, ...)
-{
-	va_list ap;
+static gnet_request_t* gnet_parse_request(const char* buf, int* len,
+                                          apr_pool_t* pool) {
+  int n = *len;
+  int empty, completed;
+  const char* p;
+  char* line;
+  char* last = NULL;
+  char* colon;
+  gnet_request_t* req = 0;
+
+  /* find an empty line */
+  *len = 0;
+  empty = 1, completed = 0;
+  for (p = buf; n > 0 && *p; p++, n--) {
+    int ch = *p;
+    /* skip spaces */
+    if (ch == ' ' || ch == '\t' || ch == '\r') continue;
+    if (ch == '\n') {
+      if (!empty) {
+        empty = 1;
+        continue;
+      }
+      p++;
+      completed = 1;
+      break;
+    }
+    empty = 0;
+  }
+  if (!completed) return 0;
+
+  /* we have a complete HTTP-style request (terminated by empty line) */
+  *len = n = p - buf;                /* consume it */
+  line = apr_pstrndup(pool, buf, n); /* dup it */
+  req = pcalloc_safe(NULL, pool, sizeof(gnet_request_t),
+                     "out of memory in gnet_parse_request");
+
+  /* for first line */
+  line = apr_strtok(line, "\n", &last);
+  if (!line) line = apr_pstrdup(pool, "");
+  line = gstring_trim(line);
+
+  if (0 != apr_tokenize_to_argv(line, &req->argv, pool)) return req;
+
+  while (req->argv[req->argc]) req->argc++;
+
+  if (last == NULL) {
+    gwarning(NULL, "last is NULL");
+    return req;
+  }
+
+  /* for each subsequent lines */
+  while (0 != (line = apr_strtok(0, "\n", &last))) {
+    if (*line == ' ' || *line == '\t') {
+      /* continuation */
+      if (req->hc == 0) /* illegal - missing first header */
+        break;
+
+      line = gstring_trim(line);
+      if (*line == 0) /* empty line */
+        break;
+
+      /* add to previous hvalue */
+      req->hvalue[req->hc - 1] = gstring_trim(
+          apr_pstrcat(pool, req->hvalue[req->hc - 1], " ", line, (char*)0));
+      continue;
+    }
+    /* find a colon, and break the line in two */
+    if (!(colon = strchr(line, ':')))
+      colon = line + strlen(line);
+    else
+      *colon++ = 0;
+
+    line = gstring_trim(line);
+    if (*line == 0) /* empty line */
+      break;
+
+    /* save name, value pair */
+    req->hname[req->hc] = line;
+    req->hvalue[req->hc] = gstring_trim(colon);
+    req->hc++;
+
+    if (req->hc >= sizeof(req->hname) / sizeof(req->hname[0])) break;
+    if (last == NULL) break;
+  }
+
+  return req;
+}
+
+static char* gstring_trim(char* s) {
+  char* p;
+  s += strspn(s, " \t\r\n");
+  for (p = s + strlen(s) - 1; p > s; p--) {
+    if (strchr(" \t\r\n", *p))
+      *p = 0;
+    else
+      break;
+  }
+  return s;
+}
+
+static char* datetime(apr_time_t t) {
+  static char buf[100];
+  apr_time_exp_t texp;
+
+  apr_time_exp_lt(&texp, t);
 
-	va_start(ap, fmt);
-	printf("[%s] [INTERNAL ERROR %s] ", datetime(), fline);
-	vprintf(fmt, ap);
-	va_end(ap);
-	printf("\n          ... exiting\n");
+  sprintf(buf, "%04d-%02d-%02d %02d:%02d:%02d", 1900 + texp.tm_year,
+          1 + texp.tm_mon, texp.tm_mday, texp.tm_hour, texp.tm_min,
+          texp.tm_sec);
 
-	exit(1);
+  return buf;
 }
 
-int gwarning(const char* fline, const char *fmt, ...)
-{
-	va_list ap;
+static char* datetime_now(void) { return datetime(apr_time_now()); }
 
-	va_start(ap, fmt);
-	printf("[%s] [WRN %s] ", datetime(), fline);
-	vprintf(fmt, ap);
-	va_end(ap);
-	printf("\n");
+/*
+ *  get process id
+ */
+static int ggetpid() {
+  static int pid = 0;
+  if (pid == 0) {
+#ifdef WIN32
+    pid = GetCurrentProcessId();
+#else
+    pid = getpid();
+#endif
+  }
 
-	return 0;
+  return pid;
 }
 
-void gfile_printf_then_putc_newline(const char *format, ...)
-{
-	va_list va;
+static void _gprint(const request_t* r, const char* level, const char* fmt,
+                    va_list args)
+    __attribute__((format(PG_PRINTF_ATTRIBUTE, 3, 0)));
+
+static void _gprint(const request_t* r, const char* level, const char* fmt,
+                    va_list args) {
+  printf("%s %d %s ", datetime_now(), ggetpid(), level);
+  if (r != NULL) {
+    printf("[%ld:%ld:%d:%d] ", GET_SID(r), r->id, r->segid, r->sock);
+  }
+  vprintf(fmt, args);
+}
 
-	va_start(va,format);
-	vprintf(format, va);
-	va_end(va);
-	putchar('\n');
+void gprint(const request_t* r, const char* fmt, ...) {
+  va_list args;
+  va_start(args, fmt);
+  _gprint(r, "INFO", fmt, args);
+  va_end(args);
 }
 
-void *gfile_malloc(size_t size)
-{
-	return malloc(size);
+void gprintln(const request_t* r, const char* fmt, ...) {
+  if (opt.s) {
+    return;
+  }
+
+  va_list args;
+  va_start(args, fmt);
+  _gprint(r, "INFO", fmt, args);
+  va_end(args);
+  printf("\n");
 }
 
-void gfile_free(void *a)
-{
-	free(a);
+/*
+ * Print for GET, or POST if Verbose.
+ */
+void gprintlnif(const request_t* r, const char* fmt, ...) {
+  if (r != NULL && !r->is_get && !opt.V) return;
+
+  if (opt.s) {
+    return;
+  }
+
+  va_list args;
+  va_start(args, fmt);
+  _gprint(r, "INFO", fmt, args);
+  va_end(args);
+  printf("\n");
+}
+
+void gfatal(const request_t* r, const char* fmt, ...) {
+  va_list args;
+  va_start(args, fmt);
+  _gprint(r, "FATAL", fmt, args);
+  va_end(args);
+
+  printf("\n          ... exiting\n");
+  exit(1);
 }
 
+void gwarning(const request_t* r, const char* fmt, ...) {
+  va_list args;
+  va_start(args, fmt);
+  _gprint(r, "WARN", fmt, args);
+  va_end(args);
+  printf("\n");
+}
+
+void gdebug(const request_t* r, const char* fmt, ...) {
+  if (!opt.V) return;
+
+  va_list args;
+  va_start(args, fmt);
+  _gprint(r, "DEBUG", fmt, args);
+  va_end(args);
+  printf("\n");
+}
+
+void gfile_printf_then_putc_newline(const char* format, ...) {
+  va_list va;
+
+  va_start(va, format);
+  vprintf(format, va);
+  va_end(va);
+  putchar('\n');
+}
+
+void* gfile_malloc(size_t size) {
+  void* p = malloc(size);
+  if (!p) gfatal(NULL, "Out of memory");
+  return p;
+}
+
+void gfile_free(void* a) { free(a); }
+
 /*
  * percent_encoding_to_char
  *
  * decode any percent encoded characters that may be included in the http
  * request into normal characters ascii characters.
  */
-void percent_encoding_to_char(char* p, char* pp, char* path)
-{
-	/*   - decode %xx to char */
-	for (p = pp = path; *pp; p++, pp++)
-	{
-		if ('%' == (*p = *pp))
-		{
-			if (pp[1] && pp[2])
-			{
-				int x = pp[1];
-				int y = pp[2];
-
-				if ('0' <= x && x <= '9')
-					x -= '0';
-				else if ('a' <= x && x <= 'f')
-					x = x - 'a' + 10;
-				else if ('A' <= x && x <= 'F')
-					x = x - 'A' + 10;
-				else
-					x = -1;
-
-				if ('0' <= y && y <= '9')
-					y -= '0';
-				else if ('a' <= y && y <= 'f')
-					y = y - 'a' + 10;
-				else if ('A' <= y && y <= 'F')
-					y = y - 'A' + 10;
-				else
-					y = -1;
-
-				if (x >= 0 && y >= 0)
-				{
-					x = (x << 4) + y;
-					*p = (char) x;
-					pp++, pp++;
-				}
-			}
-		}
-	}
-
-	*p = 0;
-}
-
-static void handle_get_request(request_t *r)
-{
-	/* setup to receive EV_WRITE events to write to socket */
-	if (setup_write(r))
-	{
-		http_error(r, FDIST_INTERNAL_ERROR, "internal error");
-		request_end(r, 1, 0);
-		return;
-	}
-
-	if (0 != http_ok(r))
-		request_end(r, 1, 0);
-}
-
-static void handle_post_request(request_t *r, int header_end)
-{
-	int h_count = r->in.req->hc;
-	char** h_names = r->in.req->hname;
-	char** h_values = r->in.req->hvalue;
-	int i = 0;
-	int b_continue = 0;
-	char *data_start = 0;
-	int data_bytes_in_req = 0;
-	int wrote = 0;
-
-	/*
-	 * If this request is a "done" request (has GP-DONE header set)
-	 * it has already marked this segment as inactive in this session.
-	 * This is all that a "done" request should do. no data to process.
-	 * we send our success response and end the request.
-	 */
-	if(r->is_final)
-		goto done_processing_request;
-
-	for(i = 0 ; i < h_count ; i++)
-	{
-		/* the request include a "Expect: 100-continue" header? */
-		if(strcmp("Expect", h_names[i]) == 0 && strcmp("100-continue", h_values[i]) == 0)
-			b_continue = 1;
-
-		/* find out how long is our data by looking at "Content-Length" header*/
-		if(strcmp("Content-Length", h_names[i]) == 0)
-			r->in.davailable = atoi(h_values[i]);
-	}
-
-	/* if client asked for 100-Continue, send it. otherwise, move on. */
-	if(b_continue)
-		http_continue(r);
-
-	gprint("available data to consume %d, starting at offset %d\n", r->in.davailable, r->in.hbuftop);
-
-	/* create a buffer to hold the incoming raw data */
-	r->in.dbufmax = opt.m; /* size of max line size */
-	r->in.dbuf = apr_palloc(r->pool, r->in.dbufmax);
-	r->in.dbuftop = 0;
-
-	/* if some data come along with the request, copy it first */
-	data_start = strstr(r->in.hbuf, "\r\n\r\n");
-	if(data_start)
-	{
-		data_start += 4;
-		data_bytes_in_req = (r->in.hbuf + r->in.hbuftop) - data_start;
-	}
-
-	if(data_bytes_in_req > 0)
-	{
-		/* we have data after the request headers. consume it */
-		memcpy(r->in.dbuf, data_start, data_bytes_in_req);
-		r->in.dbuftop += data_bytes_in_req;
-		r->in.davailable -= data_bytes_in_req;
-
-		/* only write it out if no more data is expected */
-		if(r->in.davailable == 0)
-			wrote = fstream_write(r->session->fstream, r->in.dbuf, data_bytes_in_req, 1, r->line_delim_str, r->line_delim_length);
-	}
-
-	/*
-	 * we've consumed all data that came in the first buffer (with the request)
-	 * if we're still expecting more data, get it from socket now and process it.
-	 */
-	while(r->in.davailable > 0)
-	{
-		size_t want;
-		ssize_t n;
-		size_t buf_space_left = r->in.dbufmax - r->in.dbuftop;
-
-		if (r->in.davailable >= buf_space_left)
-			want = buf_space_left;
-		else
-			want = r->in.davailable;
-
-		/* read from socket into data buf */
-		n = gpfdist_receive(r, r->in.dbuf + r->in.dbuftop, want);
-
-		if (n < 0)
-		{
+void percent_encoding_to_char(char* p, char* pp, char* path) {
+  /*   - decode %xx to char */
+  for (p = pp = path; *pp; p++, pp++) {
+    if ('%' == (*p = *pp)) {
+      if (pp[1] && pp[2]) {
+        int x = pp[1];
+        int y = pp[2];
+
+        if ('0' <= x && x <= '9')
+          x -= '0';
+        else if ('a' <= x && x <= 'f')
+          x = x - 'a' + 10;
+        else if ('A' <= x && x <= 'F')
+          x = x - 'A' + 10;
+        else
+          x = -1;
+
+        if ('0' <= y && y <= '9')
+          y -= '0';
+        else if ('a' <= y && y <= 'f')
+          y = y - 'a' + 10;
+        else if ('A' <= y && y <= 'F')
+          y = y - 'A' + 10;
+        else
+          y = -1;
+
+        if (x >= 0 && y >= 0) {
+          x = (x << 4) + y;
+          *p = (char)x;
+          pp++, pp++;
+        }
+      }
+    }
+  }
+
+  *p = 0;
+}
+
+static void handle_get_request(request_t* r) {
+  /* setup to receive EV_WRITE events to write to socket */
+  if (setup_write(r)) {
+    gwarning(r, "handle_get_request failed to setup write handler");
+    http_error(r, FDIST_INTERNAL_ERROR, "internal error");
+    request_end(r, 1, 0);
+    return;
+  }
+
+  if (0 != http_ok(r)) {
+    gwarning(r, "handle_get_request failed to send HTTP OK");
+    request_end(r, 1, 0);
+  }
+}
+
+static void handle_post_request(request_t* r, int header_end) {
+  int h_count = r->in.req->hc;
+  char** h_names = r->in.req->hname;
+  char** h_values = r->in.req->hvalue;
+  int i = 0;
+  int b_continue = 0;
+  char* data_start = 0;
+  int data_bytes_in_req = 0;
+  int wrote = 0;
+  session_t* session = r->session;
+
+  /*
+   * If this request is a "done" request (has GP-DONE header set)
+   * it has already marked this segment as inactive in this session.
+   * This is all that a "done" request should do. no data to process.
+   * we send our success response and end the request.
+   */
+  if (r->is_final) goto done_processing_request;
+
+  for (i = 0; i < h_count; i++) {
+    /* the request include a "Expect: 100-continue" header? */
+    if (strcmp("Expect", h_names[i]) == 0 &&
+        strcmp("100-continue", h_values[i]) == 0)
+      b_continue = 1;
+
+    /* find out how long is our data by looking at "Content-Length" header*/
+    if (strcmp("Content-Length", h_names[i]) == 0)
+      r->in.davailable = atoi(h_values[i]);
+  }
+
+  /* if client asked for 100-Continue, send it. otherwise, move on. */
+  if (b_continue) http_continue(r);
+
+  gdebug(r, "available data to consume %d, starting at offset %d",
+         r->in.davailable, r->in.hbuftop);
+
+  switch (r->seq) {
+    case OPEN_SEQ:
+      /* sequence number is 1, it's the first OPEN request */
+      session->seq_segs[r->segid] = r->seq;
+      goto done_processing_request;
+
+    case NO_SEQ:
+      /* don't have sequence number */
+      if (session->seq_segs[r->segid] > 0) {
+        /* missing sequence number */
+#ifdef WIN32
+        gprintln(r,
+                 "got an request missing sequence number, expected sequence "
+                 "number is %ld.",
+                 (long)session->seq_segs[r->segid] + 1);
+#else
+        gprintln(r,
+                 "got an request missing sequence number, expected sequence "
+                 "number is %" APR_INT64_T_FMT,
+                 session->seq_segs[r->segid] + 1);
+#endif
+        http_error(r, FDIST_BAD_REQUEST,
+                   "invalid request due to missing sequence number");
+        gwarning(r, "got an request missing sequence number");
+        request_end(r, 1, 0);
+        return;
+      } else {
+        /* old version GPDB, don't have sequence number */
+        break;
+      }
+
+    default:
+      /* sequence number > 1, it's the subsequent DATA request */
+      if (session->seq_segs[r->segid] == r->seq) {
+        /* duplicate DATA request, ignore it*/
 #ifdef WIN32
-			int e = WSAGetLastError();
-			int ok = (e == WSAEINTR || e == WSAEWOULDBLOCK);
+        gdebug(r, "got a duplicate request, sequence number is %ld.",
+               (long)r->seq);
 #else
-			int e = errno;
-			int ok = (e == EINTR || e == EAGAIN);
-#endif
-			if (!ok)
-			{
-				request_end(r, 1, 0);
-				return;
-			}
-		}
-		else if (n == 0)
-		{
-			/* socket close by peer will return 0 */
-			request_end(r, 1, 0);
-			return;
-		}
-		else
-		{
-			/*gprint("received %d bytes from client\n", n);*/
-
-			r->in.davailable -= n;
-			r->in.dbuftop += n;
-
-			/* if filled our buffer or no more data expected, write it */
-			if (r->in.dbufmax == r->in.dbuftop || r->in.davailable == 0)
-			{
-				/* only write up to end of last row */
-				wrote = fstream_write(r->session->fstream, r->in.dbuf, r->in.dbuftop, 1, r->line_delim_str, r->line_delim_length);
-				gprint("wrote %d bytes to file\n", wrote);
-
-				if(wrote == -1)
-				{
-					/* write error */
-					http_error(r, FDIST_INTERNAL_ERROR, fstream_get_error(r->session->fstream));
-					request_end(r, 1, 0);
-				}
-				else if(wrote == r->in.dbuftop)
-				{
-					/* wrote the whole buffer. clean it for next round */
-					r->in.dbuftop = 0;
-				}
-				else
-				{
-					/* wrote up to last line, some data left over in buffer. move to front */
-					int bytes_left_over = r->in.dbuftop - wrote;
-
-					memmove(r->in.dbuf, r->in.dbuf + wrote, bytes_left_over);
-					r->in.dbuftop = bytes_left_over;
-				}
-			}
-		}
-
-	}
+        gdebug(r,
+               "got a duplicate request, sequence number is %" APR_INT64_T_FMT
+               ".",
+               r->seq);
+#endif
+        goto done_processing_request;
+      } else if (session->seq_segs[r->segid] != r->seq - 1) {
+        /* out of order DATA request, ignore it*/
+#ifdef WIN32
+        gprintln(r,
+                 "got an out of order request, sequence number is %ld, "
+                 "expected sequence number is %ld.",
+                 (long)r->seq, (long)session->seq_segs[r->segid] + 1);
+#else
+        gprintln(
+            r,
+            "got an out of order request, sequence number is %" APR_INT64_T_FMT
+            ", expected sequence number is %" APR_INT64_T_FMT,
+            r->seq, session->seq_segs[r->segid] + 1);
+#endif
+        http_error(r, FDIST_BAD_REQUEST,
+                   "invalid request due to wrong sequence number");
+        gwarning(r, "got an out of order request");
+        request_end(r, 1, 0);
+        return;
+      }
+  }
+
+  /* create a buffer to hold the incoming raw data */
+  r->in.dbufmax = opt.m; /* size of max line size */
+  r->in.dbuftop = 0;
+  r->in.dbuf = palloc_safe(r, r->pool, r->in.dbufmax,
+                           "out of memory when allocating r->in.dbuf: %d bytes",
+                           r->in.dbufmax);
+
+  /* if some data come along with the request, copy it first */
+  data_start = strstr(r->in.hbuf, "\r\n\r\n");
+  if (data_start) {
+    data_start += 4;
+    data_bytes_in_req = (r->in.hbuf + r->in.hbuftop) - data_start;
+  }
+
+  if (data_bytes_in_req > 0) {
+    /* we have data after the request headers. consume it */
+    /* should make sure r->in.dbuftop + data_bytes_in_req <  r->in.dbufmax */
+    memcpy(r->in.dbuf, data_start, data_bytes_in_req);
+    r->in.dbuftop += data_bytes_in_req;
+    r->in.davailable -= data_bytes_in_req;
+
+    /* only write it out if no more data is expected */
+    if (r->in.davailable == 0) {
+      wrote = fstream_write(session->fstream, r->in.dbuf, data_bytes_in_req, 1,
+                            r->line_delim_str, r->line_delim_length);
+      delay_watchdog_timer();
+      if (wrote == -1) {
+        /* write error */
+        http_error(r, FDIST_INTERNAL_ERROR,
+                   fstream_get_error(session->fstream));
+        request_end(r, 1, 0);
+        return;
+      }
+    }
+  }
+
+  /*
+   * we've consumed all data that came in the first buffer (with the request)
+   * if we're still expecting more data, get it from socket now and process it.
+   */
+  while (r->in.davailable > 0) {
+    size_t want;
+    ssize_t n;
+    size_t buf_space_left = r->in.dbufmax - r->in.dbuftop;
+
+    if (r->in.davailable > buf_space_left)
+      want = buf_space_left;
+    else
+      want = r->in.davailable;
+
+    /* read from socket into data buf */
+    n = gpfdist_receive(r, r->in.dbuf + r->in.dbuftop, want);
+
+    if (n < 0) {
+#ifdef WIN32
+      int e = WSAGetLastError();
+      int ok = (e == WSAEINTR || e == WSAEWOULDBLOCK);
+#else
+      int e = errno;
+      int ok = (e == EINTR || e == EAGAIN);
+#endif
+      if (!ok) {
+        gwarning(r, "handle_post_request receive errno: %d, msg: %s", e,
+                 strerror(e));
+        http_error(r, FDIST_INTERNAL_ERROR, "internal error");
+        request_end(r, 1, 0);
+        return;
+      }
+    } else if (n == 0) {
+      /* socket close by peer will return 0 */
+      gwarning(r, "handle_post_request socket closed by peer");
+      request_end(r, 1, 0);
+      return;
+    } else {
+      /*gprint("received %d bytes from client\n", n);*/
+
+      r->bytes += n;
+      r->last = apr_time_now();
+      r->in.davailable -= n;
+      r->in.dbuftop += n;
+
+      /* if filled our buffer or no more data expected, write it */
+      if (r->in.dbufmax == r->in.dbuftop || r->in.davailable == 0) {
+        /* only write up to end of last row */
+        wrote = fstream_write(session->fstream, r->in.dbuf, r->in.dbuftop, 1,
+                              r->line_delim_str, r->line_delim_length);
+        gdebug(r, "wrote %d bytes to file", wrote);
+        delay_watchdog_timer();
+
+        if (wrote == -1) {
+          /* write error */
+          gwarning(r, "handle_post_request, write error: %s",
+                   fstream_get_error(session->fstream));
+          http_error(r, FDIST_INTERNAL_ERROR,
+                     fstream_get_error(session->fstream));
+          request_end(r, 1, 0);
+          return;
+        } else if (wrote == r->in.dbuftop) {
+          /* wrote the whole buffer. clean it for next round */
+          r->in.dbuftop = 0;
+        } else {
+          /* wrote up to last line, some data left over in buffer. move to front
+           */
+          int bytes_left_over = r->in.dbuftop - wrote;
+
+          memmove(r->in.dbuf, r->in.dbuf + wrote, bytes_left_over);
+          r->in.dbuftop = bytes_left_over;
+        }
+      }
+    }
+  }
+
+  session->seq_segs[r->segid] = r->seq;
 
 done_processing_request:
 
-	/* send our success response and end the request */
-	if (0 != http_ok(r))
-		request_end(r, 1, 0);
-	else
-		request_end(r, 0, 0); /* we're done! */
-
-}
-
-static int request_set_path(request_t *r, const char* d, char* p, char* pp, char* path)
-{
-
-	/*
-	 * disallow using a relative path in the request
-	 */
-	if (strstr(path, ".."))
-	{
-		gwarning(FLINE, "reject invalid request from %s [%s %s] - request "
-						"is using a relative path",
-						r->peer,
-						r->in.req->argv[0],
-						r->in.req->argv[1]);
-
-		http_error(r, FDIST_BAD_REQUEST, "invalid request due to relative path");
-		request_end(r, 1, 0);
-
-		return -1;
-	}
-
-	r->path = 0;
-
-	/*
-	 * make the new path relative to the user's specified dir (opt.d)
-	 */
-	do
-	{
-		while (*path == ' ')
-			path++;
-
-		p = strchr(path, ' ');
-
-		if (p)
-			*p++ = 0;
-
-		while (*path == '/')
-			path++;
-
-		if (*path)
-		{
-			if (r->path)
-				r->path = apr_psprintf(r->pool, "%s %s/%s", r->path, d,
-									   path);
-			else
-				r->path = apr_psprintf(r->pool, "%s/%s", d, path);
-		}
-
-		path = p;
-
-	} while (path);
-
-
-	if (!r->path)
-	{
-		http_error(r, FDIST_BAD_REQUEST, "invalid request (unable to set path)");
-		request_end(r, 1, 0);
-		return -1;
-	}
-
-	return 0;
-}
-
-static int request_validate(request_t *r)
-{
-	/* parse the HTTP request. Expect "GET /path HTTP/1.X" or "PUT /path HTTP/1.X" */
-	if (r->in.req->argc != 3)
-	{
-		gwarning(FLINE, "reject invalid request from %s", r->peer);
-		http_error(r, FDIST_BAD_REQUEST, "invalid request");
-		request_end(r, 1, 0);
-		return -1;
-	}
-	if (0 != strncmp("HTTP/1.", r->in.req->argv[2], 7))
-	{
-		gwarning(FLINE, "reject invalid protocol from %s [%s]", r->peer,
-				r->in.req->argv[2]);
-		http_error(r, FDIST_BAD_REQUEST, "invalid request");
-		request_end(r, 1, 0);
-		return -1;
-	}
-	if (0 != strcmp("GET", r->in.req->argv[0]) &&
-		0 != strcmp("POST", r->in.req->argv[0]))
-	{
-		gwarning(FLINE, "reject invalid request from %s [%s %s]", r->peer,
-				r->in.req->argv[0], r->in.req->argv[1]);
-		http_error(r, FDIST_BAD_REQUEST, "invalid request");
-		request_end(r, 1, 0);
-		return -1;
-	}
-
-	return 0;
-}
-
-static void base16_decode(char* data)
-{
-	int i = 0;
-	char *encoded_bytes = data;
-
-	char buf[3];
-	buf[2] = '\0';
-
-	while (encoded_bytes[0])
-	{
-		buf[0] = encoded_bytes[0];
-		buf[1] = encoded_bytes[1];
-
-        data[i] = strtoul(buf, NULL, 16);
-
-		i++;
-        encoded_bytes += 2;
-     }
-     data[i] = '\0';
+  /* send our success response and end the request */
+  if (0 != http_ok(r))
+    request_end(r, 1, 0);
+  else
+    request_end(r, 0, 0); /* we're done! */
+}
+
+static int request_set_path(request_t* r, const char* d, char* p, char* pp,
+                            char* path) {
+  /*
+   * disallow using a relative path in the request
+   */
+  if (strstr(path, "..")) {
+    gwarning(r,
+             "reject invalid request from %s [%s %s] - request "
+             "is using a relative path",
+             r->peer, r->in.req->argv[0], r->in.req->argv[1]);
+
+    http_error(r, FDIST_BAD_REQUEST, "invalid request due to relative path");
+    request_end(r, 1, 0);
+
+    return -1;
+  }
+
+  r->path = 0;
+
+  /*
+   * make the new path relative to the user's specified dir (opt.d)
+   */
+  do {
+    while (*path == ' ') path++;
+
+    p = strchr(path, ' ');
+
+    if (p) *p++ = 0;
+
+    while (*path == '/') path++;
+
+    if (*path) {
+      if (r->path)
+        r->path = apr_psprintf(r->pool, "%s %s/%s", r->path, d, path);
+      else
+        r->path = apr_psprintf(r->pool, "%s/%s", d, path);
+    }
+
+    path = p;
+
+  } while (path);
+
+  if (!r->path) {
+    http_error(r, FDIST_BAD_REQUEST, "invalid request (unable to set path)");
+    request_end(r, 1, 0);
+    return -1;
+  }
+
+  return 0;
+}
+
+static int request_validate(request_t* r) {
+  /* parse the HTTP request. Expect "GET /path HTTP/1.X" or "PUT /path HTTP/1.X"
+   */
+  if (r->in.req->argc != 3) {
+    gprintln(r, "reject invalid request from %s", r->peer);
+    http_error(r, FDIST_BAD_REQUEST, "invalid request");
+    request_end(r, 1, 0);
+    return -1;
+  }
+  if (0 != strncmp("HTTP/1.", r->in.req->argv[2], 7)) {
+    gprintln(r, "reject invalid protocol from %s [%s]", r->peer,
+             r->in.req->argv[2]);
+    http_error(r, FDIST_BAD_REQUEST, "invalid request");
+    request_end(r, 1, 0);
+    return -1;
+  }
+  if (0 != strcmp("GET", r->in.req->argv[0]) &&
+      0 != strcmp("POST", r->in.req->argv[0])) {
+    gprintln(r, "reject invalid request from %s [%s %s]", r->peer,
+             r->in.req->argv[0], r->in.req->argv[1]);
+    http_error(r, FDIST_BAD_REQUEST, "invalid request");
+    request_end(r, 1, 0);
+    return -1;
+  }
+
+  return 0;
+}
+
+static bool base16_decode(char* data) {
+  int i = 0;
+  char* encoded_bytes = data;
+
+  char buf[3];
+  buf[2] = '\0';
+
+  while (encoded_bytes[0]) {
+    buf[0] = encoded_bytes[0];
+    buf[1] = encoded_bytes[1];
+    char* endptr = NULL;
+    char ch = strtoul(buf, &endptr, 16);
+    if (*endptr != '\0') {
+      return false;
+    }
+    data[i] = ch;
+    i++;
+    encoded_bytes += 2;
+  }
+  data[i] = '\0';
+  return true;
 }
 
 /*
@@ -2709,240 +3071,254 @@ static void base16_decode(char* data)
  * Extract all X-GP-* variables from the HTTP headers.
  * Create a unique X-GP-TID value from it.
  */
-static int request_parse_gp_headers(request_t *r, int opt_g)
-{
-	const char* xid = 0;
-	const char* cid = 0;
-	const char* sn = 0;
-	const char* gp_proto = "0";
-	int 		i;
-
-	r->csvopt = "";
-	r->is_final = 0;
-
-	for (i = 0; i < r->in.req->hc; i++)
-	{
-		if (0 == strcmp("X-GP-XID", r->in.req->hname[i]))
-			xid = r->in.req->hvalue[i];
-		else if (0 == strcmp("X-GP-CID", r->in.req->hname[i]))
-			cid = r->in.req->hvalue[i];
-		else if (0 == strcmp("X-GP-SN", r->in.req->hname[i]))
-			sn = r->in.req->hvalue[i];
-		else if (0 == strcmp("X-GP-CSVOPT", r->in.req->hname[i]))
-			r->csvopt = r->in.req->hvalue[i];
-		else if (0 == strcmp("X-GP-PROTO", r->in.req->hname[i]))
-			gp_proto = r->in.req->hvalue[i];
-		else if (0 == strcmp("X-GP-DONE", r->in.req->hname[i]))
-			r->is_final = 1;
-		else if (0 == strcmp("X-GP-SEGMENT-COUNT", r->in.req->hname[i]))
-			r->totalsegs = atoi(r->in.req->hvalue[i]);
-		else if (0 == strcmp("X-GP-SEGMENT-ID", r->in.req->hname[i]))
-			r->segid = atoi(r->in.req->hvalue[i]);
-		else if (0 == strcmp("X-GP-LINE-DELIM-STR", r->in.req->hname[i]))
-		{
-			if (NULL == r->in.req->hvalue[i] || ((int)strlen(r->in.req->hvalue[i]))%2 == 1)
-			{
-				gwarning(FLINE, "reject invalid request from %s, invalid EOL encoding: %s", r->peer, r->in.req->hvalue[i]);
-				http_error(r, FDIST_BAD_REQUEST, "invalid EOL encoding");
-				request_end(r, 1, 0);
-			}
-			base16_decode(r->in.req->hvalue[i]);
-			r->line_delim_str = r->in.req->hvalue[i];
-		}
-		else if (0 == strcmp("X-GP-LINE-DELIM-LENGTH", r->in.req->hname[i]))
-			r->line_delim_length = atoi(r->in.req->hvalue[i]);
+static int request_parse_gp_headers(request_t* r, int opt_g) {
+  const char* xid = 0;
+  const char* cid = 0;
+  const char* sn = 0;
+  const char* gp_proto =
+      NULL; /* default to invalid, so that report error if not specified*/
+  int i;
+
+  r->csvopt = "";
+  r->is_final = 0;
+  r->seq = 0;
+
+  for (i = 0; i < r->in.req->hc; i++) {
+    if (0 == strcasecmp("X-GP-XID", r->in.req->hname[i]))
+      xid = r->in.req->hvalue[i];
+    else if (0 == strcasecmp("X-GP-CID", r->in.req->hname[i]))
+      cid = r->in.req->hvalue[i];
+    else if (0 == strcasecmp("X-GP-SN", r->in.req->hname[i]))
+      sn = r->in.req->hvalue[i];
+    else if (0 == strcasecmp("X-GP-CSVOPT", r->in.req->hname[i]))
+      r->csvopt = r->in.req->hvalue[i];
+    else if (0 == strcasecmp("X-GP-PROTO", r->in.req->hname[i]))
+      gp_proto = r->in.req->hvalue[i];
+    else if (0 == strcasecmp("X-GP-DONE", r->in.req->hname[i]))
+      r->is_final = 1;
+    else if (0 == strcasecmp("X-GP-SEGMENT-COUNT", r->in.req->hname[i]))
+      r->totalsegs = atoi(r->in.req->hvalue[i]);
+    else if (0 == strcasecmp("X-GP-SEGMENT-ID", r->in.req->hname[i]))
+      r->segid = atoi(r->in.req->hvalue[i]);
+    else if (0 == strcasecmp("X-GP-LINE-DELIM-STR", r->in.req->hname[i])) {
+      if (NULL == r->in.req->hvalue[i] ||
+          ((int)strlen(r->in.req->hvalue[i])) % 2 == 1 ||
+          !base16_decode(r->in.req->hvalue[i])) {
+        gwarning(r, "reject invalid request from %s, invalid EOL encoding: %s",
+                 r->peer, r->in.req->hvalue[i]);
+        http_error(r, FDIST_BAD_REQUEST, "invalid EOL encoding");
+        request_end(r, 1, 0);
+        return -1;
+      }
+      r->line_delim_str = r->in.req->hvalue[i];
+    } else if (0 == strcasecmp("X-GP-LINE-DELIM-LENGTH", r->in.req->hname[i]))
+      r->line_delim_length = atoi(r->in.req->hvalue[i]);
 #ifdef GPFXDIST
-		else if (0 == strcmp("X-GP-TRANSFORM", r->in.req->hname[i]))
-			r->trans.name = r->in.req->hvalue[i];
-#endif
-	}
-
-	if (r->line_delim_length > 0)
-	{
-		if (NULL == r->line_delim_str || (((int)strlen(r->line_delim_str)) != r->line_delim_length))
-		{
-			gwarning(FLINE, "reject invalid request from %s, invalid EOL length: %s, EOL: %s", r->peer, r->line_delim_length, r->line_delim_str);
-			http_error(r, FDIST_BAD_REQUEST, "invalid EOL length");
-			request_end(r, 1, 0);
-		}
-	}
-	r->gp_proto = strtol(gp_proto, 0, 0);
-
-	if (r->gp_proto != 0 && r->gp_proto != 1)
-	{
-		gwarning(FLINE,
-				"reject invalid request from %s [%s %s] - X-GP-PROTO "
-				"invalid '%s'",
-				r->peer, r->in.req->argv[0], r->in.req->argv[1], gp_proto);
-		http_error(r, FDIST_BAD_REQUEST, "invalid request (invalid gp-proto)");
-		request_end(r, 1, 0);
-		return -1;
-	}
-
-	if (opt_g != -1) /* override?  */
-		r->gp_proto = opt_g;
-
-	if (xid && cid && sn)
-	{
-		r->tid = apr_psprintf(r->pool, "%s.%s.%s.%d", xid, cid, sn,
-							  r->gp_proto);
-	}
-	else if (xid || cid || sn)
-	{
-		gwarning(FLINE,
-				 "reject invalid request from %s [%s %s] - missing X-GP-* "
-				 "header",
-				 r->peer, r->in.req->argv[0], r->in.req->argv[1]);
-		http_error(r, FDIST_BAD_REQUEST, "invalid request (missing X-GP-* header)");
-		request_end(r, 1, 0);
-		return -1;
-	}
-	else
-	{
-		r->tid = apr_psprintf(r->pool, "auto-tid.%d", gcb.session.gen++);
-	}
-
-	return 0;
-}
+    else if (0 == strcasecmp("X-GP-TRANSFORM", r->in.req->hname[i]))
+      r->trans.name = r->in.req->hvalue[i];
+#endif
+    else if (0 == strcasecmp("X-GP-SEQ", r->in.req->hname[i])) {
+      r->seq = atol(r->in.req->hvalue[i]);
+      /* sequence number starting from 1 */
+      if (r->seq <= 0) {
+        gwarning(r,
+                 "reject invalid request from %s, invalid sequence number: %s",
+                 r->peer, r->in.req->hvalue[i]);
+        http_error(r, FDIST_BAD_REQUEST, "invalid sequence number");
+        request_end(r, 1, 0);
+        return -1;
+      }
+    }
+  }
+
+  if (r->line_delim_length > 0) {
+    if (NULL == r->line_delim_str ||
+        (((int)strlen(r->line_delim_str)) != r->line_delim_length)) {
+      gwarning(
+          r, "reject invalid request from %s, invalid EOL length: %d, EOL: %s",
+          r->peer, r->line_delim_length, r->line_delim_str);
+      http_error(r, FDIST_BAD_REQUEST, "invalid EOL length");
+      request_end(r, 1, 0);
+      return -1;
+    }
+  }
+  if (gp_proto != NULL) {
+    r->gp_proto = strtol(gp_proto, 0, 0);
+  }
+
+  if (gp_proto == NULL || (r->gp_proto != 0 && r->gp_proto != 1)) {
+    if (gp_proto == NULL) {
+      gwarning(r, "reject invalid request from %s [%s %s] - no X-GP-PROTO",
+               r->peer, r->in.req->argv[0], r->in.req->argv[1]);
+      http_error(r, FDIST_BAD_REQUEST, "invalid request (no gp-proto)");
+    } else {
+      gwarning(
+          r, "reject invalid request from %s [%s %s] - X-GP-PROTO invalid '%s'",
+          r->peer, r->in.req->argv[0], r->in.req->argv[1], gp_proto);
+      http_error(r, FDIST_BAD_REQUEST, "invalid request (invalid gp-proto)");
+    }
 
+    request_end(r, 1, 0);
+    return -1;
+  }
+
+  if (opt_g != -1) /* override?  */
+    r->gp_proto = opt_g;
+
+  if (xid && cid && sn) {
+    r->tid = apr_psprintf(r->pool, "%s.%s.%s.%d", xid, cid, sn, r->gp_proto);
+  } else if (xid || cid || sn) {
+    gwarning(r,
+             "reject invalid request from %s [%s %s] - missing X-GP-* header",
+             r->peer, r->in.req->argv[0], r->in.req->argv[1]);
+    http_error(r, FDIST_BAD_REQUEST, "invalid request (missing X-GP-* header)");
+    request_end(r, 1, 0);
+    return -1;
+  } else {
+    r->tid = apr_psprintf(r->pool, "auto-tid.%d", gcb.session.gen++);
+  }
+
+  return 0;
+}
 
 #ifdef GPFXDIST
-static int request_set_transform(request_t *r)
-{
-    extern struct transform* transform_lookup(struct transform* trlist, const char* name, int for_write, int verbose);
-	extern char* transform_command(struct transform* tr);
-	extern int transform_stderr_server(struct transform* tr);
-	extern int transform_content_paths(struct transform* tr);
-	extern char* transform_safe(struct transform* tr);
-	extern regex_t* transform_saferegex(struct transform* tr);
-
-    struct transform* tr;
-	char* safe;
-
-	/*
-	 * Requests involving transformations should have a #transform=name in the external
-	 * table URL.  In Rio, GPDB moves the name into an X-GP-TRANSFORM header.  However
-	 * #transform= may still appear in the url in post requests.
-	 *
-	 * Note that ordinary HTTP clients and browsers do not typically transmit the portion
-	 * of the URL after a #.  RFC 2396 calls this part the fragment identifier.
-	 */
-
-	char* param = "#transform=";
-	char* start = strstr(r->path, param);
-	if (start)
-	{
-		/*
-		 * we have a transformation request encoded in the url
-		 */
-		*start = 0;
-		if (! r->trans.name)
-			r->trans.name = start + strlen(param);
-	}
-
-	if (! r->trans.name)
-		return 0;
-
+static int request_set_transform(request_t* r) {
+  extern struct transform* transform_lookup(
+      struct transform * trlist, const char* name, int for_write, int verbose);
+  extern char* transform_command(struct transform * tr);
+  extern int transform_stderr_server(struct transform * tr);
+  extern int transform_content_paths(struct transform * tr);
+  extern char* transform_safe(struct transform * tr);
+  extern regex_t* transform_saferegex(struct transform * tr);
+
+  struct transform* tr;
+  char* safe;
+
+  /*
+   * Requests involving transformations should have a #transform=name in the
+   * external table URL.  In Rio, GPDB moves the name into an X-GP-TRANSFORM
+   * header.  However #transform= may still appear in the url in post requests.
+   *
+   * Note that ordinary HTTP clients and browsers do not typically transmit the
+   * portion of the URL after a #.  RFC 2396 calls this part the fragment
+   * identifier.
+   */
+
+  char* param = "#transform=";
+  char* start = strstr(r->path, param);
+  if (start) {
     /*
-     * at this point r->trans.name is the name of the transformation requested
-     * in the url and r->is_get tells us what kind (input or output) to look for.
-     * attempt to look it up.
+     * we have a transformation request encoded in the url
      */
-	tr = transform_lookup(opt.trlist, r->trans.name, r->is_get ? 0 : 1, opt.V);
-    if (! tr)
-    {
-        if (r->is_get)
-        {
-            gwarning(FLINE, "reject invalid request from %s [%s %s] - unsppported input #transform",
-                     r->peer, r->in.req->argv[0], r->in.req->argv[1]);
-            http_error(r, FDIST_BAD_REQUEST, "invalid request (unsupported input #transform)");
-        }
-        else
-        {
-            gwarning(FLINE, "reject invalid request from %s [%s %s] - unsppported output #transform",
-                     r->peer, r->in.req->argv[0], r->in.req->argv[1]);
-            http_error(r, FDIST_BAD_REQUEST, "invalid request (unsupported output #transform)");
-        }
-        request_end(r, 1, 0);
-        return -1;
+    *start = 0;
+    if (!r->trans.name) r->trans.name = start + strlen(param);
+  }
+
+  if (!r->trans.name) return 0;
+
+  /*
+   * at this point r->trans.name is the name of the transformation requested
+   * in the url and r->is_get tells us what kind (input or output) to look for.
+   * attempt to look it up.
+   */
+  tr = transform_lookup(opt.trlist, r->trans.name, r->is_get ? 0 : 1, opt.V);
+  if (!tr) {
+    if (r->is_get) {
+      gprintln(r,
+               "reject invalid request from %s [%s %s] - unsppported input "
+               "#transform",
+               r->peer, r->in.req->argv[0], r->in.req->argv[1]);
+      http_error(r, FDIST_BAD_REQUEST,
+                 "invalid request (unsupported input #transform)");
+    } else {
+      gprintln(r,
+               "reject invalid request from %s [%s %s] - unsppported output "
+               "#transform",
+               r->peer, r->in.req->argv[0], r->in.req->argv[1]);
+      http_error(r, FDIST_BAD_REQUEST,
+                 "invalid request (unsupported output #transform)");
+    }
+    request_end(r, 1, 0);
+    return -1;
+  }
+
+  gprintln(r, "transform: %s", r->trans.name);
+
+  /*
+   * propagate details for this transformation
+   */
+  r->trans.command = transform_command(tr);
+  r->trans.paths = transform_content_paths(tr);
+
+  /*
+   * if safe regex is specified, check that the path matches it
+   */
+  safe = transform_safe(tr);
+  if (safe) {
+    regex_t* saferegex = transform_saferegex(tr);
+    int rc = regexec(saferegex, r->path, 0, NULL, 0);
+    if (rc) {
+      char buf[1024];
+      regerror(rc, saferegex, buf, sizeof(buf));
+
+      gprintln(r,
+               "reject invalid request from %s [%s %s] - path does not match "
+               "safe regex %s: %s",
+               r->peer, r->in.req->argv[0], r->in.req->argv[1], safe, buf);
+      http_error(r, FDIST_BAD_REQUEST,
+                 "invalid request (path does not match safe regex)");
+      request_end(r, 1, 0);
+      return -1;
+    } else {
+      gdebug(r, "[%d] safe regex %s matches %s", r->sock, safe, r->path);
+    }
+  }
+
+  /*
+   * if we've been requested to send stderr output to the server,
+   * we prepare a temporary file to hold it.	when the request is
+   * done we'll forward the output as error messages.
+   */
+  if (transform_stderr_server(tr)) {
+    apr_pool_t* mp = r->pool;
+    apr_file_t* f = NULL;
+    const char* tempdir = NULL;
+    char* tempfilename = NULL;
+    apr_status_t rv;
+
+    if ((rv = apr_temp_dir_get(&tempdir, mp)) != APR_SUCCESS) {
+      gprintln(r,
+               "request failed from %s [%s %s] - failed to get temporary "
+               "directory for stderr",
+               r->peer, r->in.req->argv[0], r->in.req->argv[1]);
+      http_error(r, FDIST_INTERNAL_ERROR, "internal error");
+      request_end(r, 1, 0);
+      return -1;
+    }
+
+    tempfilename = apr_pstrcat(mp, tempdir, "/stderrXXXXXX", NULL);
+    if ((rv = apr_file_mktemp(&f, tempfilename,
+                              APR_CREATE | APR_WRITE | APR_EXCL, mp)) !=
+        APR_SUCCESS) {
+      gprintln(r,
+               "request failed from %s [%s %s] - failed to create temporary "
+               "file for stderr",
+               r->peer, r->in.req->argv[0], r->in.req->argv[1]);
+      http_error(r, FDIST_INTERNAL_ERROR, "internal error");
+      request_end(r, 1, 0);
+      return -1;
     }
 
-	TR(("[%d] transform: %s\n", r->sock, r->trans.name));
-
-	/*
-	 * propagate details for this transformation
-	 */
-	r->trans.command = transform_command(tr);
-	r->trans.paths   = transform_content_paths(tr);
-
-	/*
-	 * if safe regex is specified, check that the path matches it
-	 */
-	safe = transform_safe(tr);
-	if (safe)
-	{
-		regex_t* saferegex = transform_saferegex(tr);
-		int rc = regexec(saferegex, r->path, 0, NULL, 0);
-		if (rc)
-		{
-			char buf[1024];
-			regerror(rc, saferegex, buf, sizeof(buf));
-
-			gwarning(FLINE, "reject invalid request from %s [%s %s] - path does not match safe regex %s: %s",
-					 r->peer, r->in.req->argv[0], r->in.req->argv[1], safe, buf);
-			http_error(r, FDIST_BAD_REQUEST, "invalid request (path does not match safe regex)");
-			request_end(r, 1, 0);
-			return -1;
-		}
-		else
-		{
-			TR(("[%d] safe regex %s matches %s\n", r->sock, safe, r->path));
-		}
-	}
-
-	/*
-	 * if we've been requested to send stderr output to the server,
-	 * we prepare a temporary file to hold it.	when the request is
-	 * done we'll forward the output as error messages.
-	 */
-	if (transform_stderr_server(tr))
-	{
-		apr_pool_t*	 mp = r->pool;
-		apr_file_t*	 f = NULL;
-		const char*	 tempdir = NULL;
-		char*		 tempfilename = NULL;
-		apr_status_t rv;
-
-		if ((rv = apr_temp_dir_get(&tempdir, mp)) != APR_SUCCESS)
-		{
-			gwarning(FLINE, "request failed from %s [%s %s] - failed to get temporary directory for stderr",
-					 r->peer, r->in.req->argv[0], r->in.req->argv[1]);
-			http_error(r, FDIST_INTERNAL_ERROR, "internal error");
-			request_end(r, 1, 0);
-			return -1;
-		}
-
-		tempfilename = apr_pstrcat(mp, tempdir, "/stderrXXXXXX", NULL);
-		if ((rv = apr_file_mktemp(&f, tempfilename, APR_CREATE|APR_WRITE|APR_EXCL, mp)) != APR_SUCCESS)
-		{
-			gwarning(FLINE, "request failed from %s [%s %s] - failed to create temporary file for stderr",
-					 r->peer, r->in.req->argv[0], r->in.req->argv[1]);
-			http_error(r, FDIST_INTERNAL_ERROR, "internal error");
-			request_end(r, 1, 0);
-			return -1;
-		}
-
-		TR(("[%d] request opened stderr file %s\n", r->sock, tempfilename));
-
-		r->trans.errfilename = tempfilename;
-		r->trans.errfile	 = f;
-	}
+    gdebug(r, "[%d] request opened stderr file %s\n", r->sock, tempfilename);
 
-    return 0;
+    r->trans.errfilename = tempfilename;
+    r->trans.errfile = f;
+  }
+
+  return 0;
 }
 #endif
 
-
 /*
  * gpfdist main
  *
@@ -2953,86 +3329,121 @@ static int request_set_transform(request_t *r)
  * 5) create the gpfdist log file and handle stderr/out redirection.
  * 6) sit and wait for an event.
  */
-int gpfdist_init(int argc, const char* const argv[])
-{
-	/*
-	 * Comment
-	 */
-	if (0 != apr_app_initialize(&argc, &argv, 0))
-		gfatal(FLINE, "apr_app_initialize failed");
-	atexit(apr_terminate);
+int gpfdist_init(int argc, const char* const argv[]) {
+  /*
+   * Comment
+   */
+  if (0 != apr_app_initialize(&argc, &argv, 0))
+    gfatal(NULL, "apr_app_initialize failed");
+  atexit(apr_terminate);
 
-	if (0 != apr_pool_create(&gcb.pool, 0))
-		gfatal(FLINE, "apr_app_initialize failed");
+  if (0 != apr_pool_create(&gcb.pool, 0))
+    gfatal(NULL, "apr_app_initialize failed");
 
-	apr_signal_init(gcb.pool);
+  // apr_signal_init(gcb.pool);
 
-	gcb.session.tab = apr_hash_make(gcb.pool);
+  gcb.session.tab = apr_hash_make(gcb.pool);
 
-	parse_command_line(argc, argv, gcb.pool);
+  parse_command_line(argc, argv, gcb.pool);
 
 #ifndef WIN32
 #ifdef SIGPIPE
-	signal(SIGPIPE, SIG_IGN);
+  signal(SIGPIPE, SIG_IGN);
 #endif
 #endif
-	/*
-	 * apr_signal(SIGINT, process_signal);
-	 */
-	apr_signal(SIGTERM, process_signal);
-	if (opt.V)
-		putenv("EVENT_SHOW_METHOD=1");
-	putenv("EVENT_NOKQUEUE=1");
-
-	event_init();
-	http_setup();
+  /*
+   * apr_signal(SIGINT, process_signal);
+   * apr_signal(SIGTERM, process_signal);
+   */
+  if (opt.V) putenv("EVENT_SHOW_METHOD=1");
+  putenv("EVENT_NOKQUEUE=1");
+
+  event_init();
+
+  signal_register();
+  http_setup();
+
+#ifdef USE_SSL
+  if (opt.ssl)
+    printf("Serving HTTPS on port %d, directory %s\n", opt.p, opt.d);
+  else
+    printf("Serving HTTP on port %d, directory %s\n", opt.p, opt.d);
+#else
+  printf("Serving HTTP on port %d, directory %s\n", opt.p, opt.d);
+#endif
 
-	if (opt.ssl)
-		printf("Serving HTTPS on port %d, directory %s\n", opt.p, opt.d);
-	else
-		printf("Serving HTTP on port %d, directory %s\n", opt.p, opt.d);
+  fflush(stdout);
 
-	fflush(stdout);
+  /* redirect stderr and stdout to log */
+  if (opt.l) {
+    FILE* f_stderr;
+    FILE* f_stdout;
 
-	/* redirect stderr and stdout to log */
-	if (opt.l)
-	{
-		freopen(opt.l, "a", stderr);
+    f_stderr = freopen(opt.l, "a", stderr);
+    if (f_stderr == NULL) {
+      fprintf(stderr, "failed to redirect stderr to log: %s\n",
+              strerror(errno));
+      return -1;
+    }
 #ifndef WIN32
-		setlinebuf(stderr);
+    setlinebuf(stderr);
 #endif
-		freopen(opt.l, "a", stdout);
+    f_stdout = freopen(opt.l, "a", stdout);
+    if (f_stdout == NULL) {
+      fprintf(stderr, "failed to redirect stdout to log: %s\n",
+              strerror(errno));
+      return -1;
+    }
 #ifndef WIN32
-		setlinebuf(stdout);
+    setlinebuf(stdout);
 #endif
-	}
-
-	/*
-	 * must identify errors in calls above and return non-zero for them
-	 * behaviour required for the Windows service case
-	 */
-	return 0;
-}
+  }
 
-int gpfdist_run()
-{
-	return event_dispatch();
-}
+  /*
+   * must identify errors in calls above and return non-zero for them
+   * behaviour required for the Windows service case
+   */
 
 #ifndef WIN32
+  char* wd = getenv("GPFDIST_WATCHDOG_TIMER");
+  char* endptr;
+  long val;
 
-int main(int argc, const char* const argv[])
-{
-	gpfdist_init(argc, argv);
-	return gpfdist_run();
+  if (wd != NULL) {
+    val = strtol(wd, &endptr, 10);
+
+    if (errno || endptr == wd || val > INT_MAX) {
+      fprintf(stderr, "incorrect watchdog timer: %s\n", strerror(errno));
+      return -1;
+    }
+
+    gcb.wdtimer = (int)val;
+    if (gcb.wdtimer > 0) {
+      gprintln(NULL, "Watchdog enabled, abort in %d seconds if no activity",
+               gcb.wdtimer);
+      shutdown_time = apr_time_now() + gcb.wdtimer * APR_USEC_PER_SEC;
+      static pthread_t watchdog;
+      pthread_create(&watchdog, 0, watchdog_thread, 0);
+    }
+  }
+#endif
+  return 0;
 }
 
+int gpfdist_run() { return event_dispatch(); }
 
-#else   /* in Windows gpfdist may run as a Windows service or a console application  */
+#ifndef WIN32
 
+int main(int argc, const char* const argv[]) {
+  if (gpfdist_init(argc, argv) == -1) gfatal(NULL, "Initialization failed");
+  return gpfdist_run();
+}
+
+#else /* in Windows gpfdist may run as a Windows service or a console \
+         application  */
 
-SERVICE_STATUS          ServiceStatus;
-SERVICE_STATUS_HANDLE   hStatus;
+SERVICE_STATUS ServiceStatus;
+SERVICE_STATUS_HANDLE hStatus;
 
 #define CMD_LINE_ARG_MAX_SIZE 1000
 #define CMD_LINE_ARG_SIZE 500
@@ -3040,9 +3451,8 @@ SERVICE_STATUS_HANDLE   hStatus;
 char* cmd_line_buffer[CMD_LINE_ARG_NUM];
 int cmd_line_args;
 
-void  ServiceMain(int argc, char** argv);
-void  ControlHandler(DWORD request);
-
+void ServiceMain(int argc, char** argv);
+void ControlHandler(DWORD request);
 
 /* gpfdist service registration on the WINDOWS command line
  * sc create gpfdist binpath= "c:\temp\gpfdist.exe param1 param2 param3"
@@ -3050,361 +3460,307 @@ void  ControlHandler(DWORD request);
  */
 
 /* HELPERS - START */
-void report_event(LPCTSTR _error_msg)
-{
-	HANDLE hEventSource;
-	LPCTSTR lpszStrings[2];
-	TCHAR Buffer[100];
-
-	hEventSource = RegisterEventSource(NULL, TEXT("gpfdist"));
-
-	if( NULL != hEventSource )
-	{
-		memcpy(Buffer, _error_msg, 100);
-
-		lpszStrings[0] = TEXT("gpfdist");
-		lpszStrings[1] = Buffer;
-
-		ReportEvent(hEventSource,        /* event log handle */
-					EVENTLOG_ERROR_TYPE, /* event type */
-					0,                   /* event category */
-					((DWORD)0xC0020100L),           /* event identifier */
-					NULL,                /* no security identifier */
-					2,                   /* size of lpszStrings array */
-					0,                   /* no binary data */
-					lpszStrings,         /* array of strings */
-					NULL);               /* no binary data */
-
-		DeregisterEventSource(hEventSource);
-	}
-}
-
-int verify_buf_size(char** pBuf, const char* _in_val)
-{
-	int val_len, new_len;
-
-	val_len = (int)strlen(_in_val);
-	if (val_len >= CMD_LINE_ARG_SIZE)
-	{
-		new_len = ((val_len+1) >= CMD_LINE_ARG_MAX_SIZE) ? CMD_LINE_ARG_MAX_SIZE : (val_len+1);
-		free(*pBuf);
-		*pBuf = (char*)malloc(new_len);
-		memset(*pBuf, 0, new_len);
-	}
-	else
-	{
-		new_len = val_len;
-	}
-
-
-	return new_len;
-}
-
-void init_cmd_buffer(int argc, const char* const argv[])
-{
-	int i;
-	/* 1. initialize command line params buffer*/
-	for (i = 0; i < CMD_LINE_ARG_NUM; i++)
-	{
-		cmd_line_buffer[i] = (char*)malloc(CMD_LINE_ARG_SIZE);
-		memset(cmd_line_buffer[i], 0, CMD_LINE_ARG_SIZE);
-	}
-
-	/* 2. the number of variables cannot be higher than a
-	 *    a predifined const, that is because - down the line
-	 *    this values get to a const buffer whose size is
-	 *    defined at compile time
-	 */
-	cmd_line_args = (argc <= CMD_LINE_ARG_NUM) ? argc : CMD_LINE_ARG_NUM;
-	if (argc > CMD_LINE_ARG_NUM)
-	{
-		char msg[200] = {0};
-		sprintf(msg, "too many parameters - maximum allowed: %d.", CMD_LINE_ARG_NUM);
-		report_event(TEXT("msg"));
-	}
-
-
-	for (i = 0; i < cmd_line_args; i++)
-	{
-		int len;
-		len = verify_buf_size(&cmd_line_buffer[i], argv[i]);
-		memcpy(cmd_line_buffer[i], argv[i], len);
-	}
-}
-
-void clean_cmd_buffer()
-{
-	int i;
-	for (i = 0; i < CMD_LINE_ARG_NUM; i++)
-	{
-		free(cmd_line_buffer[i]);
-	}
-}
-
-void init_service_status()
-{
-	ServiceStatus.dwServiceType = SERVICE_WIN32;
-	ServiceStatus.dwCurrentState = SERVICE_START_PENDING;
-	ServiceStatus.dwControlsAccepted   =  SERVICE_ACCEPT_STOP | SERVICE_ACCEPT_SHUTDOWN;
-	ServiceStatus.dwWin32ExitCode = 0;
-	ServiceStatus.dwServiceSpecificExitCode = 0;
-	ServiceStatus.dwCheckPoint = 0;
-	ServiceStatus.dwWaitHint = 0;
-}
-
-void do_set_srv_status(DWORD _currentState, DWORD _exitCode)
-{
-	ServiceStatus.dwCurrentState = _currentState;
-	ServiceStatus.dwWin32ExitCode = _exitCode;
-	SetServiceStatus(hStatus, &ServiceStatus);
-}
-
-void init_services_table(SERVICE_TABLE_ENTRY* ServiceTable)
-{
-	ServiceTable[0].lpServiceName = (LPSTR)"gpfdist";
-	ServiceTable[0].lpServiceProc = (LPSERVICE_MAIN_FUNCTIONA)ServiceMain;
-	ServiceTable[1].lpServiceName = (LPSTR)NULL;
-	ServiceTable[1].lpServiceProc = (LPSERVICE_MAIN_FUNCTIONA)NULL;
+void report_event(LPCTSTR _error_msg) {
+  HANDLE hEventSource;
+  LPCTSTR lpszStrings[2];
+  TCHAR Buffer[100];
+
+  hEventSource = RegisterEventSource(NULL, TEXT("gpfdist"));
+
+  if (NULL != hEventSource) {
+    memcpy(Buffer, _error_msg, 100);
+
+    lpszStrings[0] = TEXT("gpfdist");
+    lpszStrings[1] = Buffer;
+
+    ReportEvent(hEventSource,         /* event log handle */
+                EVENTLOG_ERROR_TYPE,  /* event type */
+                0,                    /* event category */
+                ((DWORD)0xC0020100L), /* event identifier */
+                NULL,                 /* no security identifier */
+                2,                    /* size of lpszStrings array */
+                0,                    /* no binary data */
+                lpszStrings,          /* array of strings */
+                NULL);                /* no binary data */
+
+    DeregisterEventSource(hEventSource);
+  }
+}
+
+int verify_buf_size(char** pBuf, const char* _in_val) {
+  int val_len, new_len;
+  char* p;
+
+  val_len = (int)strlen(_in_val);
+  if (val_len >= CMD_LINE_ARG_SIZE) {
+    new_len = ((val_len + 1) >= CMD_LINE_ARG_MAX_SIZE) ? CMD_LINE_ARG_MAX_SIZE
+                                                       : (val_len + 1);
+    p = realloc(*pBuf, new_len);
+    if (p == NULL) return 0;
+    *pBuf = p;
+    memset(*pBuf, 0, new_len);
+  } else {
+    new_len = val_len;
+  }
+
+  return new_len;
+}
+
+void init_cmd_buffer(int argc, const char* const argv[]) {
+  int i;
+  /* 1. initialize command line params buffer*/
+  for (i = 0; i < CMD_LINE_ARG_NUM; i++) {
+    cmd_line_buffer[i] = (char*)malloc(CMD_LINE_ARG_SIZE);
+    if (cmd_line_buffer[i] == NULL) gfatal(NULL, "Out of memory");
+    memset(cmd_line_buffer[i], 0, CMD_LINE_ARG_SIZE);
+  }
+
+  /* 2. the number of variables cannot be higher than a
+   *    a predifined const, that is because - down the line
+   *    this values get to a const buffer whose size is
+   *    defined at compile time
+   */
+  cmd_line_args = (argc <= CMD_LINE_ARG_NUM) ? argc : CMD_LINE_ARG_NUM;
+  if (argc > CMD_LINE_ARG_NUM) {
+    char msg[200] = {0};
+    sprintf(msg, "too many parameters - maximum allowed: %d.",
+            CMD_LINE_ARG_NUM);
+    report_event(TEXT("msg"));
+  }
+
+  for (i = 0; i < cmd_line_args; i++) {
+    int len;
+    len = verify_buf_size(&cmd_line_buffer[i], argv[i]);
+    if (!len) gfatal(NULL, "Out of memory");
+    memcpy(cmd_line_buffer[i], argv[i], len);
+  }
+}
+
+void clean_cmd_buffer() {
+  int i;
+  for (i = 0; i < CMD_LINE_ARG_NUM; i++) {
+    free(cmd_line_buffer[i]);
+  }
+}
+
+void init_service_status() {
+  ServiceStatus.dwServiceType = SERVICE_WIN32;
+  ServiceStatus.dwCurrentState = SERVICE_START_PENDING;
+  ServiceStatus.dwControlsAccepted =
+      SERVICE_ACCEPT_STOP | SERVICE_ACCEPT_SHUTDOWN;
+  ServiceStatus.dwWin32ExitCode = 0;
+  ServiceStatus.dwServiceSpecificExitCode = 0;
+  ServiceStatus.dwCheckPoint = 0;
+  ServiceStatus.dwWaitHint = 0;
+}
+
+void do_set_srv_status(DWORD _currentState, DWORD _exitCode) {
+  ServiceStatus.dwCurrentState = _currentState;
+  ServiceStatus.dwWin32ExitCode = _exitCode;
+  SetServiceStatus(hStatus, &ServiceStatus);
+}
 
+void init_services_table(SERVICE_TABLE_ENTRY* ServiceTable) {
+  ServiceTable[0].lpServiceName = (LPSTR) "gpfdist";
+  ServiceTable[0].lpServiceProc = (LPSERVICE_MAIN_FUNCTIONA)ServiceMain;
+  ServiceTable[1].lpServiceName = (LPSTR)NULL;
+  ServiceTable[1].lpServiceProc = (LPSERVICE_MAIN_FUNCTIONA)NULL;
 }
 /* HELPERS - STOP */
 
-int main(int argc, const char* const argv[])
-{
-	int main_ret = 0, srv_ret;
-
-	/*
-	 * 1. command line parameters transfer to a global buffer - for ServiceMain
-	 */
-	init_cmd_buffer(argc, argv);
-
-	/*
-	 * 2. services table init
-	 */
-	SERVICE_TABLE_ENTRY ServiceTable[2];
-	init_services_table(ServiceTable);
-
-	/*
-	 * 3. Start the control dispatcher thread for our service
-	 */
-	srv_ret = StartServiceCtrlDispatcher(ServiceTable);
-	if (0 == srv_ret) /* program is being run as a Windows console application */
-	{
-		gpfdist_init(argc, argv);
-		main_ret = gpfdist_run();
-	}
-
-
-	return main_ret;
-}
-
-void ServiceMain(int argc, char** argv)
-{
-	int error = 0;
-	init_service_status();
-
-	hStatus = RegisterServiceCtrlHandler((LPCSTR)"gpfdist", (LPHANDLER_FUNCTION)ControlHandler);
-	if (hStatus == (SERVICE_STATUS_HANDLE)0)
-	{
-		/*
-		 * Registering Control Handler failed
-		 */
-		return;
-	}
-	/*
-	 * Initialize Service
-	 * If we don't pass a const char* const [], to gpfdist_init
-	 * we will get a warning that will fail the build
-	 */
-	const char* const buf[CMD_LINE_ARG_NUM] = {
-		cmd_line_buffer[0],
-		cmd_line_buffer[1],
-		cmd_line_buffer[2],
-		cmd_line_buffer[3],
-		cmd_line_buffer[4],
-		cmd_line_buffer[5],
-		cmd_line_buffer[6],
-		cmd_line_buffer[7],
-		cmd_line_buffer[8],
-		cmd_line_buffer[9],
-		cmd_line_buffer[10],
-		cmd_line_buffer[11],
-		cmd_line_buffer[12],
-		cmd_line_buffer[13],
-		cmd_line_buffer[14],
-		cmd_line_buffer[15],
-		cmd_line_buffer[16],
-		cmd_line_buffer[17],
-		cmd_line_buffer[18],
-		cmd_line_buffer[19],
-		cmd_line_buffer[20],
-		cmd_line_buffer[21],
-		cmd_line_buffer[22],
-		cmd_line_buffer[23],
-		cmd_line_buffer[24],
-		cmd_line_buffer[25],
-		cmd_line_buffer[26],
-		cmd_line_buffer[27],
-		cmd_line_buffer[28],
-		cmd_line_buffer[29],
-		cmd_line_buffer[30],
-		cmd_line_buffer[31],
-		cmd_line_buffer[32],
-		cmd_line_buffer[33],
-		cmd_line_buffer[34],
-		cmd_line_buffer[35],
-		cmd_line_buffer[36],
-		cmd_line_buffer[37],
-		cmd_line_buffer[38],
-		cmd_line_buffer[39]
-	};
-	error = gpfdist_init(cmd_line_args, buf);
-	if (error != 0)
-	{
-		/*
-		 * Initialization failed
-		 */
-		do_set_srv_status(SERVICE_STOPPED, -1);
-		return;
-	}
-	else
-	{
-		do_set_srv_status(SERVICE_RUNNING, 0);
-	}
-
-	/*
-	 * free the command line arguments buffer - it's not used anymore
-	 */
-	clean_cmd_buffer();
-
-	/*
-	 * actual service work
-	 */
-	gpfdist_run();
-}
-
-void ControlHandler(DWORD request)
-{
-	switch(request)
-	{
-		case SERVICE_CONTROL_STOP:
-		case SERVICE_CONTROL_SHUTDOWN:
-		{
-			do_set_srv_status(SERVICE_STOPPED, 0);
-			return;
-		}
-
-		default:
-			break;
-	}
-
-	/*
-	 * Report current status
-	 */
-	do_set_srv_status(SERVICE_RUNNING, 0);
-}
-#endif
-
-#define find_max(a,b) ((a) >= (b) ? (a) : (b))
-
-static SSL_CTX *initialize_ctx(void)
-{
-	int			stringSize;
-	char 		*fileName, slash;
-	SSL_CTX 	*ctx;
-
-	if (!gcb.bio_err){
-		/* Global system initialization*/
-		SSL_library_init();
-		SSL_load_error_strings();
-		/* An error write context */
-		gcb.bio_err=BIO_new_fp(stderr, BIO_NOCLOSE);
-	}
-
-	/* Create our context*/
-	ctx = SSL_CTX_new( TLSv1_server_method() );
-
-	/* Generate random seed */
-	if ( RAND_poll() == 0 )
-		gfatal(FLINE,"Can't generate random seed for SSL");
-
-	/* The size of the string will consist of the path and the filename (the longest one) */
-	// +1 for the '/' charachter (/filename)
-	// +1 for the \0,
-	stringSize = find_max( strlen(CertificateFilename), find_max(strlen(PrivateKeyFilename),strlen(TrustedCaFilename)) ) + strlen(opt.ssl) + 2;
-	/* Allocate the memory for the file name */
-	fileName = (char *) calloc( (stringSize), sizeof(char) );
-	if ( fileName == NULL )
-	{
-		gfatal (FLINE,"Unable to allocate memory for SSL initialization");
-	}
+int main(int argc, const char* const argv[]) {
+  int main_ret = 0, srv_ret;
+
+  /*
+   * 1. command line parameters transfer to a global buffer - for ServiceMain
+   */
+  init_cmd_buffer(argc, argv);
+
+  /*
+   * 2. services table init
+   */
+  SERVICE_TABLE_ENTRY ServiceTable[2];
+  init_services_table(ServiceTable);
+
+  /*
+   * 3. Start the control dispatcher thread for our service
+   */
+  srv_ret = StartServiceCtrlDispatcher(ServiceTable);
+  if (0 == srv_ret) /* program is being run as a Windows console application */
+  {
+    if (gpfdist_init(argc, argv) == -1) gfatal(NULL, "Initialization failed");
+    main_ret = gpfdist_run();
+  }
+
+  return main_ret;
+}
+
+void ServiceMain(int argc, char** argv) {
+  int error = 0;
+  init_service_status();
+
+  hStatus = RegisterServiceCtrlHandler((LPCSTR) "gpfdist",
+                                       (LPHANDLER_FUNCTION)ControlHandler);
+  if (hStatus == (SERVICE_STATUS_HANDLE)0) {
+    /*
+     * Registering Control Handler failed
+     */
+    return;
+  }
+  /*
+   * Initialize Service
+   * If we don't pass a const char* const [], to gpfdist_init
+   * we will get a warning that will fail the build
+   */
+  const char* const buf[CMD_LINE_ARG_NUM] = {
+      cmd_line_buffer[0],  cmd_line_buffer[1],  cmd_line_buffer[2],
+      cmd_line_buffer[3],  cmd_line_buffer[4],  cmd_line_buffer[5],
+      cmd_line_buffer[6],  cmd_line_buffer[7],  cmd_line_buffer[8],
+      cmd_line_buffer[9],  cmd_line_buffer[10], cmd_line_buffer[11],
+      cmd_line_buffer[12], cmd_line_buffer[13], cmd_line_buffer[14],
+      cmd_line_buffer[15], cmd_line_buffer[16], cmd_line_buffer[17],
+      cmd_line_buffer[18], cmd_line_buffer[19], cmd_line_buffer[20],
+      cmd_line_buffer[21], cmd_line_buffer[22], cmd_line_buffer[23],
+      cmd_line_buffer[24], cmd_line_buffer[25], cmd_line_buffer[26],
+      cmd_line_buffer[27], cmd_line_buffer[28], cmd_line_buffer[29],
+      cmd_line_buffer[30], cmd_line_buffer[31], cmd_line_buffer[32],
+      cmd_line_buffer[33], cmd_line_buffer[34], cmd_line_buffer[35],
+      cmd_line_buffer[36], cmd_line_buffer[37], cmd_line_buffer[38],
+      cmd_line_buffer[39]};
+  error = gpfdist_init(cmd_line_args, buf);
+  if (error != 0) {
+    /*
+     * Initialization failed
+     */
+    do_set_srv_status(SERVICE_STOPPED, -1);
+    return;
+  } else {
+    do_set_srv_status(SERVICE_RUNNING, 0);
+  }
+
+  /*
+   * free the command line arguments buffer - it's not used anymore
+   */
+  clean_cmd_buffer();
+
+  /*
+   * actual service work
+   */
+  gpfdist_run();
+}
+
+void ControlHandler(DWORD request) {
+  switch (request) {
+    case SERVICE_CONTROL_STOP:
+    case SERVICE_CONTROL_SHUTDOWN: {
+      do_set_srv_status(SERVICE_STOPPED, 0);
+      return;
+    }
+
+    default:
+      break;
+  }
+
+  /*
+   * Report current status
+   */
+  do_set_srv_status(SERVICE_RUNNING, 0);
+}
+#endif
+
+#define find_max(a, b) ((a) >= (b) ? (a) : (b))
+
+#ifdef USE_SSL
+static SSL_CTX* initialize_ctx(void) {
+  int stringSize;
+  char *fileName, slash;
+  SSL_CTX* ctx;
+
+  if (!gcb.bio_err) {
+    /* Global system initialization*/
+    SSL_library_init();
+    SSL_load_error_strings();
+    /* An error write context */
+    gcb.bio_err = BIO_new_fp(stderr, BIO_NOCLOSE);
+  }
+
+  /* Create our context*/
+  ctx = SSL_CTX_new(TLSv1_server_method());
+
+  /* Generate random seed */
+  if (RAND_poll() == 0) gfatal(NULL, "Can't generate random seed for SSL");
+
+  /*
+   * The size of the string will consist of the path and the filename (the
+   * longest one)
+   * +1 for the '/' character (/filename)
+   * +1 for the \0
+   */
+  stringSize = find_max(strlen(CertificateFilename),
+                        find_max(strlen(PrivateKeyFilename),
+                                 strlen(TrustedCaFilename))) +
+               strlen(opt.ssl) + 2;
+  /* Allocate the memory for the file name */
+  fileName = (char*)calloc((stringSize), sizeof(char));
+  if (fileName == NULL)
+    gfatal(NULL, "Unable to allocate memory for SSL initialization");
 
 #ifdef WIN32
-	slash = '\\';
+  slash = '\\';
 #else
-	slash = '/';
-#endif
-
-	/* Copy the path + the filename */
-	snprintf(fileName,stringSize,"%s%c%s",opt.ssl,slash,CertificateFilename);
-
-	/* Load our keys and certificates*/
-	if (!(SSL_CTX_use_certificate_chain_file(ctx,fileName)))
-	{
-		gfatal(FLINE,"Unable to load the certificate from file: \"%s\"", fileName);
-	}
-	else
-	{
-		if ( opt.v )
-		{
-			gprint("The certificate was successfully loaded from \"%s\"\n",fileName);
-		}
-	}
-
-	/* Copy the path + the filename */
-	snprintf(fileName,stringSize,"%s%c%s",opt.ssl,slash,PrivateKeyFilename);
-
-	if (!(SSL_CTX_use_PrivateKey_file(ctx,fileName,SSL_FILETYPE_PEM)))
-	{
-		gfatal (FLINE,"Unable to load the private key from file: \"%s\"", fileName);
-	}
-	else
-	{
-		if ( opt.v )
-		{
-			gprint("The private key was successfully loaded from \"%s\"\n",fileName);
-		}
-	}
-
-	/* Copy the path + the filename */
-	snprintf(fileName,stringSize,"%s%c%s",opt.ssl,slash,TrustedCaFilename);
-
-	/* Load the CAs we trust*/
-	if (!(SSL_CTX_load_verify_locations(ctx, fileName,0)))
-	{
-		gfatal (FLINE,"Unable to to load CA from file: \"%s\"", fileName);
-	}
-	else
-	{
-		if ( opt.v )
-		{
-			gprint("The CA file successfully loaded from \"%s\"\n",fileName);
-		}
-	}
-
-	/* Set the verification flags for ctx 	*/
-	/* We always require client certificate	*/
-	SSL_CTX_set_verify(ctx, SSL_VERIFY_PEER | SSL_VERIFY_FAIL_IF_NO_PEER_CERT, 0);
-
-	/* Consider using these - experinments on Mac showed no improvement,
-	 * but perhaps it will on other platforms, or when opt.m is very big
-	 */
-	//SSL_CTX_set_mode(ctx, SSL_MODE_AUTO_RETRY | SSL_MODE_ENABLE_PARTIAL_WRITE);
-
-	free(fileName);
-	return ctx;
-}
+  slash = '/';
+#endif
+
+  /* Copy the path + the filename */
+  snprintf(fileName, stringSize, "%s%c%s", opt.ssl, slash, CertificateFilename);
+
+  /* Load our keys and certificates*/
+  if (!(SSL_CTX_use_certificate_chain_file(ctx, fileName))) {
+    gfatal(NULL, "Unable to load the certificate from file: \"%s\"", fileName);
+  } else {
+    if (opt.v) {
+      gprint(NULL, "The certificate was successfully loaded from \"%s\"\n",
+             fileName);
+    }
+  }
 
+  /* Copy the path + the filename */
+  snprintf(fileName, stringSize, "%s%c%s", opt.ssl, slash, PrivateKeyFilename);
+
+  if (!(SSL_CTX_use_PrivateKey_file(ctx, fileName, SSL_FILETYPE_PEM))) {
+    gfatal(NULL, "Unable to load the private key from file: \"%s\"", fileName);
+  } else {
+    if (opt.v) {
+      gprint(NULL, "The private key was successfully loaded from \"%s\"\n",
+             fileName);
+    }
+  }
+
+  /* Copy the path + the filename */
+  snprintf(fileName, stringSize, "%s%c%s", opt.ssl, slash, TrustedCaFilename);
+
+  /* Load the CAs we trust*/
+  if (!(SSL_CTX_load_verify_locations(ctx, fileName, 0))) {
+    gfatal(NULL, "Unable to to load CA from file: \"%s\"", fileName);
+  } else {
+    if (opt.v) {
+      gprint(NULL, "The CA file successfully loaded from \"%s\"\n", fileName);
+    }
+  }
+
+  /* Set the verification flags for ctx 	*/
+  /* We always require client certificate	*/
+  SSL_CTX_set_verify(ctx, SSL_VERIFY_PEER | SSL_VERIFY_FAIL_IF_NO_PEER_CERT, 0);
+
+  /* Consider using these - experinments on Mac showed no improvement,
+   * but perhaps it will on other platforms, or when opt.m is very big
+   */
+  // SSL_CTX_set_mode(ctx, SSL_MODE_AUTO_RETRY | SSL_MODE_ENABLE_PARTIAL_WRITE);
+
+  free(fileName);
+  return ctx;
+}
+#endif
 
 /*
  * gpfdist_socket_send
@@ -3412,113 +3768,121 @@ static SSL_CTX *initialize_ctx(void)
  * Sends the requested buf, of size buflen to the network
  * via appropriate socket
  */
-static int gpfdist_socket_send(const request_t *r, const void *buf, const size_t buflen)
-{
-	return send(r->sock, buf, buflen, 0);
+static int gpfdist_socket_send(const request_t* r, const void* buf,
+                               const size_t buflen) {
+  return send(r->sock, buf, buflen, 0);
 }
 
-
+#ifdef USE_SSL
 /*
  * gpfdist_SSL_send
  *
  * Sends the requested buf, of size len to the network via SSL
  */
-static int gpfdist_SSL_send(const request_t *r, const void *buf, const size_t buflen)
-{
-
-	/* Write the data to socket */
-	int n = BIO_write(r->io, buf, buflen);
-	/* Try to flush */
-	(void)BIO_flush(r->io);
-
-	/* If we could not write to BIO */
-	if ( n < 0)
-	{
-		/* If BIO indicates retry => we should retry later, this is not an error */
-		if ( BIO_should_retry(r->io) > 1 )
-		{
-			/* Do not indicate error */
-			n = 0;
-		}
-		else
-		{
-			/* If errno == 0 => this is not a real error */
-			if ( errno == 0 )
-			{
-				/* Do not indicate error */
-				n = 0;
-			}
-			else
-			{
-				/* If errno == EPIPE, it means that the client has closed the connection   	*/
-				/* This error will be handled in the calling function, do not print it here	*/
-				if (errno != EPIPE)
-				{
-					gwarning(FLINE, "Error during SSL gpfdist_send (Error = %d. errno = %d)", SSL_get_error(r->ssl,n), (int)errno);
-					ERR_print_errors(gcb.bio_err);
-				}
-			}
-		}
-	}
-
-	return n;
-}
+static int gpfdist_SSL_send(const request_t* r, const void* buf,
+                            const size_t buflen) {
+  /* Write the data to socket */
+  int n = BIO_write(r->io, buf, buflen);
+  /* Try to flush */
+  (void)BIO_flush(r->io);
+
+  /* If we could not write to BIO */
+  if (n < 0) {
+    /* If BIO indicates retry => we should retry later, this is not an error */
+    if (BIO_should_retry(r->io) > 1) {
+      /* Do not indicate error */
+      n = 0;
+    } else {
+      /* If errno == 0 => this is not a real error */
+      if (errno == 0) {
+        /* Do not indicate error */
+        n = 0;
+      } else {
+        /* If errno == EPIPE, it means that the client has closed the connection
+         */
+        /* This error will be handled in the calling function, do not print it
+         * here	*/
+        if (errno != EPIPE) {
+          gwarning(r, "Error during SSL gpfdist_send (Error = %d. errno = %d)",
+                   SSL_get_error(r->ssl, n), (int)errno);
+          ERR_print_errors(gcb.bio_err);
+        }
+      }
+    }
+  }
 
+  return n;
+}
+#endif
 
 /*
  * gpfdist_socket_receive
  *
  * read from a socket
  */
-static int gpfdist_socket_receive(const request_t *r, void *buf, const size_t buflen)
-{
-	return ( recv(r->sock, buf, buflen, 0) );
+static int gpfdist_socket_receive(const request_t* r, void* buf,
+                                  const size_t buflen) {
+  return (recv(r->sock, buf, buflen, 0));
 }
 
+/*
+ * request_shutdown_sock
+ *
+ * Shutdown request socket transmission.
+ */
+static void request_shutdown_sock(const request_t* r) {
+  int ret = shutdown(r->sock, SHUT_WR);
+  if (ret == 0) {
+    gprintlnif(r, "successfully shutdown socket");
+  } else {
+    gprintln(r, "failed to shutdown socket, errno: %d, msg: %s", errno,
+             strerror(errno));
+  }
+}
 
+#ifdef USE_SSL
 /*
  * gpfdist_SSL_receive
  *
  * read from an SSL socket
  */
-static int gpfdist_SSL_receive(const request_t *r, void *buf, const size_t buflen)
-{
-	return ( BIO_read(r->io, buf, buflen) );
-	/* todo: add error checks here */
+static int gpfdist_SSL_receive(const request_t* r, void* buf,
+                               const size_t buflen) {
+  return (BIO_read(r->io, buf, buflen));
+  /* todo: add error checks here */
 }
 
-
 /*
  * free_SSL_resources
  *
  * Frees all SSL resources that were allocated per request r
  */
-static void free_SSL_resources(const request_t *r)
-{
-	BIO_ssl_shutdown(r->sbio);
-	BIO_vfree(r->io);
-	BIO_vfree(r->sbio);
-	//BIO_vfree(r->ssl_bio);
-	SSL_free(r->ssl);
+static void free_SSL_resources(const request_t* r) {
+  // send close_notify to client
+  SSL_shutdown(r->ssl);  // or BIO_ssl_shutdown(r->ssl_bio);
 
-}
+  request_shutdown_sock(r);
 
+  BIO_vfree(r->io);  // ssl_bio is pushed to r->io list, so ssl_bio is freed
+                     // too.
+  BIO_vfree(r->sbio);
+  // BIO_vfree(r->ssl_bio);
+  SSL_free(r->ssl);
+}
 
 /*
  * handle_ssl_error
  *
  * Frees SSL resources that were allocated during do_accept
  */
-static void handle_ssl_error(SOCKET sock, BIO *sbio, SSL *ssl)
-{
-	gwarning(FLINE, "SSL accept failed");
-	if (opt.v)
-	{
-		ERR_print_errors(gcb.bio_err);
-	}
-
-	BIO_ssl_shutdown(sbio);
-	SSL_free(ssl);
+static void handle_ssl_error(SOCKET sock, BIO* sbio, SSL* ssl) {
+  gwarning(NULL, "SSL accept failed");
+  if (opt.v) {
+    ERR_print_errors(gcb.bio_err);
+  }
+
+  SSL_shutdown(ssl);
+  SSL_free(ssl);
 }
 
 /*
@@ -3526,84 +3890,207 @@ static void handle_ssl_error(SOCKET sock, BIO *sbio, SSL *ssl)
  *
  * Flush all the data that is still pending in the current buffer
  */
-static void flush_ssl_buffer(int fd, short event, void* arg)
-{
-	request_t* r = (request_t*)arg;
+static void flush_ssl_buffer(int fd, short event, void* arg) {
+  request_t* r = (request_t*)arg;
 
-	(void)BIO_flush(r->io);
+  (void)BIO_flush(r->io);
 
-	if ( event & EV_TIMEOUT )
-	{
-		gwarning(FLINE,"Buffer flush timeout");
-	}
+  if (event & EV_TIMEOUT) {
+    gwarning(r, "Buffer flush timeout");
+  }
 
-	if ( BIO_wpending(r->io) )
-	{
-		setup_flush_ssl_buffer(r);
-	}
-	else
-	{
-		// Prepare and start a 5 seconds timer.
-		// While working with BIO_SSL, we are working with 3 buffers:
-		// [1] BIO layer buffer [2] SSL buffer [3] Socket buffer
-		// Sometimes we have a delay somewhere between these buffers (MPP-16402)
-		// So, even if the BIO_wpending() shows that there is no more pending data in the BIO_SSL buffer,
-		// we might still have data in the socket's buffer that wasn't sent yet.
-
-		TR(("[%s] [%d] SSL cleanup started\n", datetime(), r->sock));
-
-		event_del(&r->ev);
-		evtimer_set(&r->ev, request_cleanup_and_free_SSL_resources, r);
-		r->tm.tv_sec  = opt.sslclean;
-		r->tm.tv_usec = 0;
-		(void)evtimer_add(&r->ev, &r->tm);
-	}
+  if (BIO_wpending(r->io)) {
+    setup_flush_ssl_buffer(r);
+  } else {
+    // Do ssl cleanup immediately.
+    request_cleanup_and_free_SSL_resources(r);
+  }
 }
 
-
 /*
  * setup_flush_ssl_buffer
  *
  * Create event that will call to 'flush_ssl_buffer', with 5 seconds timeout
  */
-static void setup_flush_ssl_buffer(request_t* r)
-{
-	event_del(&r->ev);
-	event_set(&r->ev, r->sock, EV_WRITE, flush_ssl_buffer, r);
-	r->tm.tv_sec  = 5;
-	r->tm.tv_usec = 0;
-	(void)event_add(&r->ev, &r->tm);
+static void setup_flush_ssl_buffer(request_t* r) {
+  event_del(&r->ev);
+  event_set(&r->ev, r->sock, EV_WRITE, flush_ssl_buffer, r);
+  r->tm.tv_sec = 5;
+  r->tm.tv_usec = 0;
+  (void)event_add(&r->ev, &r->tm);
+}
+#endif
+
+/*
+ * log unsent/unacked bytes in socket buffer.
+ */
+static int get_unsent_bytes(request_t* r) {
+  int unsent_bytes = -1;
+#ifdef __linux__
+  int ret = ioctl(r->sock, TIOCOUTQ, &unsent_bytes);
+  if (ret < 0) {
+    gwarning(r, "failed to use ioctl to get unsent bytes");
+  }
+#endif
+  return unsent_bytes;
+}
+
+static void log_unsent_bytes(request_t* r) {
+  gprintlnif(r, "unsent bytes: %d (-1 means not supported)",
+             get_unsent_bytes(r));
 }
 
+/*
+ * call close after timeout or EV_READ ready.
+ */
+static void do_close(int fd, short event, void* arg) {
+  request_t* r = (request_t*)arg;
+  char buffer[256] = {0};
+
+  if (event & EV_TIMEOUT) {
+    gwarning(r,
+             "gpfdist shutdown the connection, while have not received "
+             "response from segment");
+  }
+
+  int ret = recv(r->sock, buffer, sizeof(buffer) - 1, 0);
+  if (ret < 0) {
+    gwarning(r, "gpfdist read error after shutdown. errno: %d, msg: %s", errno,
+             strerror(errno));
+
+#ifdef WIN32
+    int e = WSAGetLastError();
+    bool should_retry = (e == WSAEINTR || e == WSAEWOULDBLOCK);
+#else
+    int e = errno;
+    bool should_retry = (e == EINTR || e == EAGAIN);
+#endif
+    if (should_retry) {
+      setup_do_close(r);
+      return;
+    }
+  } else if (ret == 0) {
+    gprintlnif(r, "peer closed after gpfdist shutdown");
+  } else {
+    gwarning(r, "gpfdist read unexpected data after shutdown %s", buffer);
+  }
+
+  log_unsent_bytes(r);
+
+  ret = closesocket(r->sock);
+  if (ret == 0) {
+    gprintlnif(r, "successfully closed socket");
+  } else {
+    gwarning(r, "failed to close socket. errno: %d, msg: %s", errno,
+             strerror(errno));
+  }
+
+  event_del(&r->ev);
+  r->sock = -1;
+  apr_pool_destroy(r->pool);
+
+  fflush(stdout);
+}
 
 /*
  * request_cleanup
  *
  * Cleanup request related resources
  */
-static void request_cleanup(request_t *r)
-{
-	event_del(&r->ev);
-	shutdown(r->sock, SHUT_WR | SHUT_RD);
-	closesocket(r->sock);
-	r->sock = -1;
-	apr_pool_destroy(r->pool);
+static void request_cleanup(request_t* r) {
+  request_shutdown_sock(r);
+  setup_do_close(r);
 }
 
+static void setup_do_close(request_t* r) {
+  event_del(&r->ev);
+  event_set(&r->ev, r->sock, EV_READ, do_close, r);
+
+  r->tm.tv_sec = 60;
+  r->tm.tv_usec = 0;
+  if (0 != event_add(&r->ev, &r->tm)) {
+    gfatal(r, "failed to event_add!");
+  }
... 715 lines suppressed ...