You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@knox.apache.org by GitBox <gi...@apache.org> on 2020/06/12 01:31:52 UTC

[GitHub] [knox] pzampino opened a new pull request #345: KNOX-2377 - Address potential loss of token state

pzampino opened a new pull request #345:
URL: https://github.com/apache/knox/pull/345


   ## What changes were proposed in this pull request?
   
   With the resolution of KNOX-2375, the potential for token state to be lost between the issuance of a token and its persistence to the keystore. To address this risk, this patchset introduces a TokenStateJournal, the default implementation of which persists small individual files for each token's state until it is persisted in the keystore. This new facility has been integrated into the AliasBasedTokenStateService, which is the driver behind creating it. When token state it added, an entry (file) is added to the journal; Then, once that state has been persisted in the keystore, the journal entry (file) is removed since the risk of loss is no longer present.
   
   Part of the integration includes loading all the persisted token state journal entries when the AliasBasedTokenStateService is initialized, which effectively allows it to resume processing from where it has left off if the gateway was stopped or crashed after the addition of one or more tokens but before the state for those tokens could be persisted.
   
   ## How was this patch tested?
   
   - 'mvn -T1.5C -Ppackage,release clean install'
   - Added FileTokenStateJournalTest and MultiFileTokenStateJournalTest unit test classes
   - Augmented AliasBasedTokenStateServiceTest to test the integration of the TokenStateJournal with the AliasBasedTokenStateService, the intended (and only) consumer.
   - Manual testing involving two clients concurrently requesting tokens, and allowing this type of test to run through multiple token state eviction cycles, such that I feel there is no additional issue wrt thread-safety or performance.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino commented on a change in pull request #345: KNOX-2377 - Address potential loss of token state

Posted by GitBox <gi...@apache.org>.
pzampino commented on a change in pull request #345:
URL: https://github.com/apache/knox/pull/345#discussion_r439433146



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/JournalEntry.java
##########
@@ -0,0 +1,29 @@
+/*
+ *
+ *  * 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.
+ *
+ */

Review comment:
       Uggh! IntelliJ. Thanks




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on pull request #345: KNOX-2377 - Address potential loss of token state

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on pull request #345:
URL: https://github.com/apache/knox/pull/345#issuecomment-644062256


   In addition to my previous comments I'd also suggest another change:
   in `org.apache.knox.gateway.services.token.impl.AliasBasedTokenStateService.stop()` you may persist all entries in `unpersistedState` after shutting down the scheduler to provide one more final round of token persistence.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino commented on pull request #345: KNOX-2377 - Address potential loss of token state

Posted by GitBox <gi...@apache.org>.
pzampino commented on pull request #345:
URL: https://github.com/apache/knox/pull/345#issuecomment-644128164


   > In addition to my previous comments I'd also suggest another change:
   > in `org.apache.knox.gateway.services.token.impl.AliasBasedTokenStateService.stop()` you may persist all entries in `unpersistedState` after shutting down the scheduler to provide one more final round of token persistence.
   
   Yes, this is a good suggestion.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #345: KNOX-2377 - Address potential loss of token state

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #345:
URL: https://github.com/apache/knox/pull/345#discussion_r439405293



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/TokenStateJournal.java
##########
@@ -0,0 +1,92 @@
+/*
+ *
+ *  * 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.
+ *
+ */

Review comment:
       Incorrect license format.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/JournalEntry.java
##########
@@ -0,0 +1,29 @@
+/*
+ *
+ *  * 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.
+ *
+ */

Review comment:
       Incorrect license format

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/FileTokenStateJournal.java
##########
@@ -0,0 +1,215 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl.state;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.services.token.impl.JournalEntry;
+import org.apache.knox.gateway.services.token.impl.TokenStateJournal;
+import org.apache.knox.gateway.services.token.impl.TokenStateServiceMessages;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * Base class for TokenStateJournal implementations that employ files for persistence.
+ */
+abstract class FileTokenStateJournal implements TokenStateJournal {
+
+    protected static final int INDEX_TOKEN_ID     = 0;
+    protected static final int INDEX_ISSUE_TIME   = 1;
+    protected static final int INDEX_EXPIRATION   = 2;
+    protected static final int INDEX_MAX_LIFETIME = 3;
+
+    protected static final TokenStateServiceMessages log = MessagesFactory.get(TokenStateServiceMessages.class);
+
+    // The name of the journal directory
+    protected static final String JOURNAL_DIR_NAME = "token-state";
+
+    /**
+     * The journal directory path
+     */
+    protected final Path journalDir;
+
+    protected FileTokenStateJournal(GatewayConfig config) throws IOException {
+        journalDir = Paths.get(config.getGatewaySecurityDir(), JOURNAL_DIR_NAME);
+        if (!Files.exists(journalDir)) {
+            Files.createDirectories(journalDir);
+        }
+    }
+
+    @Override
+    public abstract void add(String tokenId, long issueTime, long expiration, long maxLifetime) throws IOException;
+
+    @Override
+    public void add(JournalEntry entry) throws IOException {
+        add(Collections.singletonList(entry));
+    }
+
+    @Override
+    public abstract void add(List<JournalEntry> entries) throws IOException;
+
+    @Override
+    public List<JournalEntry> get() throws IOException {
+        return loadJournal();
+    }
+
+    @Override
+    public abstract JournalEntry get(String tokenId) throws IOException;
+
+    @Override
+    public void remove(final String tokenId) throws IOException {
+        remove(Collections.singleton(tokenId));
+    }
+
+    @Override
+    public abstract void remove(Collection<String> tokenIds) throws IOException;
+
+    @Override
+    public void remove(final JournalEntry entry) throws IOException {
+        remove(entry.getTokenId());
+    }
+
+    protected abstract List<JournalEntry> loadJournal() throws IOException;
+
+    protected List<FileJournalEntry> loadJournal(FileChannel channel) throws IOException {
+        List<FileJournalEntry> entries = new ArrayList<>();
+
+        try (InputStream input = Channels.newInputStream(channel)) {
+            BufferedReader reader = new BufferedReader(new InputStreamReader(input, StandardCharsets.UTF_8));
+            String line;
+            while ((line = reader.readLine()) != null) {
+                entries.add(FileJournalEntry.parse(line));
+            }
+        }
+
+        return entries;
+    }
+
+    /**
+     * Parse the String representation of an entry.
+     *
+     * @param entry A journal file entry line
+     *
+     * @return A FileJournalEntry object created from the specified entry.
+     */
+    protected FileJournalEntry parse(final String entry) {
+        return FileJournalEntry.parse(entry);
+    }
+
+    /**
+     * A JournalEntry implementation for File-based TokenStateJournal implementations
+     */
+    static final class FileJournalEntry implements JournalEntry {

Review comment:
       I believe this class deserves to be in its own file :)

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/FileTokenStateJournal.java
##########
@@ -0,0 +1,215 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl.state;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.services.token.impl.JournalEntry;
+import org.apache.knox.gateway.services.token.impl.TokenStateJournal;
+import org.apache.knox.gateway.services.token.impl.TokenStateServiceMessages;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * Base class for TokenStateJournal implementations that employ files for persistence.
+ */
+abstract class FileTokenStateJournal implements TokenStateJournal {
+
+    protected static final int INDEX_TOKEN_ID     = 0;
+    protected static final int INDEX_ISSUE_TIME   = 1;
+    protected static final int INDEX_EXPIRATION   = 2;
+    protected static final int INDEX_MAX_LIFETIME = 3;
+
+    protected static final TokenStateServiceMessages log = MessagesFactory.get(TokenStateServiceMessages.class);
+
+    // The name of the journal directory
+    protected static final String JOURNAL_DIR_NAME = "token-state";
+
+    /**
+     * The journal directory path
+     */
+    protected final Path journalDir;
+
+    protected FileTokenStateJournal(GatewayConfig config) throws IOException {
+        journalDir = Paths.get(config.getGatewaySecurityDir(), JOURNAL_DIR_NAME);
+        if (!Files.exists(journalDir)) {
+            Files.createDirectories(journalDir);
+        }
+    }
+
+    @Override
+    public abstract void add(String tokenId, long issueTime, long expiration, long maxLifetime) throws IOException;
+
+    @Override
+    public void add(JournalEntry entry) throws IOException {
+        add(Collections.singletonList(entry));
+    }
+
+    @Override
+    public abstract void add(List<JournalEntry> entries) throws IOException;
+
+    @Override
+    public List<JournalEntry> get() throws IOException {
+        return loadJournal();
+    }
+
+    @Override
+    public abstract JournalEntry get(String tokenId) throws IOException;
+
+    @Override
+    public void remove(final String tokenId) throws IOException {
+        remove(Collections.singleton(tokenId));
+    }
+
+    @Override
+    public abstract void remove(Collection<String> tokenIds) throws IOException;
+
+    @Override
+    public void remove(final JournalEntry entry) throws IOException {
+        remove(entry.getTokenId());
+    }
+
+    protected abstract List<JournalEntry> loadJournal() throws IOException;
+
+    protected List<FileJournalEntry> loadJournal(FileChannel channel) throws IOException {
+        List<FileJournalEntry> entries = new ArrayList<>();
+
+        try (InputStream input = Channels.newInputStream(channel)) {
+            BufferedReader reader = new BufferedReader(new InputStreamReader(input, StandardCharsets.UTF_8));

Review comment:
       Should not be reader defined in `try` statement so that it'd be closed?

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/MultiFileTokenStateJournal.java
##########
@@ -0,0 +1,142 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl.state;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.services.token.impl.JournalEntry;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A TokenStateJournal implementation that manages separate files for token state.
+ */
+class MultiFileTokenStateJournal extends FileTokenStateJournal {
+
+    // File extension for journal entry files
+    static final String ENTRY_FILE_EXT = ".ts";
+
+    // Filter used when listing all journal entry files in the journal directory
+    static final String ENTRY_FILE_EXT_FILTER = "*" + ENTRY_FILE_EXT;
+
+    MultiFileTokenStateJournal(GatewayConfig config) throws IOException {
+        super(config);
+    }
+
+    @Override
+    public void add(final String tokenId, long issueTime, long expiration, long maxLifetime) throws IOException {
+        add(Collections.singletonList(new FileJournalEntry(tokenId, issueTime, expiration, maxLifetime)));
+    }
+
+    @Override
+    public void add(final List<JournalEntry> entries) throws IOException {
+        // Persist each journal entry as an individual file in the journal directory
+        for (JournalEntry entry : entries) {
+            final Path entryFile = journalDir.resolve(entry.getTokenId() + ENTRY_FILE_EXT);
+            log.persistingJournalEntry(entryFile.toString());
+            try (FileChannel fileChannel = FileChannel.open(entryFile, StandardOpenOption.WRITE,
+                    StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING)) {
+                fileChannel.lock();
+                try (OutputStream out = Channels.newOutputStream(fileChannel)) {
+                    BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8));
+                    writer.write(entry.toString());
+                    writer.newLine();
+                    writer.flush();
+                }
+                log.addedJournalEntry(entry.getTokenId());
+            } catch (IOException e){
+                log.failedToPersistJournalEntry(entry.getTokenId(), e);
+                throw e;
+            }
+        }
+    }
+
+    @Override
+    public JournalEntry get(final String tokenId) throws IOException {
+        JournalEntry result = null;
+
+        Path entryFilePath = journalDir.resolve(tokenId + ENTRY_FILE_EXT);
+        if (Files.exists(entryFilePath)) {
+            try (FileChannel fileChannel = FileChannel.open(entryFilePath, StandardOpenOption.READ)) {
+                fileChannel.lock(0L, Long.MAX_VALUE, true);
+                List<FileJournalEntry> entries = loadJournal(fileChannel);
+                if (entries.isEmpty()) {
+                    log.journalEntryNotFound(tokenId);

Review comment:
       This is a different case than what we have in line 99 (the file is there but empty). I'd create a different log entry for this case (to help us when debugging).

##########
File path: gateway-server/src/test/java/org/apache/knox/gateway/services/token/impl/state/FileTokenStateJournalTest.java
##########
@@ -0,0 +1,98 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements. See the NOTICE file distributed with this
+ *  * work for additional information regarding copyright ownership. The ASF
+ *  * licenses this file to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance with the License.
+ *  * You may obtain a copy of the License at
+ *  *
+ *  * http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ *  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ *  * License for the specific language governing permissions and limitations under
+ *  * the License.
+ *
+ */

Review comment:
       Incorrect license format.

##########
File path: gateway-server/src/test/java/org/apache/knox/gateway/services/token/impl/state/MultiFileTokenStateJournalTest.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ *  * 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.
+ *
+ */

Review comment:
       Incorrect license format.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/TokenStateJournal.java
##########
@@ -0,0 +1,92 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl;

Review comment:
       The same comment here about this being an interface.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/MultiFileTokenStateJournal.java
##########
@@ -0,0 +1,142 @@
+/*
+ *
+ *  * 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.
+ *
+ */

Review comment:
       Incorrect licence format.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/FileTokenStateJournal.java
##########
@@ -0,0 +1,215 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl.state;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.services.token.impl.JournalEntry;
+import org.apache.knox.gateway.services.token.impl.TokenStateJournal;
+import org.apache.knox.gateway.services.token.impl.TokenStateServiceMessages;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * Base class for TokenStateJournal implementations that employ files for persistence.
+ */
+abstract class FileTokenStateJournal implements TokenStateJournal {
+
+    protected static final int INDEX_TOKEN_ID     = 0;
+    protected static final int INDEX_ISSUE_TIME   = 1;
+    protected static final int INDEX_EXPIRATION   = 2;
+    protected static final int INDEX_MAX_LIFETIME = 3;
+
+    protected static final TokenStateServiceMessages log = MessagesFactory.get(TokenStateServiceMessages.class);
+
+    // The name of the journal directory
+    protected static final String JOURNAL_DIR_NAME = "token-state";
+
+    /**
+     * The journal directory path
+     */
+    protected final Path journalDir;
+
+    protected FileTokenStateJournal(GatewayConfig config) throws IOException {
+        journalDir = Paths.get(config.getGatewaySecurityDir(), JOURNAL_DIR_NAME);
+        if (!Files.exists(journalDir)) {
+            Files.createDirectories(journalDir);
+        }
+    }
+
+    @Override
+    public abstract void add(String tokenId, long issueTime, long expiration, long maxLifetime) throws IOException;
+
+    @Override
+    public void add(JournalEntry entry) throws IOException {
+        add(Collections.singletonList(entry));
+    }
+
+    @Override
+    public abstract void add(List<JournalEntry> entries) throws IOException;
+
+    @Override
+    public List<JournalEntry> get() throws IOException {
+        return loadJournal();
+    }
+
+    @Override
+    public abstract JournalEntry get(String tokenId) throws IOException;
+
+    @Override
+    public void remove(final String tokenId) throws IOException {
+        remove(Collections.singleton(tokenId));
+    }
+
+    @Override
+    public abstract void remove(Collection<String> tokenIds) throws IOException;
+
+    @Override
+    public void remove(final JournalEntry entry) throws IOException {
+        remove(entry.getTokenId());
+    }
+
+    protected abstract List<JournalEntry> loadJournal() throws IOException;
+
+    protected List<FileJournalEntry> loadJournal(FileChannel channel) throws IOException {
+        List<FileJournalEntry> entries = new ArrayList<>();
+
+        try (InputStream input = Channels.newInputStream(channel)) {
+            BufferedReader reader = new BufferedReader(new InputStreamReader(input, StandardCharsets.UTF_8));
+            String line;
+            while ((line = reader.readLine()) != null) {
+                entries.add(FileJournalEntry.parse(line));

Review comment:
       Calling `parse(line)` would be enough here.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/FileTokenStateJournal.java
##########
@@ -0,0 +1,215 @@
+/*
+ *
+ *  * 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.
+ *
+ */

Review comment:
       Incorrect license format.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/FileTokenStateJournal.java
##########
@@ -0,0 +1,215 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl.state;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.services.token.impl.JournalEntry;
+import org.apache.knox.gateway.services.token.impl.TokenStateJournal;
+import org.apache.knox.gateway.services.token.impl.TokenStateServiceMessages;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * Base class for TokenStateJournal implementations that employ files for persistence.
+ */
+abstract class FileTokenStateJournal implements TokenStateJournal {
+
+    protected static final int INDEX_TOKEN_ID     = 0;
+    protected static final int INDEX_ISSUE_TIME   = 1;
+    protected static final int INDEX_EXPIRATION   = 2;
+    protected static final int INDEX_MAX_LIFETIME = 3;
+
+    protected static final TokenStateServiceMessages log = MessagesFactory.get(TokenStateServiceMessages.class);
+
+    // The name of the journal directory
+    protected static final String JOURNAL_DIR_NAME = "token-state";
+
+    /**
+     * The journal directory path
+     */
+    protected final Path journalDir;
+
+    protected FileTokenStateJournal(GatewayConfig config) throws IOException {
+        journalDir = Paths.get(config.getGatewaySecurityDir(), JOURNAL_DIR_NAME);

Review comment:
       Maybe adding a new method in `GatewayConfig` to fetch the journal dir name?

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/JournalEntry.java
##########
@@ -0,0 +1,29 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl;
+
+public interface JournalEntry {
+    String getTokenId();
+
+    String getIssueTime();
+
+    String getExpiration();
+
+    String getMaxLifetime();

Review comment:
       In general, I'm not a big fan of Javadoc bit I feel it'd beneficial to add some description for these methods (e.g max lifetime indicates a date representation or a duration in secs/hours,...).

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/FileTokenStateJournal.java
##########
@@ -0,0 +1,215 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl.state;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.services.token.impl.JournalEntry;
+import org.apache.knox.gateway.services.token.impl.TokenStateJournal;
+import org.apache.knox.gateway.services.token.impl.TokenStateServiceMessages;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * Base class for TokenStateJournal implementations that employ files for persistence.
+ */
+abstract class FileTokenStateJournal implements TokenStateJournal {
+
+    protected static final int INDEX_TOKEN_ID     = 0;
+    protected static final int INDEX_ISSUE_TIME   = 1;
+    protected static final int INDEX_EXPIRATION   = 2;
+    protected static final int INDEX_MAX_LIFETIME = 3;
+
+    protected static final TokenStateServiceMessages log = MessagesFactory.get(TokenStateServiceMessages.class);
+
+    // The name of the journal directory
+    protected static final String JOURNAL_DIR_NAME = "token-state";
+
+    /**
+     * The journal directory path
+     */
+    protected final Path journalDir;
+
+    protected FileTokenStateJournal(GatewayConfig config) throws IOException {
+        journalDir = Paths.get(config.getGatewaySecurityDir(), JOURNAL_DIR_NAME);
+        if (!Files.exists(journalDir)) {
+            Files.createDirectories(journalDir);
+        }
+    }
+
+    @Override
+    public abstract void add(String tokenId, long issueTime, long expiration, long maxLifetime) throws IOException;
+
+    @Override
+    public void add(JournalEntry entry) throws IOException {
+        add(Collections.singletonList(entry));
+    }
+
+    @Override
+    public abstract void add(List<JournalEntry> entries) throws IOException;
+
+    @Override
+    public List<JournalEntry> get() throws IOException {
+        return loadJournal();
+    }
+
+    @Override
+    public abstract JournalEntry get(String tokenId) throws IOException;
+
+    @Override
+    public void remove(final String tokenId) throws IOException {
+        remove(Collections.singleton(tokenId));
+    }
+
+    @Override
+    public abstract void remove(Collection<String> tokenIds) throws IOException;
+
+    @Override
+    public void remove(final JournalEntry entry) throws IOException {
+        remove(entry.getTokenId());
+    }
+
+    protected abstract List<JournalEntry> loadJournal() throws IOException;
+
+    protected List<FileJournalEntry> loadJournal(FileChannel channel) throws IOException {
+        List<FileJournalEntry> entries = new ArrayList<>();
+
+        try (InputStream input = Channels.newInputStream(channel)) {
+            BufferedReader reader = new BufferedReader(new InputStreamReader(input, StandardCharsets.UTF_8));
+            String line;
+            while ((line = reader.readLine()) != null) {
+                entries.add(FileJournalEntry.parse(line));
+            }
+        }
+
+        return entries;
+    }
+
+    /**
+     * Parse the String representation of an entry.
+     *
+     * @param entry A journal file entry line
+     *
+     * @return A FileJournalEntry object created from the specified entry.
+     */
+    protected FileJournalEntry parse(final String entry) {
+        return FileJournalEntry.parse(entry);
+    }
+
+    /**
+     * A JournalEntry implementation for File-based TokenStateJournal implementations
+     */
+    static final class FileJournalEntry implements JournalEntry {
+        private final String tokenId;
+        private final String issueTime;
+        private final String expiration;
+        private final String maxLifetime;
+
+        FileJournalEntry(final String tokenId, long issueTime, long expiration, long maxLifetime) {
+            this(tokenId, String.valueOf(issueTime), String.valueOf(expiration), String.valueOf(maxLifetime));
+        }
+
+        FileJournalEntry(final String tokenId,
+                         final String issueTime,
+                         final String expiration,
+                         final String maxLifetime) {
+            this.tokenId = tokenId;
+            this.issueTime = String.valueOf(issueTime);
+            this.expiration = String.valueOf(expiration);
+            this.maxLifetime = String.valueOf(maxLifetime);
+        }
+
+        @Override
+        public String getTokenId() {
+            return tokenId;
+        }
+
+        @Override
+        public String getIssueTime() {
+            return issueTime;
+        }
+
+        @Override
+        public String getExpiration() {
+            return expiration;
+        }
+
+        @Override
+        public String getMaxLifetime() {
+            return maxLifetime;
+        }
+
+        @Override
+        public String toString() {
+            String[] elements = new String[4];
+
+            elements[INDEX_TOKEN_ID] = getTokenId();
+
+            String issueTime = getIssueTime();
+            elements[INDEX_ISSUE_TIME] = (issueTime != null) ? issueTime : "";
+
+            String expiration = getExpiration();
+            elements[INDEX_EXPIRATION] = (expiration != null) ? expiration : "";
+
+            String maxLifetime = getMaxLifetime();
+            elements[INDEX_MAX_LIFETIME] = (maxLifetime != null) ? maxLifetime : "";
+
+            return String.format(Locale.ROOT,
+                                 "%s,%s,%s,%s",
+                                 elements[INDEX_TOKEN_ID],
+                                 elements[INDEX_ISSUE_TIME],
+                                 elements[INDEX_EXPIRATION],
+                                 elements[INDEX_MAX_LIFETIME]);
+        }
+
+        /**
+          * Parse the String representation of an entry.
+          *
+          * @param entry A journal file entry line
+          *
+          * @return A FileJournalEntry object created from the specified entry.
+          */
+        static FileJournalEntry parse(final String entry) {
+            String[] elements = entry.split(",");

Review comment:
       I feel the ',' (the colon) should be a constant indicating the SEPARATOR and used within `toString()` above too.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/TokenStateJournal.java
##########
@@ -0,0 +1,92 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ *
+ */
+public interface TokenStateJournal {
+
+    /**
+     * Persist the token state to the journal.
+     *
+     * @param tokenId     The unique token identifier
+     * @param issueTime   The issue timestamp
+     * @param expiration  The expiration time
+     * @param maxLifetime The maximum allowed lifetime
+     */
+    void add(String tokenId, long issueTime, long expiration, long maxLifetime)
+        throws IOException;
+
+    /**
+     * Persist the token state to the journal.
+     *
+     * @param entry The entry to persist
+     */
+    void add(JournalEntry entry) throws IOException;
+
+    /**
+     * Persist the token state to the journal.
+     *
+     * @param entries The entries to persist
+     */
+    void add(List<JournalEntry> entries) throws IOException;
+
+    /**
+     * Get the journaled state for the specified token identifier.
+     *
+     * @param tokenId The unique token identifier.
+     *
+     * @return A JournalEntry with the specified token's journaled state.
+     */
+    JournalEntry get(String tokenId) throws IOException;
+
+    /**
+     * Get all the the journaled tokens' state.
+     *
+     * @return A List of JournalEntry objects.
+     */
+    List<JournalEntry> get() throws IOException;

Review comment:
       I'd call it `getAll()`

##########
File path: gateway-server/src/test/java/org/apache/knox/gateway/services/token/impl/state/AbstractFileTokenStateJournalTest.java
##########
@@ -0,0 +1,230 @@
+/*
+ *
+ *  * 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.
+ *
+ */

Review comment:
       Incorrect license format.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/TokenStateJournalFactory.java
##########
@@ -0,0 +1,32 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements. See the NOTICE file distributed with this
+ *  * work for additional information regarding copyright ownership. The ASF
+ *  * licenses this file to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance with the License.
+ *  * You may obtain a copy of the License at
+ *  *
+ *  * http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ *  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ *  * License for the specific language governing permissions and limitations under
+ *  * the License.
+ *
+ */

Review comment:
       Incorrect license format.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/state/FileTokenStateJournal.java
##########
@@ -0,0 +1,215 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl.state;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.services.token.impl.JournalEntry;
+import org.apache.knox.gateway.services.token.impl.TokenStateJournal;
+import org.apache.knox.gateway.services.token.impl.TokenStateServiceMessages;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * Base class for TokenStateJournal implementations that employ files for persistence.
+ */
+abstract class FileTokenStateJournal implements TokenStateJournal {
+
+    protected static final int INDEX_TOKEN_ID     = 0;
+    protected static final int INDEX_ISSUE_TIME   = 1;
+    protected static final int INDEX_EXPIRATION   = 2;
+    protected static final int INDEX_MAX_LIFETIME = 3;
+
+    protected static final TokenStateServiceMessages log = MessagesFactory.get(TokenStateServiceMessages.class);
+
+    // The name of the journal directory
+    protected static final String JOURNAL_DIR_NAME = "token-state";
+
+    /**
+     * The journal directory path
+     */
+    protected final Path journalDir;
+
+    protected FileTokenStateJournal(GatewayConfig config) throws IOException {
+        journalDir = Paths.get(config.getGatewaySecurityDir(), JOURNAL_DIR_NAME);
+        if (!Files.exists(journalDir)) {
+            Files.createDirectories(journalDir);
+        }
+    }
+
+    @Override
+    public abstract void add(String tokenId, long issueTime, long expiration, long maxLifetime) throws IOException;
+
+    @Override
+    public void add(JournalEntry entry) throws IOException {
+        add(Collections.singletonList(entry));
+    }
+
+    @Override
+    public abstract void add(List<JournalEntry> entries) throws IOException;
+
+    @Override
+    public List<JournalEntry> get() throws IOException {
+        return loadJournal();
+    }
+
+    @Override
+    public abstract JournalEntry get(String tokenId) throws IOException;
+
+    @Override
+    public void remove(final String tokenId) throws IOException {
+        remove(Collections.singleton(tokenId));
+    }
+
+    @Override
+    public abstract void remove(Collection<String> tokenIds) throws IOException;
+
+    @Override
+    public void remove(final JournalEntry entry) throws IOException {
+        remove(entry.getTokenId());
+    }
+
+    protected abstract List<JournalEntry> loadJournal() throws IOException;
+
+    protected List<FileJournalEntry> loadJournal(FileChannel channel) throws IOException {
+        List<FileJournalEntry> entries = new ArrayList<>();
+
+        try (InputStream input = Channels.newInputStream(channel)) {
+            BufferedReader reader = new BufferedReader(new InputStreamReader(input, StandardCharsets.UTF_8));
+            String line;
+            while ((line = reader.readLine()) != null) {
+                entries.add(FileJournalEntry.parse(line));
+            }
+        }
+
+        return entries;
+    }
+
+    /**
+     * Parse the String representation of an entry.
+     *
+     * @param entry A journal file entry line
+     *
+     * @return A FileJournalEntry object created from the specified entry.
+     */
+    protected FileJournalEntry parse(final String entry) {
+        return FileJournalEntry.parse(entry);
+    }
+
+    /**
+     * A JournalEntry implementation for File-based TokenStateJournal implementations
+     */
+    static final class FileJournalEntry implements JournalEntry {
+        private final String tokenId;
+        private final String issueTime;
+        private final String expiration;
+        private final String maxLifetime;
+
+        FileJournalEntry(final String tokenId, long issueTime, long expiration, long maxLifetime) {
+            this(tokenId, String.valueOf(issueTime), String.valueOf(expiration), String.valueOf(maxLifetime));
+        }
+
+        FileJournalEntry(final String tokenId,
+                         final String issueTime,
+                         final String expiration,
+                         final String maxLifetime) {
+            this.tokenId = tokenId;
+            this.issueTime = String.valueOf(issueTime);
+            this.expiration = String.valueOf(expiration);
+            this.maxLifetime = String.valueOf(maxLifetime);
+        }
+
+        @Override
+        public String getTokenId() {
+            return tokenId;
+        }
+
+        @Override
+        public String getIssueTime() {
+            return issueTime;
+        }
+
+        @Override
+        public String getExpiration() {
+            return expiration;
+        }
+
+        @Override
+        public String getMaxLifetime() {
+            return maxLifetime;
+        }
+
+        @Override
+        public String toString() {
+            String[] elements = new String[4];
+
+            elements[INDEX_TOKEN_ID] = getTokenId();
+
+            String issueTime = getIssueTime();
+            elements[INDEX_ISSUE_TIME] = (issueTime != null) ? issueTime : "";
+
+            String expiration = getExpiration();
+            elements[INDEX_EXPIRATION] = (expiration != null) ? expiration : "";
+
+            String maxLifetime = getMaxLifetime();
+            elements[INDEX_MAX_LIFETIME] = (maxLifetime != null) ? maxLifetime : "";
+
+            return String.format(Locale.ROOT,
+                                 "%s,%s,%s,%s",
+                                 elements[INDEX_TOKEN_ID],
+                                 elements[INDEX_ISSUE_TIME],
+                                 elements[INDEX_EXPIRATION],
+                                 elements[INDEX_MAX_LIFETIME]);
+        }
+
+        /**
+          * Parse the String representation of an entry.
+          *
+          * @param entry A journal file entry line
+          *
+          * @return A FileJournalEntry object created from the specified entry.
+          */
+        static FileJournalEntry parse(final String entry) {
+            String[] elements = entry.split(",");
+            if (elements.length < 4) {
+                throw new IllegalArgumentException("Invalid journal entry: " + entry);
+            }
+            return new FileJournalEntry(elements[INDEX_TOKEN_ID],
+                                        elements[INDEX_ISSUE_TIME],
+                                        elements[INDEX_EXPIRATION],
+                                        elements[INDEX_MAX_LIFETIME]);
+        }

Review comment:
       I don't see the `null` handling here. Based on the `toString()` method if an entry is loaded and any of its columns has an empty string it should be set to `null`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino commented on a change in pull request #345: KNOX-2377 - Address potential loss of token state

Posted by GitBox <gi...@apache.org>.
pzampino commented on a change in pull request #345:
URL: https://github.com/apache/knox/pull/345#discussion_r439433286



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/JournalEntry.java
##########
@@ -0,0 +1,29 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl;
+
+public interface JournalEntry {
+    String getTokenId();
+
+    String getIssueTime();
+
+    String getExpiration();
+
+    String getMaxLifetime();

Review comment:
       Agreed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino commented on a change in pull request #345: KNOX-2377 - Address potential loss of token state

Posted by GitBox <gi...@apache.org>.
pzampino commented on a change in pull request #345:
URL: https://github.com/apache/knox/pull/345#discussion_r439433732



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/TokenStateJournal.java
##########
@@ -0,0 +1,92 @@
+/*
+ *
+ *  * 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.
+ *
+ */
+package org.apache.knox.gateway.services.token.impl;

Review comment:
       What do you mean about this being an interface?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino merged pull request #345: KNOX-2377 - Address potential loss of token state

Posted by GitBox <gi...@apache.org>.
pzampino merged pull request #345:
URL: https://github.com/apache/knox/pull/345


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org