[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-04 Thread lavjain
Github user lavjain commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r200169586
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -89,32 +188,103 @@ public Boolean run() throws IOException, 
ServletException {
 };
 
 // create proxy user UGI from the UGI of the logged in user 
and execute the servlet chain as that user
-UserGroupInformation proxyUGI = null;
+TimedProxyUGI timedProxyUGI = getTimedProxyUGI(user, session);
 try {
-LOG.debug("Creating proxy user = " + user);
-proxyUGI = UserGroupInformation.createProxyUser(user, 
UserGroupInformation.getLoginUser());
-proxyUGI.doAs(action);
+timedProxyUGI.proxyUGI.doAs(action);
 } catch (UndeclaredThrowableException ute) {
 // unwrap the real exception thrown by the action
 throw new ServletException(ute.getCause());
 } catch (InterruptedException ie) {
 throw new ServletException(ie);
-} finally {
-try {
-if (proxyUGI != null) {
-LOG.debug("Closing FileSystem for proxy user = " + 
proxyUGI.getUserName());
-FileSystem.closeAllForUGI(proxyUGI);
-}
-} catch (Throwable t) {
-LOG.warn("Error closing FileSystem for proxy user = " 
+ proxyUGI.getUserName());
-}
+}
+finally {
+release(timedProxyUGI, fragmentIndex, fragmentCount);
 }
 } else {
 // no user impersonation is configured
 chain.doFilter(request, response);
 }
 }
 
+   private TimedProxyUGI getTimedProxyUGI(String user, 
SegmentTransactionId session) throws IOException {
+synchronized (session.segmentTransactionId.intern()) {
+TimedProxyUGI timedProxyUGI = cache.get(session);
+if (timedProxyUGI == null || timedProxyUGI.getDelayMillis() < 
0) {
+cleanup();
+LOG.info(session.toString() + " Creating proxy user = " + 
user);
+UserGroupInformation proxyUGI =
+UserGroupInformation.createProxyUser(user, 
UserGroupInformation.getLoginUser());
+timedProxyUGI = new TimedProxyUGI(proxyUGI, session);
+delayQueue.offer(timedProxyUGI);
+cache.put(session, timedProxyUGI);
+} else {
+timedProxyUGI.incrementCounter();
+}
+return timedProxyUGI;
+}
+}
+
+private void release(TimedProxyUGI timedProxyUGI, Integer 
fragmentIndex, Integer fragmentCount) {
+synchronized (timedProxyUGI.session.segmentTransactionId.intern()) 
{
+timedProxyUGI.resetTime();
+timedProxyUGI.decrementCounter();
+if (fragmentIndex != null && 
fragmentCount.equals(fragmentIndex))
+closeUGI(timedProxyUGI);
+}
+}
+
+private void cleanup() {
+TimedProxyUGI timedProxyUGI = delayQueue.poll();
+while (timedProxyUGI != null) {
+closeUGI(timedProxyUGI);
+LOG.info(timedProxyUGI.session.toString() + " Delay Queue Size 
= " + delayQueue.size());
+timedProxyUGI = delayQueue.poll();
+}
+}
+
+private void closeUGI(TimedProxyUGI timedProxyUGI) {
+synchronized (timedProxyUGI.session.segmentTransactionId.intern()) 
{
--- End diff --

Good catch. The locking should only operate on one lock at a time. This 
could be accomplished if the clean method is called by a reaper thread.


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-04 Thread lavjain
Github user lavjain commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r200168483
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -89,32 +182,98 @@ public Boolean run() throws IOException, 
ServletException {
 };
 
 // create proxy user UGI from the UGI of the logged in user 
and execute the servlet chain as that user
-UserGroupInformation proxyUGI = null;
+TimedProxyUGI timedProxyUGI = getTimedProxyUGI(user, session);
 try {
-LOG.debug("Creating proxy user = " + user);
-proxyUGI = UserGroupInformation.createProxyUser(user, 
UserGroupInformation.getLoginUser());
-proxyUGI.doAs(action);
+timedProxyUGI.proxyUGI.doAs(action);
 } catch (UndeclaredThrowableException ute) {
 // unwrap the real exception thrown by the action
 throw new ServletException(ute.getCause());
 } catch (InterruptedException ie) {
 throw new ServletException(ie);
-} finally {
-try {
-if (proxyUGI != null) {
-LOG.debug("Closing FileSystem for proxy user = " + 
proxyUGI.getUserName());
-FileSystem.closeAllForUGI(proxyUGI);
-}
-} catch (Throwable t) {
-LOG.warn("Error closing FileSystem for proxy user = " 
+ proxyUGI.getUserName());
-}
+}
+finally {
+release(timedProxyUGI, fragmentIndex, fragmentCount);
 }
 } else {
 // no user impersonation is configured
 chain.doFilter(request, response);
 }
 }
 
+   private TimedProxyUGI getTimedProxyUGI(String user, 
SegmentTransactionId session) throws IOException {
+synchronized (session.segmentTransactionId.intern()) {
--- End diff --

Agreed. Using the lock on interned string helped me to test the logic 
without any explicit lock management.


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-03 Thread denalex
Github user denalex commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199975945
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -89,32 +188,103 @@ public Boolean run() throws IOException, 
ServletException {
 };
 
 // create proxy user UGI from the UGI of the logged in user 
and execute the servlet chain as that user
-UserGroupInformation proxyUGI = null;
+TimedProxyUGI timedProxyUGI = getTimedProxyUGI(user, session);
 try {
-LOG.debug("Creating proxy user = " + user);
-proxyUGI = UserGroupInformation.createProxyUser(user, 
UserGroupInformation.getLoginUser());
-proxyUGI.doAs(action);
+timedProxyUGI.proxyUGI.doAs(action);
 } catch (UndeclaredThrowableException ute) {
 // unwrap the real exception thrown by the action
 throw new ServletException(ute.getCause());
 } catch (InterruptedException ie) {
 throw new ServletException(ie);
-} finally {
-try {
-if (proxyUGI != null) {
-LOG.debug("Closing FileSystem for proxy user = " + 
proxyUGI.getUserName());
-FileSystem.closeAllForUGI(proxyUGI);
-}
-} catch (Throwable t) {
-LOG.warn("Error closing FileSystem for proxy user = " 
+ proxyUGI.getUserName());
-}
+}
+finally {
+release(timedProxyUGI, fragmentIndex, fragmentCount);
 }
 } else {
 // no user impersonation is configured
 chain.doFilter(request, response);
 }
 }
 
+   private TimedProxyUGI getTimedProxyUGI(String user, 
SegmentTransactionId session) throws IOException {
+synchronized (session.segmentTransactionId.intern()) {
+TimedProxyUGI timedProxyUGI = cache.get(session);
+if (timedProxyUGI == null || timedProxyUGI.getDelayMillis() < 
0) {
+cleanup();
+LOG.info(session.toString() + " Creating proxy user = " + 
user);
+UserGroupInformation proxyUGI =
+UserGroupInformation.createProxyUser(user, 
UserGroupInformation.getLoginUser());
+timedProxyUGI = new TimedProxyUGI(proxyUGI, session);
+delayQueue.offer(timedProxyUGI);
+cache.put(session, timedProxyUGI);
+} else {
+timedProxyUGI.incrementCounter();
+}
+return timedProxyUGI;
+}
+}
+
+private void release(TimedProxyUGI timedProxyUGI, Integer 
fragmentIndex, Integer fragmentCount) {
+synchronized (timedProxyUGI.session.segmentTransactionId.intern()) 
{
+timedProxyUGI.resetTime();
+timedProxyUGI.decrementCounter();
+if (fragmentIndex != null && 
fragmentCount.equals(fragmentIndex))
+closeUGI(timedProxyUGI);
+}
+}
+
+private void cleanup() {
+TimedProxyUGI timedProxyUGI = delayQueue.poll();
+while (timedProxyUGI != null) {
+closeUGI(timedProxyUGI);
+LOG.info(timedProxyUGI.session.toString() + " Delay Queue Size 
= " + delayQueue.size());
+timedProxyUGI = delayQueue.poll();
+}
+}
+
+private void closeUGI(TimedProxyUGI timedProxyUGI) {
+synchronized (timedProxyUGI.session.segmentTransactionId.intern()) 
{
--- End diff --

this can deadlock -- if 2 cleanup attempts happen at the same time for 
session A and session B -- session A comes first and gets a lock on, say, 
"seg1:tx1" in release() method and then comes here and tries to get a lock on 
session B "seg2:tx1", while session B would get that loc already and will, in 
turn try to get lock on session A. This might happen when new requests for both 
sessions arrive simultaneously after a delay when their previous UGIs have 
expired. Granted the probability of this is low, but non-zero, meaning this is 
dangerous and cannot be ignored. In other words, to prevent deadlocks, do not 
acquire new locks while holding other locks if another thread can do the same 
in reverse order, which is exactly the case here.


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-03 Thread denalex
Github user denalex commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199952336
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -89,32 +182,98 @@ public Boolean run() throws IOException, 
ServletException {
 };
 
 // create proxy user UGI from the UGI of the logged in user 
and execute the servlet chain as that user
-UserGroupInformation proxyUGI = null;
+TimedProxyUGI timedProxyUGI = getTimedProxyUGI(user, session);
 try {
-LOG.debug("Creating proxy user = " + user);
-proxyUGI = UserGroupInformation.createProxyUser(user, 
UserGroupInformation.getLoginUser());
-proxyUGI.doAs(action);
+timedProxyUGI.proxyUGI.doAs(action);
 } catch (UndeclaredThrowableException ute) {
 // unwrap the real exception thrown by the action
 throw new ServletException(ute.getCause());
 } catch (InterruptedException ie) {
 throw new ServletException(ie);
-} finally {
-try {
-if (proxyUGI != null) {
-LOG.debug("Closing FileSystem for proxy user = " + 
proxyUGI.getUserName());
-FileSystem.closeAllForUGI(proxyUGI);
-}
-} catch (Throwable t) {
-LOG.warn("Error closing FileSystem for proxy user = " 
+ proxyUGI.getUserName());
-}
+}
+finally {
+release(timedProxyUGI, fragmentIndex, fragmentCount);
 }
 } else {
 // no user impersonation is configured
 chain.doFilter(request, response);
 }
 }
 
+   private TimedProxyUGI getTimedProxyUGI(String user, 
SegmentTransactionId session) throws IOException {
+synchronized (session.segmentTransactionId.intern()) {
--- End diff --

synchronizing on interned strings might not be a good practice 
(https://stackoverflow.com/questions/133988/synchronizing-on-string-objects-in-java),
 basically due to:
- possibilities that other logic elsewhere does it as well and creates 
possibility of deadlock
- intern() has cost and might require internal synchronization
- static pool of interned instances will grow and then GC will reclaim 
them, so unclear lifecycle of objects
It would seem preferable to use local object we have full control over


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread shivzone
Github user shivzone commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199593983
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
+private static DelayQueue delayQueue = new 
DelayQueue<>();
+private static long UGI_CACHE_EXPIRY = 15 * 60 * 1000L; // 15 Minutes
+
+
+private static class TimedProxyUGI implements Delayed {
+long startTime;
+UserGroupInformation proxyUGI;
+SegmentTransactionId session;
+
+public TimedProxyUGI(UserGroupInformation proxyUGI, 
SegmentTransactionId session) {
+this.startTime = System.currentTimeMillis();
+this.proxyUGI = proxyUGI;
+this.session = session;
+}
+
+@Override
+public long getDelay(TimeUnit unit) {
+return unit.convert(getDelayMillis(), TimeUnit.MILLISECONDS);
+}
+
+@Override
+public int compareTo(Delayed other) {
+return Long.compare(this.getDelayMillis(), 
((TimedProxyUGI)other).getDelayMillis());
+}
+
+private long getDelayMillis() {
+return (startTime + UGI_CACHE_EXPIRY) - 
System.currentTimeMillis();
+}
+}
+
+private static class SegmentTransactionId {
+String segmentIndex;
--- End diff --

Not having to explicitly invoke close() was based on our choice to rely on 
expiration and inUse status. @lavjain can you recollect why transactionID 
woudln't do ?


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread shivzone
Github user shivzone commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199593515
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
+private static DelayQueue delayQueue = new 
DelayQueue<>();
+private static long UGI_CACHE_EXPIRY = 15 * 60 * 1000L; // 15 Minutes
--- End diff --

We could use pxf-env in that case


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread lavjain
Github user lavjain commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199588824
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
--- End diff --

@shivzone @frankgh Will take a look at Guava cache to see if it meets our 
used case.


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread denalex
Github user denalex commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199577452
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
+private static DelayQueue delayQueue = new 
DelayQueue<>();
+private static long UGI_CACHE_EXPIRY = 15 * 60 * 1000L; // 15 Minutes
+
+
+private static class TimedProxyUGI implements Delayed {
+long startTime;
+UserGroupInformation proxyUGI;
+SegmentTransactionId session;
+
+public TimedProxyUGI(UserGroupInformation proxyUGI, 
SegmentTransactionId session) {
+this.startTime = System.currentTimeMillis();
+this.proxyUGI = proxyUGI;
+this.session = session;
+}
+
+@Override
+public long getDelay(TimeUnit unit) {
+return unit.convert(getDelayMillis(), TimeUnit.MILLISECONDS);
+}
+
+@Override
+public int compareTo(Delayed other) {
+return Long.compare(this.getDelayMillis(), 
((TimedProxyUGI)other).getDelayMillis());
+}
+
+private long getDelayMillis() {
+return (startTime + UGI_CACHE_EXPIRY) - 
System.currentTimeMillis();
+}
+}
+
+private static class SegmentTransactionId {
+String segmentIndex;
--- End diff --

I think that's what we decided on the meeting, even though I don't remember 
why transactionID only is not enough. I believe we decided no need for close() 
call also.


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread denalex
Github user denalex commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199576826
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
+private static DelayQueue delayQueue = new 
DelayQueue<>();
+private static long UGI_CACHE_EXPIRY = 15 * 60 * 1000L; // 15 Minutes
--- End diff --

I don't think we have pxf-site anymore, config properties are now managed 
explicitly via pxf-env.sh and are passed as Java properties via -D parameter to 
Tomcat.


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread frankgh
Github user frankgh commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199547098
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
--- End diff --

I agree with @shivzone about using guava, especially considering that we 
are already using it in this project. 
https://github.com/google/guava/wiki/CachesExplained 


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread shivzone
Github user shivzone commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199542303
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
+private static DelayQueue delayQueue = new 
DelayQueue<>();
+private static long UGI_CACHE_EXPIRY = 15 * 60 * 1000L; // 15 Minutes
+
+
+private static class TimedProxyUGI implements Delayed {
+long startTime;
+UserGroupInformation proxyUGI;
+SegmentTransactionId session;
+
+public TimedProxyUGI(UserGroupInformation proxyUGI, 
SegmentTransactionId session) {
+this.startTime = System.currentTimeMillis();
+this.proxyUGI = proxyUGI;
+this.session = session;
+}
+
+@Override
+public long getDelay(TimeUnit unit) {
+return unit.convert(getDelayMillis(), TimeUnit.MILLISECONDS);
+}
+
+@Override
+public int compareTo(Delayed other) {
+return Long.compare(this.getDelayMillis(), 
((TimedProxyUGI)other).getDelayMillis());
+}
+
+private long getDelayMillis() {
+return (startTime + UGI_CACHE_EXPIRY) - 
System.currentTimeMillis();
+}
+}
+
+private static class SegmentTransactionId {
+String segmentIndex;
--- End diff --

Would this mean the cache will not be shared across even segments from the 
same host accessing the JVM ?


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread shivzone
Github user shivzone commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199541890
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
+private static DelayQueue delayQueue = new 
DelayQueue<>();
+private static long UGI_CACHE_EXPIRY = 15 * 60 * 1000L; // 15 Minutes
--- End diff --

Instead consume this from pxf-site and make it configurable.
{code}
Configuration config = new Configuration();
config.addResource("pxf-site.xml");
{code}


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-07-02 Thread shivzone
Github user shivzone commented on a diff in the pull request:

https://github.com/apache/incubator-hawq/pull/1379#discussion_r199541316
  
--- Diff: 
pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
 ---
@@ -42,8 +52,51 @@
 
 private static final Log LOG = 
LogFactory.getLog(SecurityServletFilter.class);
 private static final String USER_HEADER = "X-GP-USER";
-private static final String MISSING_HEADER_ERROR = 
String.format("Header %s is missing in the request", USER_HEADER);
-private static final String EMPTY_HEADER_ERROR = String.format("Header 
%s is empty in the request", USER_HEADER);
+private static final String SEGMENT_INDEX_HEADER = "X-GP-SEGMENT-ID";
+private static final String TRANSACTION_ID_HEADER = "X-GP-XID";
+private static final String MISSING_HEADER_ERROR = "Header %s is 
missing in the request";
+private static final String EMPTY_HEADER_ERROR = "Header %s is empty 
in the request";
+private static Map cache = new 
ConcurrentHashMap<>();
--- End diff --

Why not use apache commons JCS or guava's caching library as opposed to 
implementing it ourselves with timer ?


---


[GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...

2018-06-29 Thread lavjain
GitHub user lavjain opened a pull request:

https://github.com/apache/incubator-hawq/pull/1379

WIP: Cache UGI objects and clean them periodically



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/lavjain/incubator-hawq pxf_ugi_caching

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/incubator-hawq/pull/1379.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #1379


commit b05aa204c0ed7f6d0adbd90c0ceff2a97dad6fb4
Author: lavjain 
Date:   2018-06-29T16:34:48Z

WIP: Cache UGI objects and clean them periodically




---