From b07151a4b384e1b188e976dd2f9a23932139d241 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 6 Nov 2024 12:45:53 +0530 Subject: [PATCH 01/73] pushing malicious requests to kafka --- .../java/com/akto/filters/HttpCallFilter.java | 173 ++++++++++-------- .../java/com/akto/threat/detection/Main.java | 119 ++++++------ .../akto/dto/bulk_updates/BulkUpdates.java | 17 +- .../akto/dto/traffic/SuspectSampleData.java | 75 ++++++-- .../src/main/java/com/akto/kafka/Kafka.java | 7 +- 5 files changed, 231 insertions(+), 160 deletions(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index ecbc8e5ac7..37190f1029 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -1,27 +1,21 @@ package com.akto.filters; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.UUID; +import java.util.*; import com.akto.dao.context.Context; import com.akto.dao.monitoring.FilterYamlTemplateDao; import com.akto.data_actor.DataActor; import com.akto.data_actor.DataActorFactory; import com.akto.dto.ApiInfo.ApiInfoKey; +import com.akto.dto.HttpRequestParams; import com.akto.dto.HttpResponseParams; import com.akto.dto.RawApi; -import com.akto.dto.bulk_updates.BulkUpdates; -import com.akto.dto.bulk_updates.UpdatePayload; import com.akto.dto.monitoring.FilterConfig; import com.akto.dto.test_editor.YamlTemplate; import com.akto.dto.traffic.SuspectSampleData; import com.akto.dto.type.URLMethods.Method; import com.akto.hybrid_parsers.HttpCallParser; +import com.akto.kafka.Kafka; import com.akto.log.LoggerMaker; import com.akto.log.LoggerMaker.LogDb; import com.akto.rules.TestPlugin; @@ -30,15 +24,16 @@ import com.akto.test_editor.filter.data_operands_impl.ValidationResult; public class HttpCallFilter { - private static final LoggerMaker loggerMaker = new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); + private static final LoggerMaker loggerMaker = + new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); private Map apiFilters; - private List bulkUpdates = new ArrayList<>(); - private final int sync_threshold_count; - private final int sync_threshold_time; - private int last_synced; - private int sync_count; - private HttpCallParser httpCallParser; + private final HttpCallParser httpCallParser; + private final Kafka kafka; + + private static final int KAFKA_BATCH_SIZE = 1000; + private static final int KAFKA_BATCH_LINGER_MS = 1000; + private static final String KAFKA_MALICIOUS_TOPIC = "akto.malicious"; private static final int FILTER_REFRESH_INTERVAL = 10 * 60; private int lastFilterFetch; @@ -46,14 +41,18 @@ public class HttpCallFilter { private static final DataActor dataActor = DataActorFactory.fetchInstance(); public HttpCallFilter(int sync_threshold_count, int sync_threshold_time) { - apiFilters = new HashMap<>(); - bulkUpdates = new ArrayList<>(); - this.sync_threshold_count = sync_threshold_count; - this.sync_threshold_time = sync_threshold_time; - last_synced = 0; - sync_count = 0; - lastFilterFetch = 0; - httpCallParser = new HttpCallParser(sync_threshold_count, sync_threshold_time); + this.apiFilters = new HashMap<>(); + this.lastFilterFetch = 0; + this.httpCallParser = new HttpCallParser(sync_threshold_count, sync_threshold_time); + + String kafkaBootstrapServers = "kafka1:19092"; + String isKubernetes = System.getenv("IS_KUBERNETES"); + if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { + loggerMaker.infoAndAddToDb("is_kubernetes: true"); + kafkaBootstrapServers = "127.0.0.1:29092"; + } + + this.kafka = new Kafka(kafkaBootstrapServers, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); } public void filterFunction(List responseParams) { @@ -66,56 +65,86 @@ public void filterFunction(List responseParams) { lastFilterFetch = now; } - if (apiFilters != null && !apiFilters.isEmpty()) { - for (HttpResponseParams responseParam : responseParams) { - for (Entry apiFilterEntry : apiFilters.entrySet()) { - try { - FilterConfig apiFilter = apiFilterEntry.getValue(); - String filterId = apiFilterEntry.getKey(); - String message = responseParam.getOrig(); - List sourceIps = ApiAccessTypePolicy.getSourceIps(responseParam); - RawApi rawApi = RawApi.buildFromMessage(message); - int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); - responseParam.requestParams.setApiCollectionId(apiCollectionId); - String url = responseParam.getRequestParams().getURL(); - Method method = Method.fromString(responseParam.getRequestParams().getMethod()); - ApiInfoKey apiInfoKey = new ApiInfoKey(apiCollectionId, url, method); - Map varMap = apiFilter.resolveVarMap(); - VariableResolver.resolveWordList(varMap, new HashMap>() { - { - put(apiInfoKey, Arrays.asList(message)); - } - }, apiInfoKey); - String filterExecutionLogId = UUID.randomUUID().toString(); - ValidationResult res = TestPlugin.validateFilter(apiFilter.getFilter().getNode(), rawApi, - apiInfoKey, varMap, filterExecutionLogId); - if (res.getIsValid()) { - now = Context.now(); - SuspectSampleData sampleData = new SuspectSampleData( - sourceIps, apiCollectionId, url, method, - message, now, filterId); - Map filterMap = new HashMap<>(); - UpdatePayload updatePayload = new UpdatePayload("obj", sampleData, "set"); - ArrayList updates = new ArrayList<>(); - updates.add(updatePayload.toString()); - bulkUpdates.add(new BulkUpdates(filterMap, updates)); - } - } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, String.format("Error in httpCallFilter %s", e.toString())); - } + if (apiFilters == null || apiFilters.isEmpty()) { + return; + } + + List maliciousSamples = new ArrayList<>(); + for (HttpResponseParams responseParam : responseParams) { + for (FilterConfig apiFilter : apiFilters.values()) { + boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); + + // If a request passes any of the filter, then it's a malicious request, + // and so we push it to kafka + if (hasPassedFilter) { + HttpRequestParams requestParams = responseParam.getRequestParams(); + List sourceIps = ApiAccessTypePolicy.getSourceIps(responseParam); + Method method = Method.fromString(requestParams.getMethod()); + + maliciousSamples.add( + new SuspectSampleData( + sourceIps, + requestParams.getApiCollectionId(), + requestParams.getURL(), + method, + responseParam.getOrig(), + Context.now(), + apiFilter.getId())); + + // Later we will also add aggregation support + // Eg: 100 4xx requests in last 10 minutes. + // But regardless of whether request falls in aggregation or not, + // we still push malicious requests to kafka } } } - sync_count = bulkUpdates.size(); - if (sync_count > 0 && (sync_count >= sync_threshold_count || - (Context.now() - last_synced) > sync_threshold_time)) { - List updates = new ArrayList<>(); - updates.addAll(bulkUpdates); - dataActor.bulkWriteSuspectSampleData(updates); - loggerMaker.infoAndAddToDb(String.format("Inserting %d records in SuspectSampleData", sync_count)); - last_synced = Context.now(); - sync_count = 0; - bulkUpdates.clear(); + + // Should we push all the messages in one go + // or call kafka.send for each HttpRequestParams + maliciousSamples.forEach( + sample -> { + sample.marshall() + .ifPresent( + s -> { + kafka.send(s, KAFKA_MALICIOUS_TOPIC); + }); + }); + } + + private boolean validateFilterForRequest( + HttpResponseParams responseParam, FilterConfig apiFilter) { + try { + String message = responseParam.getOrig(); + RawApi rawApi = RawApi.buildFromMessage(message); + int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); + responseParam.requestParams.setApiCollectionId(apiCollectionId); + String url = responseParam.getRequestParams().getURL(); + Method method = Method.fromString(responseParam.getRequestParams().getMethod()); + ApiInfoKey apiInfoKey = new ApiInfoKey(apiCollectionId, url, method); + Map varMap = apiFilter.resolveVarMap(); + VariableResolver.resolveWordList( + varMap, + new HashMap>() { + { + put(apiInfoKey, Collections.singletonList(message)); + } + }, + apiInfoKey); + String filterExecutionLogId = UUID.randomUUID().toString(); + ValidationResult res = + TestPlugin.validateFilter( + apiFilter.getFilter().getNode(), + rawApi, + apiInfoKey, + varMap, + filterExecutionLogId); + + return res.getIsValid(); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + e, String.format("Error in httpCallFilter %s", e.toString())); } + + return false; } -} \ No newline at end of file +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java index 28693a405c..aa52f49561 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -1,14 +1,11 @@ package com.akto.threat.detection; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import org.apache.commons.lang3.function.FailableFunction; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.producer.ProducerConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,71 +26,83 @@ public class Main { private static final int sync_threshold_time = 120; private static final int sync_threshold_count = 1000; private static long lastSyncOffset = 0; - + public static void main(String[] args) { - + Map httpCallFilterMap = new HashMap<>(); String topicName = System.getenv("AKTO_KAFKA_TOPIC_NAME"); if (topicName == null) { String defaultTopic = "akto.api.protection"; - loggerMaker.infoAndAddToDb(String.format("Kafka topic is not defined, using default topic : %s", defaultTopic)); + loggerMaker.infoAndAddToDb( + String.format( + "Kafka topic is not defined, using default topic : %s", defaultTopic)); topicName = defaultTopic; } - - FailableFunction, Void, Exception> func = records -> { - long start = System.currentTimeMillis(); - - // TODO: what happens if exception - Map> responseParamsToAccountMap = new HashMap<>(); - for (ConsumerRecord r : records) { - HttpResponseParams httpResponseParams; - try { - Utils.printL(r.value()); - AllMetrics.instance.setRuntimeKafkaRecordCount(1); - AllMetrics.instance.setRuntimeKafkaRecordSize(r.value().length()); - lastSyncOffset++; - if (lastSyncOffset % 100 == 0) { - logger.info("Committing offset at position: " + lastSyncOffset); + Properties kafkaProperties = new Properties(); + kafkaProperties.put( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, System.getenv("AKTO_KAFKA_BROKER_IP")); + + FailableFunction, Void, Exception> func = + records -> { + long start = System.currentTimeMillis(); + + // TODO: what happens if exception + Map> responseParamsToAccountMap = + new HashMap<>(); + for (ConsumerRecord r : records) { + HttpResponseParams httpResponseParams; + try { + Utils.printL(r.value()); + AllMetrics.instance.setRuntimeKafkaRecordCount(1); + AllMetrics.instance.setRuntimeKafkaRecordSize(r.value().length()); + lastSyncOffset++; + if (lastSyncOffset % 100 == 0) { + logger.info("Committing offset at position: " + lastSyncOffset); + } + httpResponseParams = HttpCallParser.parseKafkaMessage(r.value()); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + e, "Error while parsing kafka message " + e, LogDb.RUNTIME); + continue; } - httpResponseParams = HttpCallParser.parseKafkaMessage(r.value()); - } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, "Error while parsing kafka message " + e, LogDb.RUNTIME); - continue; - } - String accountId = httpResponseParams.getAccountId(); - if (!responseParamsToAccountMap.containsKey(accountId)) { - responseParamsToAccountMap.put(accountId, new ArrayList<>()); + String accountId = httpResponseParams.getAccountId(); + if (!responseParamsToAccountMap.containsKey(accountId)) { + responseParamsToAccountMap.put(accountId, new ArrayList<>()); + } + responseParamsToAccountMap.get(accountId).add(httpResponseParams); } - responseParamsToAccountMap.get(accountId).add(httpResponseParams); - } - for (String accountId : responseParamsToAccountMap.keySet()) { - int accountIdInt; - try { - accountIdInt = Integer.parseInt(accountId); - } catch (Exception ignored) { - loggerMaker.errorAndAddToDb("Account id not string", LogDb.RUNTIME); - continue; - } + for (String accountId : responseParamsToAccountMap.keySet()) { + int accountIdInt; + try { + accountIdInt = Integer.parseInt(accountId); + } catch (Exception ignored) { + loggerMaker.errorAndAddToDb("Account id not string", LogDb.RUNTIME); + continue; + } - Context.accountId.set(accountIdInt); + Context.accountId.set(accountIdInt); - if (!httpCallFilterMap.containsKey(accountId)) { - HttpCallFilter filter = new HttpCallFilter(sync_threshold_count, sync_threshold_time); - httpCallFilterMap.put(accountId, filter); - loggerMaker.infoAndAddToDb("New filter created for account: " + accountId); - } + if (!httpCallFilterMap.containsKey(accountId)) { + HttpCallFilter filter = + new HttpCallFilter(sync_threshold_count, sync_threshold_time); + httpCallFilterMap.put(accountId, filter); + loggerMaker.infoAndAddToDb( + "New filter created for account: " + accountId); + } - HttpCallFilter filter = httpCallFilterMap.get(accountId); - List accWiseResponse = responseParamsToAccountMap.get(accountId); - filter.filterFunction(accWiseResponse); - } + HttpCallFilter filter = httpCallFilterMap.get(accountId); + List accWiseResponse = + responseParamsToAccountMap.get(accountId); + filter.filterFunction(accWiseResponse); + } - AllMetrics.instance.setRuntimeProcessLatency(System.currentTimeMillis()-start); + AllMetrics.instance.setRuntimeProcessLatency( + System.currentTimeMillis() - start); - return null; - }; + return null; + }; KafkaRunner.processKafkaRecords(module, Arrays.asList(topicName), func); } -} \ No newline at end of file +} diff --git a/libs/dao/src/main/java/com/akto/dto/bulk_updates/BulkUpdates.java b/libs/dao/src/main/java/com/akto/dto/bulk_updates/BulkUpdates.java index 256c9fbd04..06c4230190 100644 --- a/libs/dao/src/main/java/com/akto/dto/bulk_updates/BulkUpdates.java +++ b/libs/dao/src/main/java/com/akto/dto/bulk_updates/BulkUpdates.java @@ -4,34 +4,31 @@ import java.util.List; import java.util.Map; - public class BulkUpdates { private Map filters; - private ArrayList updates; + private List updates; - public BulkUpdates() { - } + public BulkUpdates() {} - public BulkUpdates(Map filters, ArrayList updates) { + public BulkUpdates(Map filters, List updates) { this.filters = filters; this.updates = updates; } - public Map getFilters() { + public Map getFilters() { return this.filters; } - public void setFilters(Map filters) { + public void setFilters(Map filters) { this.filters = filters; } - public ArrayList getUpdates() { + public List getUpdates() { return this.updates; } public void setUpdates(ArrayList updates) { this.updates = updates; } - -} \ No newline at end of file +} diff --git a/libs/dao/src/main/java/com/akto/dto/traffic/SuspectSampleData.java b/libs/dao/src/main/java/com/akto/dto/traffic/SuspectSampleData.java index accfc5b398..ed544a6cfa 100644 --- a/libs/dao/src/main/java/com/akto/dto/traffic/SuspectSampleData.java +++ b/libs/dao/src/main/java/com/akto/dto/traffic/SuspectSampleData.java @@ -1,7 +1,9 @@ package com.akto.dto.traffic; import java.util.List; +import java.util.Optional; +import com.fasterxml.jackson.databind.ObjectMapper; import org.bson.types.ObjectId; import com.akto.dto.type.URLMethods.Method; @@ -23,7 +25,7 @@ public class SuspectSampleData { * we retrospectively match all sus-samples' url * with the urls present in the db to match them. */ - public final static String MATCHING_URL = "matchingUrl"; + public static final String MATCHING_URL = "matchingUrl"; String matchingUrl; /* @@ -31,11 +33,18 @@ public class SuspectSampleData { */ String filterId; - public SuspectSampleData() { - } + private final ObjectMapper objectMapper = new ObjectMapper(); + + public SuspectSampleData() {} - public SuspectSampleData(List sourceIPs, int apiCollectionId, String url, Method method, String sample, - int discovered, String filterId) { + public SuspectSampleData( + List sourceIPs, + int apiCollectionId, + String url, + Method method, + String sample, + int discovered, + String filterId) { this.sourceIPs = sourceIPs; this.apiCollectionId = apiCollectionId; this.url = url; @@ -123,16 +132,46 @@ public void setFilterId(String filterId) { @Override public String toString() { - return "{" + - " \"apiCollectionId\":\"" + getApiCollectionId() + "\"" + - ", \"url\":\"" + getUrl() + "\"" + - ", \"method\":\"" + getMethod() + "\"" + - ", \"matchingUrl\":\"" + (getMatchingUrl() != null ? getMatchingUrl() : "/") + "\"" + - ", \"discovered\":\"" + getDiscovered() + "\"" + - ", \"filter\":\"" + getFilterId() + "\"" + - ", \"IPs\":\"" + (getSourceIPs() !=null ? getSourceIPs() : "[]" )+ "\"" + - ", \"sample\":" + getSample() + - "}"; - } - -} \ No newline at end of file + return "{" + + " \"apiCollectionId\":\"" + + getApiCollectionId() + + "\"" + + ", \"url\":\"" + + getUrl() + + "\"" + + ", \"method\":\"" + + getMethod() + + "\"" + + ", \"matchingUrl\":\"" + + (getMatchingUrl() != null ? getMatchingUrl() : "/") + + "\"" + + ", \"discovered\":\"" + + getDiscovered() + + "\"" + + ", \"filter\":\"" + + getFilterId() + + "\"" + + ", \"IPs\":\"" + + (getSourceIPs() != null ? getSourceIPs() : "[]") + + "\"" + + ", \"sample\":" + + getSample() + + "}"; + } + + public Optional marshall() { + try { + return Optional.of(objectMapper.writeValueAsString(this)); + } catch (Exception e) { + return Optional.empty(); + } + } + + public Optional unmarshall() { + try { + return Optional.of(objectMapper.readValue(sample, SampleData.class)); + } catch (Exception e) { + return Optional.empty(); + } + } +} diff --git a/libs/utils/src/main/java/com/akto/kafka/Kafka.java b/libs/utils/src/main/java/com/akto/kafka/Kafka.java index 95a143987c..4dcd0b673c 100644 --- a/libs/utils/src/main/java/com/akto/kafka/Kafka.java +++ b/libs/utils/src/main/java/com/akto/kafka/Kafka.java @@ -22,10 +22,10 @@ public Kafka(String brokerIP, int lingerMS, int batchSize) { } } - public void send(String message,String topic) { + public void send(String message, String topic) { if (!this.producerReady) return; - ProducerRecord record = new ProducerRecord<>(topic,message); + ProducerRecord record = new ProducerRecord<>(topic, message); producer.send(record, new DemoProducerCallback()); } @@ -70,7 +70,4 @@ public void onCompletion(RecordMetadata recordMetadata, Exception e) { } } } - } - - From 1151e8895ebdc584041744bdcbf263f23037fc7f Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Sat, 9 Nov 2024 11:34:00 +0530 Subject: [PATCH 02/73] added flush kafka messages task --- .../java/com/akto/filters/HttpCallFilter.java | 57 +++---- .../akto/suspect_data/FlushMessagesTask.java | 89 +++++++++++ .../java/com/akto/suspect_data/Message.java | 58 +++++++ .../java/com/akto/threat/detection/Main.java | 142 ++++++++++-------- .../akto/interceptor/UsageInterceptor.java | 4 +- .../akto/dto/traffic/SuspectSampleData.java | 20 --- .../java/com/akto/traffic/KafkaRunner.java | 118 ++++++++------- 7 files changed, 316 insertions(+), 172 deletions(-) create mode 100644 apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 37190f1029..6821b74a66 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -20,12 +20,12 @@ import com.akto.log.LoggerMaker.LogDb; import com.akto.rules.TestPlugin; import com.akto.runtime.policies.ApiAccessTypePolicy; +import com.akto.suspect_data.Message; import com.akto.test_editor.execution.VariableResolver; import com.akto.test_editor.filter.data_operands_impl.ValidationResult; public class HttpCallFilter { - private static final LoggerMaker loggerMaker = - new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); + private static final LoggerMaker loggerMaker = new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); private Map apiFilters; private final HttpCallParser httpCallParser; @@ -69,7 +69,7 @@ public void filterFunction(List responseParams) { return; } - List maliciousSamples = new ArrayList<>(); + List maliciousSamples = new ArrayList<>(); for (HttpResponseParams responseParam : responseParams) { for (FilterConfig apiFilter : apiFilters.values()) { boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); @@ -82,14 +82,16 @@ public void filterFunction(List responseParams) { Method method = Method.fromString(requestParams.getMethod()); maliciousSamples.add( - new SuspectSampleData( - sourceIps, - requestParams.getApiCollectionId(), - requestParams.getURL(), - method, - responseParam.getOrig(), - Context.now(), - apiFilter.getId())); + new Message( + responseParam.getAccountId(), + new SuspectSampleData( + sourceIps, + requestParams.getApiCollectionId(), + requestParams.getURL(), + method, + responseParam.getOrig(), + Context.now(), + apiFilter.getId()))); // Later we will also add aggregation support // Eg: 100 4xx requests in last 10 minutes. @@ -101,14 +103,18 @@ public void filterFunction(List responseParams) { // Should we push all the messages in one go // or call kafka.send for each HttpRequestParams - maliciousSamples.forEach( - sample -> { - sample.marshall() - .ifPresent( - s -> { - kafka.send(s, KAFKA_MALICIOUS_TOPIC); - }); - }); + try { + maliciousSamples.forEach( + sample -> { + sample.marshall() + .ifPresent( + s -> { + kafka.send(s, KAFKA_MALICIOUS_TOPIC); + }); + }); + } catch (Exception e) { + e.printStackTrace(); + } } private boolean validateFilterForRequest( @@ -131,13 +137,12 @@ private boolean validateFilterForRequest( }, apiInfoKey); String filterExecutionLogId = UUID.randomUUID().toString(); - ValidationResult res = - TestPlugin.validateFilter( - apiFilter.getFilter().getNode(), - rawApi, - apiInfoKey, - varMap, - filterExecutionLogId); + ValidationResult res = TestPlugin.validateFilter( + apiFilter.getFilter().getNode(), + rawApi, + apiInfoKey, + varMap, + filterExecutionLogId); return res.getIsValid(); } catch (Exception e) { diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java new file mode 100644 index 0000000000..920d2b43f2 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java @@ -0,0 +1,89 @@ +package com.akto.suspect_data; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; + +import com.akto.dao.SuspectSampleDataDao; +import com.akto.dao.context.Context; +import com.akto.dto.traffic.SuspectSampleData; +import com.akto.runtime.utils.Utils; +import com.mongodb.client.model.BulkWriteOptions; +import com.mongodb.client.model.InsertOneModel; +import com.mongodb.client.model.WriteModel; + +public class FlushMessagesTask { + + private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); + private final Consumer consumer; + + private FlushMessagesTask() { + String kafkaBrokerUrl = "127.0.0.1:29092"; + String isKubernetes = System.getenv("IS_KUBERNETES"); + if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { + kafkaBrokerUrl = "127.0.0.1:29092"; + } + + String groupId = "akto-flush-malicious-messages"; + + Properties properties = Utils.configProperties(kafkaBrokerUrl, groupId, 100); + this.consumer = new KafkaConsumer<>(properties); + } + + public static FlushMessagesTask instance = new FlushMessagesTask(); + + public void init() { + consumer.subscribe(Collections.singletonList("akto.malicious")); + pollingExecutor.execute(new Runnable() { + @Override + public void run() { + while (true) { + try { + ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); + processRecords(records); + } catch (Exception e) { + e.printStackTrace(); + consumer.close(); + } + } + } + }); + } + + public void processRecords(ConsumerRecords records) { + Map> accWiseMessages = new HashMap<>(); + for (ConsumerRecord record : records) { + String msgStr = record.value(); + Message.unmarshall(msgStr).ifPresent(msg -> { + accWiseMessages.computeIfAbsent(msg.getAccountId(), k -> new ArrayList<>()).add(msg.getData()); + }); + } + + for (Map.Entry> entry : accWiseMessages.entrySet()) { + String accountId = entry.getKey(); + List sampleDatas = entry.getValue(); + Context.accountId.set(Integer.parseInt(accountId)); + + try { + List> bulkUpdates = new ArrayList<>(); + sampleDatas + .forEach(sampleData -> bulkUpdates.add(new InsertOneModel<>(sampleData))); + + SuspectSampleDataDao.instance.bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); + } catch (Exception e) { + e.printStackTrace(); + } + } + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java new file mode 100644 index 0000000000..313f6c3968 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java @@ -0,0 +1,58 @@ +package com.akto.suspect_data; + +import com.akto.dto.traffic.SuspectSampleData; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.util.Optional; + +// Kafka Message Wrapper for suspect data +public class Message { + private String accountId; + private SuspectSampleData data; + + private static final ObjectMapper objectMapper = new ObjectMapper(); + + public Message() { + } + + public Message(String accountId, SuspectSampleData data) { + this.accountId = accountId; + this.data = data; + } + + public String getAccountId() { + return accountId; + } + + public void setAccountId(String accountId) { + this.accountId = accountId; + } + + public SuspectSampleData getData() { + return data; + } + + public void setData(SuspectSampleData data) { + this.data = data; + } + + public static Optional marshall(Message m) { + try { + return Optional.of(objectMapper.writeValueAsString(m)); + } catch (Exception e) { + return Optional.empty(); + } + } + + public Optional marshall() { + return marshall(this); + } + + public static Optional unmarshall(String s) { + try { + return Optional.of(objectMapper.readValue(s, Message.class)); + } catch (Exception e) { + return Optional.empty(); + } + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java index aa52f49561..3e04c81c60 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -1,11 +1,10 @@ package com.akto.threat.detection; import java.util.*; - -import org.apache.commons.lang3.function.FailableFunction; +import com.akto.suspect_data.FlushMessagesTask; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -27,9 +26,13 @@ public class Main { private static final int sync_threshold_count = 1000; private static long lastSyncOffset = 0; + private static final Map httpCallFilterMap = new HashMap<>(); + public static void main(String[] args) { - Map httpCallFilterMap = new HashMap<>(); + // Flush Messages task + FlushMessagesTask.instance.init(); + String topicName = System.getenv("AKTO_KAFKA_TOPIC_NAME"); if (topicName == null) { String defaultTopic = "akto.api.protection"; @@ -39,70 +42,77 @@ public static void main(String[] args) { topicName = defaultTopic; } - Properties kafkaProperties = new Properties(); - kafkaProperties.put( - ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, System.getenv("AKTO_KAFKA_BROKER_IP")); + String kafkaBrokerUrl = "127.0.0.1:29092"; + String isKubernetes = System.getenv("IS_KUBERNETES"); + if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { + loggerMaker.infoAndAddToDb("is_kubernetes: true"); + kafkaBrokerUrl = "127.0.0.1:29092"; + } + String groupId = "akto-threat-detection"; + int maxPollRecords = Integer.parseInt( + System.getenv().getOrDefault("AKTO_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); + + Properties kafkaRecordProperties = Utils.configProperties(kafkaBrokerUrl, groupId, maxPollRecords); - FailableFunction, Void, Exception> func = + KafkaRunner recordRunner = new KafkaRunner(module, new KafkaConsumer<>(kafkaRecordProperties)); + recordRunner.consume( + Collections.singletonList(topicName), records -> { - long start = System.currentTimeMillis(); - - // TODO: what happens if exception - Map> responseParamsToAccountMap = - new HashMap<>(); - for (ConsumerRecord r : records) { - HttpResponseParams httpResponseParams; - try { - Utils.printL(r.value()); - AllMetrics.instance.setRuntimeKafkaRecordCount(1); - AllMetrics.instance.setRuntimeKafkaRecordSize(r.value().length()); - lastSyncOffset++; - if (lastSyncOffset % 100 == 0) { - logger.info("Committing offset at position: " + lastSyncOffset); - } - httpResponseParams = HttpCallParser.parseKafkaMessage(r.value()); - } catch (Exception e) { - loggerMaker.errorAndAddToDb( - e, "Error while parsing kafka message " + e, LogDb.RUNTIME); - continue; - } - String accountId = httpResponseParams.getAccountId(); - if (!responseParamsToAccountMap.containsKey(accountId)) { - responseParamsToAccountMap.put(accountId, new ArrayList<>()); - } - responseParamsToAccountMap.get(accountId).add(httpResponseParams); - } - - for (String accountId : responseParamsToAccountMap.keySet()) { - int accountIdInt; - try { - accountIdInt = Integer.parseInt(accountId); - } catch (Exception ignored) { - loggerMaker.errorAndAddToDb("Account id not string", LogDb.RUNTIME); - continue; - } - - Context.accountId.set(accountIdInt); - - if (!httpCallFilterMap.containsKey(accountId)) { - HttpCallFilter filter = - new HttpCallFilter(sync_threshold_count, sync_threshold_time); - httpCallFilterMap.put(accountId, filter); - loggerMaker.infoAndAddToDb( - "New filter created for account: " + accountId); - } - - HttpCallFilter filter = httpCallFilterMap.get(accountId); - List accWiseResponse = - responseParamsToAccountMap.get(accountId); - filter.filterFunction(accWiseResponse); - } - - AllMetrics.instance.setRuntimeProcessLatency( - System.currentTimeMillis() - start); - + processRecords(records); return null; - }; - KafkaRunner.processKafkaRecords(module, Arrays.asList(topicName), func); + }); + } + + public static void processRecords(ConsumerRecords records) { + long start = System.currentTimeMillis(); + + // TODO: what happens if exception + Map> responseParamsToAccountMap = new HashMap<>(); + for (ConsumerRecord r : records) { + HttpResponseParams httpResponseParams; + try { + Utils.printL(r.value()); + AllMetrics.instance.setRuntimeKafkaRecordCount(1); + AllMetrics.instance.setRuntimeKafkaRecordSize(r.value().length()); + lastSyncOffset++; + if (lastSyncOffset % 100 == 0) { + logger.info("Committing offset at position: " + lastSyncOffset); + } + httpResponseParams = HttpCallParser.parseKafkaMessage(r.value()); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + e, "Error while parsing kafka message " + e, LogDb.RUNTIME); + continue; + } + String accountId = httpResponseParams.getAccountId(); + if (!responseParamsToAccountMap.containsKey(accountId)) { + responseParamsToAccountMap.put(accountId, new ArrayList<>()); + } + responseParamsToAccountMap.get(accountId).add(httpResponseParams); + } + + for (String accountId : responseParamsToAccountMap.keySet()) { + int accountIdInt; + try { + accountIdInt = Integer.parseInt(accountId); + } catch (Exception ignored) { + loggerMaker.errorAndAddToDb("Account id not string", LogDb.RUNTIME); + continue; + } + + Context.accountId.set(accountIdInt); + + if (!httpCallFilterMap.containsKey(accountId)) { + HttpCallFilter filter = new HttpCallFilter(sync_threshold_count, sync_threshold_time); + httpCallFilterMap.put(accountId, filter); + loggerMaker.infoAndAddToDb("New filter created for account: " + accountId); + } + + HttpCallFilter filter = httpCallFilterMap.get(accountId); + List accWiseResponse = responseParamsToAccountMap.get(accountId); + filter.filterFunction(accWiseResponse); + } + + AllMetrics.instance.setRuntimeProcessLatency(System.currentTimeMillis() - start); } } diff --git a/apps/dashboard/src/main/java/com/akto/interceptor/UsageInterceptor.java b/apps/dashboard/src/main/java/com/akto/interceptor/UsageInterceptor.java index 6726faa24d..46ab823fb1 100644 --- a/apps/dashboard/src/main/java/com/akto/interceptor/UsageInterceptor.java +++ b/apps/dashboard/src/main/java/com/akto/interceptor/UsageInterceptor.java @@ -48,7 +48,7 @@ public String intercept(ActionInvocation invocation) throws Exception { HashMap featureWiseAllowed = organization.getFeatureWiseAllowed(); - if(featureWiseAllowed == null || featureWiseAllowed.isEmpty()) { + if (featureWiseAllowed == null || featureWiseAllowed.isEmpty()) { throw new Exception("feature map not found or empty for organization " + organization.getId()); } @@ -58,7 +58,7 @@ public String intercept(ActionInvocation invocation) throws Exception { for (String feature : features) { feature = feature.trim(); - if(feature.isEmpty()) { + if (feature.isEmpty()) { continue; } diff --git a/libs/dao/src/main/java/com/akto/dto/traffic/SuspectSampleData.java b/libs/dao/src/main/java/com/akto/dto/traffic/SuspectSampleData.java index ed544a6cfa..c8d4253039 100644 --- a/libs/dao/src/main/java/com/akto/dto/traffic/SuspectSampleData.java +++ b/libs/dao/src/main/java/com/akto/dto/traffic/SuspectSampleData.java @@ -1,9 +1,7 @@ package com.akto.dto.traffic; import java.util.List; -import java.util.Optional; -import com.fasterxml.jackson.databind.ObjectMapper; import org.bson.types.ObjectId; import com.akto.dto.type.URLMethods.Method; @@ -33,8 +31,6 @@ public class SuspectSampleData { */ String filterId; - private final ObjectMapper objectMapper = new ObjectMapper(); - public SuspectSampleData() {} public SuspectSampleData( @@ -158,20 +154,4 @@ public String toString() { + getSample() + "}"; } - - public Optional marshall() { - try { - return Optional.of(objectMapper.writeValueAsString(this)); - } catch (Exception e) { - return Optional.empty(); - } - } - - public Optional unmarshall() { - try { - return Optional.of(objectMapper.readValue(sample, SampleData.class)); - } catch (Exception e) { - return Optional.empty(); - } - } } diff --git a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java index ea364a1de5..5ef51a6dc5 100644 --- a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java +++ b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java @@ -3,7 +3,6 @@ import java.time.Duration; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -27,19 +26,25 @@ import com.mongodb.ConnectionString; public class KafkaRunner { - private Consumer consumer; - private static final LoggerMaker loggerMaker = new LoggerMaker(KafkaRunner.class, LogDb.RUNTIME); + private final Consumer consumer; + private final LogDb module; + private final LoggerMaker loggerMaker = new LoggerMaker(KafkaRunner.class, LogDb.RUNTIME); private static final DataActor dataActor = DataActorFactory.fetchInstance(); public static final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2); - public static void processKafkaRecords(LogDb module, List topics, - FailableFunction, Void, Exception> recordProcessor) { + public KafkaRunner(LogDb module, Consumer consumer) { + this.consumer = consumer; + this.module = module; + this.loggerMaker.setDb(module); + } - loggerMaker.setDb(module); + public void consume( + List topics, + FailableFunction, Void, Exception> recordProcessor) { boolean hybridSaas = RuntimeMode.isHybridDeployment(); - boolean connected =false; + boolean connected = false; if (hybridSaas) { AccountSettings accountSettings = dataActor.fetchAccountSettings(); if (accountSettings != null) { @@ -55,30 +60,17 @@ public static void processKafkaRecords(LogDb module, List topics, } if (connected) { - loggerMaker.infoAndAddToDb(String.format("Starting module for account : %d", Context.accountId.get())); - AllMetrics.instance.init(module); - } - - String kafkaBrokerUrl = "kafka1:19092"; - String isKubernetes = System.getenv("IS_KUBERNETES"); - if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { - loggerMaker.infoAndAddToDb("is_kubernetes: true"); - kafkaBrokerUrl = "127.0.0.1:29092"; + loggerMaker.infoAndAddToDb( + String.format("Starting module for account : %d", Context.accountId.get())); + AllMetrics.instance.init(this.module); } - String groupIdConfig = System.getenv("AKTO_KAFKA_GROUP_ID_CONFIG"); - int maxPollRecordsConfig = Integer - .parseInt(System.getenv().getOrDefault("AKTO_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); - - Properties properties = Utils.configProperties(kafkaBrokerUrl, groupIdConfig, maxPollRecordsConfig); - final KafkaRunner main = new KafkaRunner(); - main.consumer = new KafkaConsumer<>(properties); final Thread mainThread = Thread.currentThread(); final AtomicBoolean exceptionOnCommitSync = new AtomicBoolean(false); Runtime.getRuntime().addShutdownHook(new Thread() { public void run() { - main.consumer.wakeup(); + consumer.wakeup(); try { if (!exceptionOnCommitSync.get()) { mainThread.join(); @@ -92,21 +84,17 @@ public void run() { }); scheduler.scheduleAtFixedRate(() -> { - try { - logKafkaMetrics(main, module); - } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, - String.format("Failed to get kafka metrics for %s error: %s", module.name(), e.toString())); - } + logKafkaMetrics(); }, 0, 1, TimeUnit.MINUTES); try { - main.consumer.subscribe(topics); - loggerMaker.infoAndAddToDb(String.format("Consumer subscribed for topics : %s", topics.toString())); + consumer.subscribe(topics); + loggerMaker.infoAndAddToDb( + String.format("Consumer subscribed for topics : %s", topics.toString())); while (true) { - ConsumerRecords records = main.consumer.poll(Duration.ofMillis(10000)); + ConsumerRecords records = consumer.poll(Duration.ofMillis(10000)); try { - main.consumer.commitSync(); + consumer.commitSync(); } catch (Exception e) { throw e; } @@ -126,35 +114,49 @@ public void run() { e.printStackTrace(); System.exit(0); } finally { - main.consumer.close(); + consumer.close(); } } - private static void logKafkaMetrics(KafkaRunner main, LogDb module) { - Map metrics = main.consumer.metrics(); - for (Map.Entry entry : metrics.entrySet()) { - MetricName key = entry.getKey(); - Metric value = entry.getValue(); - - if (key.name().equals("records-lag-max")) { - double val = value.metricValue().equals(Double.NaN) ? 0d : (double) value.metricValue(); - AllMetrics.instance.setKafkaRecordsLagMax((float) val); - } - if (key.name().equals("records-consumed-rate")) { - double val = value.metricValue().equals(Double.NaN) ? 0d : (double) value.metricValue(); - AllMetrics.instance.setKafkaRecordsConsumedRate((float) val); - } + public void logKafkaMetrics() { + try { + Map metrics = this.consumer.metrics(); + for (Map.Entry entry : metrics.entrySet()) { + MetricName key = entry.getKey(); + Metric value = entry.getValue(); + + if (key.name().equals("records-lag-max")) { + double val = value.metricValue().equals(Double.NaN) + ? 0d + : (double) value.metricValue(); + AllMetrics.instance.setKafkaRecordsLagMax((float) val); + } + if (key.name().equals("records-consumed-rate")) { + double val = value.metricValue().equals(Double.NaN) + ? 0d + : (double) value.metricValue(); + AllMetrics.instance.setKafkaRecordsConsumedRate((float) val); + } - if (key.name().equals("fetch-latency-avg")) { - double val = value.metricValue().equals(Double.NaN) ? 0d : (double) value.metricValue(); - AllMetrics.instance.setKafkaFetchAvgLatency((float) val); - } + if (key.name().equals("fetch-latency-avg")) { + double val = value.metricValue().equals(Double.NaN) + ? 0d + : (double) value.metricValue(); + AllMetrics.instance.setKafkaFetchAvgLatency((float) val); + } - if (key.name().equals("bytes-consumed-rate")) { - double val = value.metricValue().equals(Double.NaN) ? 0d : (double) value.metricValue(); - AllMetrics.instance.setKafkaBytesConsumedRate((float) val); + if (key.name().equals("bytes-consumed-rate")) { + double val = value.metricValue().equals(Double.NaN) + ? 0d + : (double) value.metricValue(); + AllMetrics.instance.setKafkaBytesConsumedRate((float) val); + } } + } catch (Exception e) { + this.loggerMaker.errorAndAddToDb( + e, + String.format( + "Failed to get kafka metrics for %s error: %s", this.module.name(), e)); } } - -} \ No newline at end of file +} From 4550ea341d3c24528c1255f853e5a115ce7034b7 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 12 Nov 2024 11:24:40 +0530 Subject: [PATCH 03/73] added window based aggregation support for malicious requests --- apps/api-threat-detection/pom.xml | 12 ++ .../com/akto/cache/RedisWriteBackCache.java | 169 ++++++++++++++++++ .../java/com/akto/cache/TypeValueCache.java | 18 ++ .../java/com/akto/cache/TypeValueCode.java | 43 +++++ .../java/com/akto/filters/HttpCallFilter.java | 38 +++- .../RealTimeThresholdNotifier.java | 22 +++ .../key_generator/KeyGenerator.java | 13 ++ .../key_generator/SourceIPKeyGenerator.java | 18 ++ .../aggregators/window_based/Data.java | 52 ++++++ .../WindowBasedThresholdNotifier.java | 86 +++++++++ .../com/akto/malicious_request/Request.java | 70 ++++++++ .../notifier/MaliciousRequestsNotifier.java | 11 ++ .../notifier/SaveRedisNotifier.java | 21 +++ .../java/com/akto/threat/detection/Main.java | 31 ++-- .../WindowBasedThresholdNotifierTest.java | 118 ++++++++++++ .../java/com/akto/traffic/KafkaRunner.java | 19 +- 16 files changed, 722 insertions(+), 19 deletions(-) create mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCache.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCode.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/RealTimeThresholdNotifier.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/KeyGenerator.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Data.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/malicious_request/Request.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/MaliciousRequestsNotifier.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/SaveRedisNotifier.java create mode 100644 apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java diff --git a/apps/api-threat-detection/pom.xml b/apps/api-threat-detection/pom.xml index 64a129a342..4f5b3b1fc7 100644 --- a/apps/api-threat-detection/pom.xml +++ b/apps/api-threat-detection/pom.xml @@ -70,6 +70,18 @@ test + + io.lettuce + lettuce-core + 6.4.0.RELEASE + + + + com.github.ben-manes.caffeine + caffeine + 2.9.3 + + diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java new file mode 100644 index 0000000000..647bffea62 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java @@ -0,0 +1,169 @@ +package com.akto.cache; + +import io.lettuce.core.KeyValue; +import io.lettuce.core.RedisClient; +import io.lettuce.core.api.StatefulRedisConnection; +import io.lettuce.core.api.async.RedisAsyncCommands; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.*; + +import org.slf4j.LoggerFactory; +import org.slf4j.Logger; + +import java.util.Optional; + +import com.github.benmanes.caffeine.cache.Caffeine; + +public class RedisWriteBackCache implements TypeValueCache { + + class InsertEntry { + String key; + V value; + + public InsertEntry(String key, V value) { + this.key = key; + this.value = value; + } + + public String getKey() { + return key; + } + + public V getValue() { + return value; + } + } + + private final com.github.benmanes.caffeine.cache.Cache inMemoryCache; + private final ConcurrentLinkedQueue insertQueue; + private final ScheduledExecutorService scheduler; + private final StatefulRedisConnection redis; + private final String prefix; + + private static final int WRITE_SYNC_INTERVAL = 10; // in seconds + private static final int READ_SYNC_INTERVAL = 10; // in seconds + + private static final int MAX_CACHE_SIZE = 100_000; + private static final int CACHE_EXPIRY_MINUTES = 60 * 2; + + private static final Logger logger = LoggerFactory.getLogger(RedisWriteBackCache.class); + + public RedisWriteBackCache(RedisClient redisClient, String prefix) { + this.redis = redisClient.connect(new TypeValueCode()); + this.prefix = prefix; + this.insertQueue = new ConcurrentLinkedQueue<>(); + + // Schedule a task to write the batch to Redis every minute + this.scheduler = Executors.newScheduledThreadPool(1); + + // Periodically sync the in-memory cache with Redis + this.scheduler.scheduleAtFixedRate( + this::syncWrite, WRITE_SYNC_INTERVAL, WRITE_SYNC_INTERVAL, TimeUnit.SECONDS); + this.scheduler.scheduleAtFixedRate( + this::syncRead, READ_SYNC_INTERVAL, READ_SYNC_INTERVAL, TimeUnit.SECONDS); + + this.inMemoryCache = Caffeine.newBuilder() + .maximumSize(MAX_CACHE_SIZE) + .expireAfterWrite(CACHE_EXPIRY_MINUTES, TimeUnit.MINUTES) + .build(); + } + + public boolean containsKey(String key) { + return inMemoryCache.asMap().containsKey(key); + } + + public Optional get(String key) { + try { + return Optional.ofNullable(inMemoryCache.getIfPresent(key)); + } catch (Exception e) { + e.printStackTrace(); + return Optional.empty(); + } + + } + + public V getOrDefault(String key, V defaultValue) { + try { + return this.get(key).orElse(defaultValue); + } catch (Exception e) { + e.printStackTrace(); + return defaultValue; + } + + } + + public long size() { + return inMemoryCache.estimatedSize(); + } + + public void put(String key, V value) { + inMemoryCache.put(key, value); + + // Add the key to the insert queue + insertQueue.add(new InsertEntry(key, value)); + } + + public void destroy() { + scheduler.shutdown(); + redis.close(); + } + + public void syncWrite() { + if (insertQueue.isEmpty()) { + logger.info("[REDIS-{}] No writes to sync", prefix); + return; + } + + try { + RedisAsyncCommands async = this.redis.async(); + Map map = new HashMap<>(); + for (InsertEntry entry : insertQueue) { + map.put(entry.getKey(), entry.getValue()); + } + + async.hset(this.prefix, map).whenComplete((result, ex) -> { + if (ex != null) { + logger.error("[REDIS-{}] Failed to write to Redis", prefix, ex); + return; + } + + this.insertQueue.clear(); + }); + } catch (Exception e) { + e.printStackTrace(); + } + } + + private void syncRead() { + try { + if (this.size() == 0L) { + logger.debug("[REDIS-{}] No reads to sync", prefix); + return; + } + + RedisAsyncCommands async = this.redis.async(); + String[] keys = this.inMemoryCache.asMap().keySet().toArray(new String[0]); + + // NOTE: We only read the keys that are present in local cache + async.hmget(prefix, keys).whenComplete((dlist, ex) -> { + if (ex != null) { + logger.error("[REDIS-{}] Failed to read from Redis", prefix, ex); + return; + } + + for (KeyValue kv : dlist) { + if (kv.hasValue()) { + logger.debug("[REDIS-{}] Syncing key: {}, value: {}", prefix, kv.getKey(), kv.getValue()); + inMemoryCache.put(kv.getKey(), kv.getValue()); + } + } + }); + } catch (Exception e) { + e.printStackTrace(); + } finally { + logger.debug("[REDIS-{}] Synced reads with Redis", prefix); + } + } +} \ No newline at end of file diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCache.java new file mode 100644 index 0000000000..e6379293cc --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCache.java @@ -0,0 +1,18 @@ +package com.akto.cache; + +import java.util.Optional; + +public interface TypeValueCache { + + Optional get(String key); + + V getOrDefault(String key, V defaultValue); + + boolean containsKey(String key); + + void put(String key, V value); + + long size(); + + void destroy(); +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCode.java b/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCode.java new file mode 100644 index 0000000000..55172e2eef --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCode.java @@ -0,0 +1,43 @@ +package com.akto.cache; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.lettuce.core.codec.RedisCodec; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; + +public class TypeValueCode implements RedisCodec { + private final Charset charset = StandardCharsets.UTF_8; + private final ObjectMapper mapper = new ObjectMapper(); + + public String decodeKey(ByteBuffer bytes) { + return charset.decode(bytes).toString(); + } + + public V decodeValue(ByteBuffer byteBuffer) { + try { + // Generate byte array from ByteBuffer + String jsonString = charset.decode(byteBuffer).toString(); + return this.mapper.readValue(jsonString, new TypeReference() { + }); + } catch (Exception e) { + System.out.println("Error decoding value: " + e); + return null; + } + } + + public ByteBuffer encodeKey(String key) { + return ByteBuffer.wrap(key.getBytes(charset)); + } + + public ByteBuffer encodeValue(V value) { + try { + byte[] bytes = this.mapper.writeValueAsBytes(value); + return ByteBuffer.wrap(bytes); + } catch (IOException e) { + return null; + } + } +} \ No newline at end of file diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 6821b74a66..2e955120ef 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -2,6 +2,7 @@ import java.util.*; +import com.akto.cache.RedisWriteBackCache; import com.akto.dao.context.Context; import com.akto.dao.monitoring.FilterYamlTemplateDao; import com.akto.data_actor.DataActor; @@ -14,16 +15,22 @@ import com.akto.dto.test_editor.YamlTemplate; import com.akto.dto.traffic.SuspectSampleData; import com.akto.dto.type.URLMethods.Method; +import com.akto.filters.aggregators.key_generator.SourceIPKeyGenerator; +import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier; import com.akto.hybrid_parsers.HttpCallParser; import com.akto.kafka.Kafka; import com.akto.log.LoggerMaker; import com.akto.log.LoggerMaker.LogDb; +import com.akto.malicious_request.Request; +import com.akto.malicious_request.notifier.SaveRedisNotifier; import com.akto.rules.TestPlugin; import com.akto.runtime.policies.ApiAccessTypePolicy; import com.akto.suspect_data.Message; import com.akto.test_editor.execution.VariableResolver; import com.akto.test_editor.filter.data_operands_impl.ValidationResult; +import io.lettuce.core.RedisClient; + public class HttpCallFilter { private static final LoggerMaker loggerMaker = new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); @@ -40,12 +47,16 @@ public class HttpCallFilter { private static final DataActor dataActor = DataActorFactory.fetchInstance(); - public HttpCallFilter(int sync_threshold_count, int sync_threshold_time) { + private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; + + private final SaveRedisNotifier saveRedisNotifier; + + public HttpCallFilter(RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { this.apiFilters = new HashMap<>(); this.lastFilterFetch = 0; this.httpCallParser = new HttpCallParser(sync_threshold_count, sync_threshold_time); - String kafkaBootstrapServers = "kafka1:19092"; + String kafkaBootstrapServers = "localhost:29092"; String isKubernetes = System.getenv("IS_KUBERNETES"); if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { loggerMaker.infoAndAddToDb("is_kubernetes: true"); @@ -53,6 +64,11 @@ public HttpCallFilter(int sync_threshold_count, int sync_threshold_time) { } this.kafka = new Kafka(kafkaBootstrapServers, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); + this.windowBasedThresholdNotifier = new WindowBasedThresholdNotifier( + new RedisWriteBackCache<>(redisClient, "wbt"), + new WindowBasedThresholdNotifier.Config(100, 10 * 60)); + + this.saveRedisNotifier = new SaveRedisNotifier(redisClient); } public void filterFunction(List responseParams) { @@ -97,6 +113,24 @@ public void filterFunction(List responseParams) { // Eg: 100 4xx requests in last 10 minutes. // But regardless of whether request falls in aggregation or not, // we still push malicious requests to kafka + + SourceIPKeyGenerator.instance.generate(responseParam).ifPresent(aggKey -> { + boolean thresholdBreached = this.windowBasedThresholdNotifier.shouldNotify(aggKey, + responseParam); + + if (!thresholdBreached) { + return; + } + + Request request = new Request( + UUID.randomUUID().toString(), + apiFilter.getId(), + aggKey, + System.currentTimeMillis()); + + // For now just writing breached IP to redis + this.saveRedisNotifier.notifyRequest(request); + }); } } } diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/RealTimeThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/RealTimeThresholdNotifier.java new file mode 100644 index 0000000000..07d0828248 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/RealTimeThresholdNotifier.java @@ -0,0 +1,22 @@ +package com.akto.filters.aggregators; + +import com.akto.dto.HttpResponseParams; + +/* + * RealTimeThresholdNotifier is an abstract class that provides the basic structure for all the notifiers. + * It provides the basic structure for the notifiers to work with. + */ +public abstract class RealTimeThresholdNotifier { + + protected static final int NOTIFICATION_COOLDOWN_MINUTES = 60; + + /* + * Check if the aggregator should notify the system. + * Types of notifiers: + * Window based Threshold based notifier (for something like X bad requests in + * last Y minutes) + */ + public boolean shouldNotify(String aggKey, HttpResponseParams responseParam) { + throw new UnsupportedOperationException("Not implemented"); + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/KeyGenerator.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/KeyGenerator.java new file mode 100644 index 0000000000..2da78aa11b --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/KeyGenerator.java @@ -0,0 +1,13 @@ +package com.akto.filters.aggregators.key_generator; + +import com.akto.dto.HttpResponseParams; +import java.util.Optional; + +public interface KeyGenerator { + + /* + * Get the aggregation key. + * Key can be something like source IP + */ + Optional generate(HttpResponseParams responseParams); +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java new file mode 100644 index 0000000000..cdefdfefeb --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java @@ -0,0 +1,18 @@ +package com.akto.filters.aggregators.key_generator; + +import com.akto.dto.HttpResponseParams; + +import java.util.Optional; + +public class SourceIPKeyGenerator implements KeyGenerator { + + private SourceIPKeyGenerator() { + } + + public static SourceIPKeyGenerator instance = new SourceIPKeyGenerator(); + + @Override + public Optional generate(HttpResponseParams responseParams) { + return Optional.of(responseParams.getSourceIP()); + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Data.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Data.java new file mode 100644 index 0000000000..f93595edd5 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Data.java @@ -0,0 +1,52 @@ +package com.akto.filters.aggregators.window_based; + +import java.util.ArrayList; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class Data { + @JsonProperty("ln") + public long lastNotifiedAt = 0; + + @JsonProperty("rq") + public List requests = new ArrayList<>(); + + public static class Request { + private long receivedAt; + + public Request() { + } + + public Request(long receivedAt) { + this.receivedAt = receivedAt; + } + + public long getReceivedAt() { + return receivedAt; + } + + public void setReceivedAt(long receivedAt) { + this.receivedAt = receivedAt; + } + } + + public Data() { + } + + public long getLastNotifiedAt() { + return lastNotifiedAt; + } + + public void setLastNotifiedAt(long lastNotifiedAt) { + this.lastNotifiedAt = lastNotifiedAt; + } + + public List getRequests() { + return requests; + } + + public void setRequests(List requests) { + this.requests = requests; + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java new file mode 100644 index 0000000000..488fc76467 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -0,0 +1,86 @@ +package com.akto.filters.aggregators.window_based; + +import com.akto.cache.TypeValueCache; +import com.akto.dto.HttpResponseParams; +import com.akto.filters.aggregators.RealTimeThresholdNotifier; + +import java.util.List; + +public class WindowBasedThresholdNotifier extends RealTimeThresholdNotifier { + + private final Config config; + + public static class Config { + private final int threshold; + private final int windowInSeconds; + + public Config(int threshold, int windowInSeconds) { + this.threshold = threshold; + this.windowInSeconds = windowInSeconds; + } + + public int getThreshold() { + return threshold; + } + + public int getWindowInSeconds() { + return windowInSeconds; + } + } + + private final TypeValueCache cache; + + public WindowBasedThresholdNotifier(TypeValueCache cache, Config config) { + this.cache = cache; + this.config = config; + } + + @Override + public boolean shouldNotify(String aggKey, HttpResponseParams responseParam) { + long now = System.currentTimeMillis(); + + Data data = this.cache.getOrDefault(aggKey, new Data()); + List requests = data.getRequests(); + requests.add(new Data.Request(responseParam.getTime() * 1000L)); + + // Check if the current request is in the same window as previous requests + boolean sameWindow = now - requests.get(0).getReceivedAt() <= this.config.getWindowInSeconds() * 1000L; + + // Qualify request for notification pending last notified verification + boolean thresholdCrossedForWindow = false; + + if (!sameWindow) { + // Remove all the requests that are outside the window + while (!requests.isEmpty() + && now - requests.get(0).getReceivedAt() > this.config.getWindowInSeconds() * 1000L) { + requests.remove(0); + } + } else { + thresholdCrossedForWindow = requests.size() >= this.config.getThreshold(); + + // This is to ensure that we don't keep on adding requests to the list + // Eg: 10k requests in 1 second qualify. So we keep only last N requests + // where N is the threshold + while (requests.size() > this.config.getThreshold()) { + requests.remove(0); + } + } + + boolean shouldNotify = thresholdCrossedForWindow; + // Note: This also has a dependency on the cache expiry. If cache expiry is less + // than notification cooldown, then this will not work as expected. + // Eg: If cache expiry is 1 minute and notification cooldown is 1 hour, then + // this will always notify. + if (thresholdCrossedForWindow) { + if (now - data.getLastNotifiedAt() >= NOTIFICATION_COOLDOWN_MINUTES * 60 * 1000L) { + data.setLastNotifiedAt(now); + } else { + shouldNotify = false; + } + } + + data.setRequests(requests); + this.cache.put(aggKey, data); + return shouldNotify; + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/Request.java b/apps/api-threat-detection/src/main/java/com/akto/malicious_request/Request.java new file mode 100644 index 0000000000..a13e0d0c39 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/malicious_request/Request.java @@ -0,0 +1,70 @@ +package com.akto.malicious_request; + +import java.util.Optional; + +import com.fasterxml.jackson.databind.ObjectMapper; + +public class Request { + + private String id; + + private String filterId; + + private long detectedAt; + + private String actor; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + public Request() { + } + + public Request(String id, String filterId, String actor, long detectedAt) { + this.id = id; + this.filterId = filterId; + this.detectedAt = detectedAt; + this.actor = actor; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getFilterId() { + return filterId; + } + + public void setFilterId(String filterId) { + this.filterId = filterId; + } + + public long getDetectedAt() { + return detectedAt; + } + + public void setDetectedAt(long detectedAt) { + this.detectedAt = detectedAt; + } + + public String getActor() { + return actor; + } + + public void setActor(String actor) { + this.actor = actor; + } + + public Optional marshall() { + try { + return Optional.of(this.objectMapper.writeValueAsString(this)); + } catch (Exception e) { + e.printStackTrace(); + return Optional.empty(); + } + } + +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/MaliciousRequestsNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/MaliciousRequestsNotifier.java new file mode 100644 index 0000000000..ba57f06e4c --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/MaliciousRequestsNotifier.java @@ -0,0 +1,11 @@ +package com.akto.malicious_request.notifier; + +import com.akto.malicious_request.Request; + +public interface MaliciousRequestsNotifier { + + // This method is used to notify the system about a malicious request + // Notification can be of any type, like email, SMS, logging it somewhere + // or save it to a database + void notifyRequest(Request request); +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/SaveRedisNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/SaveRedisNotifier.java new file mode 100644 index 0000000000..bef0649889 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/SaveRedisNotifier.java @@ -0,0 +1,21 @@ +package com.akto.malicious_request.notifier; + +import com.akto.malicious_request.Request; + +import io.lettuce.core.RedisClient; +import io.lettuce.core.api.StatefulRedisConnection; + +public class SaveRedisNotifier implements MaliciousRequestsNotifier { + + private final StatefulRedisConnection redis; + + public SaveRedisNotifier(RedisClient redisClient) { + this.redis = redisClient.connect(); + } + + @Override + public void notifyRequest(Request request) { + // Save the request to Redis + request.marshall().ifPresent(data -> this.redis.sync().hset("malicious_requests", request.getId(), data)); + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java index 3e04c81c60..1af9bf02eb 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -4,7 +4,6 @@ import com.akto.suspect_data.FlushMessagesTask; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -15,6 +14,9 @@ import com.akto.metrics.AllMetrics; import com.akto.runtime.utils.Utils; import com.akto.traffic.KafkaRunner; + +import io.lettuce.core.RedisClient; + import com.akto.filters.HttpCallFilter; import com.akto.parsers.HttpCallParser; @@ -28,6 +30,8 @@ public class Main { private static final Map httpCallFilterMap = new HashMap<>(); + private static final RedisClient redisClient = createRedisClient(); + public static void main(String[] args) { // Flush Messages task @@ -42,19 +46,7 @@ public static void main(String[] args) { topicName = defaultTopic; } - String kafkaBrokerUrl = "127.0.0.1:29092"; - String isKubernetes = System.getenv("IS_KUBERNETES"); - if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { - loggerMaker.infoAndAddToDb("is_kubernetes: true"); - kafkaBrokerUrl = "127.0.0.1:29092"; - } - String groupId = "akto-threat-detection"; - int maxPollRecords = Integer.parseInt( - System.getenv().getOrDefault("AKTO_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); - - Properties kafkaRecordProperties = Utils.configProperties(kafkaBrokerUrl, groupId, maxPollRecords); - - KafkaRunner recordRunner = new KafkaRunner(module, new KafkaConsumer<>(kafkaRecordProperties)); + KafkaRunner recordRunner = new KafkaRunner(module); recordRunner.consume( Collections.singletonList(topicName), records -> { @@ -103,7 +95,8 @@ public static void processRecords(ConsumerRecords records) { Context.accountId.set(accountIdInt); if (!httpCallFilterMap.containsKey(accountId)) { - HttpCallFilter filter = new HttpCallFilter(sync_threshold_count, sync_threshold_time); + HttpCallFilter filter = new HttpCallFilter(redisClient, sync_threshold_count, + sync_threshold_time); httpCallFilterMap.put(accountId, filter); loggerMaker.infoAndAddToDb("New filter created for account: " + accountId); } @@ -115,4 +108,12 @@ public static void processRecords(ConsumerRecords records) { AllMetrics.instance.setRuntimeProcessLatency(System.currentTimeMillis() - start); } + + public static RedisClient createRedisClient() { + String host = System.getenv("AKTO_PROTECTION_REDIS_HOST"); + int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_PROTECTION_REDIS_PORT", "6379")); + int database = Integer.parseInt(System.getenv().getOrDefault("AKTO_PROTECTION_REDIS_DB", "0")); + + return RedisClient.create("redis://" + host + ":" + port + "/" + database); + } } diff --git a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java new file mode 100644 index 0000000000..3be3863d92 --- /dev/null +++ b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java @@ -0,0 +1,118 @@ +package com.akto.filters.aggregators.window_based; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + +import java.util.HashMap; +import java.util.Optional; + +import com.akto.dto.HttpRequestParams; +import com.akto.dto.HttpResponseParams; +import java.util.concurrent.ConcurrentHashMap; + +import org.junit.Test; + +import com.akto.cache.TypeValueCache; + +class MemCache implements TypeValueCache { + + private final ConcurrentHashMap cache; + + public MemCache() { + this.cache = new ConcurrentHashMap<>(); + } + + @Override + public Optional get(String key) { + return Optional.ofNullable(cache.get(key)); + } + + @Override + public V getOrDefault(String key, V defaultValue) { + return this.cache.getOrDefault(key, defaultValue); + } + + @Override + public boolean containsKey(String key) { + return this.cache.containsKey(key); + } + + @Override + public void put(String key, V value) { + this.cache.put(key, value); + } + + @Override + public long size() { + return this.cache.size(); + } + + @Override + public void destroy() { + this.cache.clear(); + } +} + +public class WindowBasedThresholdNotifierTest { + private static HttpResponseParams generateResponseParamsForStatusCode(int statusCode) { + return new HttpResponseParams( + "HTTP/1.1", + statusCode, + "Bad Request", + new HashMap<>(), + "{'error': 'Bad Request'}", + new HttpRequestParams( + "POST", + "/api/v1/endpoint", + "HTTP/1.1", + new HashMap<>(), + "{'error': 'Bad Request'}", + 1), + (int) (System.currentTimeMillis() / 1000L), + "100000", + false, + HttpResponseParams.Source.OTHER, + "", + "192.168.0.1"); + } + + @Test + public void testShouldNotify() throws InterruptedException { + + MemCache cache = new MemCache<>(); + WindowBasedThresholdNotifier notifier = new WindowBasedThresholdNotifier( + cache, new WindowBasedThresholdNotifier.Config(10, 1)); + + boolean shouldNotify = false; + String ip = "192.168.0.1"; + + for (int i = 0; i < 1000; i++) { + shouldNotify = shouldNotify + || notifier.shouldNotify( + ip, + WindowBasedThresholdNotifierTest + .generateResponseParamsForStatusCode(400)); + } + + Data data = cache.get(ip).orElse(new Data()); + assertEquals(10, data.getRequests().size()); + + long lastNotifiedAt = data.getLastNotifiedAt(); + assertNotEquals(lastNotifiedAt, 0); + + assertTrue(shouldNotify); + + Thread.sleep(2000L); + + shouldNotify = notifier.shouldNotify( + ip, + WindowBasedThresholdNotifierTest.generateResponseParamsForStatusCode(401)); + assertFalse(shouldNotify); + + data = cache.get(ip).orElse(new Data()); + + } + +} diff --git a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java index 5ef51a6dc5..c377714818 100644 --- a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java +++ b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java @@ -3,6 +3,7 @@ import java.time.Duration; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -30,13 +31,27 @@ public class KafkaRunner { private final LogDb module; private final LoggerMaker loggerMaker = new LoggerMaker(KafkaRunner.class, LogDb.RUNTIME); private static final DataActor dataActor = DataActorFactory.fetchInstance(); + private static final String KAFKA_GROUP_ID = "akto-threat-detection"; public static final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2); - public KafkaRunner(LogDb module, Consumer consumer) { - this.consumer = consumer; + public KafkaRunner(LogDb module) { this.module = module; this.loggerMaker.setDb(module); + this.consumer = new KafkaConsumer<>(generateKafkaProperties()); + } + + private Properties generateKafkaProperties() { + String kafkaBrokerUrl = "127.0.0.1:29092"; + String isKubernetes = System.getenv("IS_KUBERNETES"); + if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { + loggerMaker.infoAndAddToDb("is_kubernetes: true"); + kafkaBrokerUrl = "127.0.0.1:29092"; + } + int maxPollRecords = Integer.parseInt( + System.getenv().getOrDefault("AKTO_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); + + return Utils.configProperties(kafkaBrokerUrl, KAFKA_GROUP_ID, maxPollRecords); } public void consume( From 731d25e48ded8e43539c69be932e9541e73e7e74 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 12 Nov 2024 14:32:00 +0530 Subject: [PATCH 04/73] using separate mongo for storing suspect data --- .../java/com/akto/filters/HttpCallFilter.java | 28 ++++---- .../notifier/MaliciousRequestsNotifier.java | 11 ---- .../notifier/SaveRedisNotifier.java | 21 ------ .../java/com/akto/threat/detection/Main.java | 13 +++- .../DetectedThreatAlertDao.java | 20 ++++++ .../threat_detection/DetectedThreatAlert.java | 9 ++- .../java/com/akto/traffic/KafkaRunner.java | 64 ++++++++----------- 7 files changed, 73 insertions(+), 93 deletions(-) delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/MaliciousRequestsNotifier.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/SaveRedisNotifier.java create mode 100644 libs/dao/src/main/java/com/akto/dao/threat_detection/DetectedThreatAlertDao.java rename apps/api-threat-detection/src/main/java/com/akto/malicious_request/Request.java => libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java (85%) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 2e955120ef..b194ec68d3 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -5,6 +5,7 @@ import com.akto.cache.RedisWriteBackCache; import com.akto.dao.context.Context; import com.akto.dao.monitoring.FilterYamlTemplateDao; +import com.akto.dao.threat_detection.DetectedThreatAlertDao; import com.akto.data_actor.DataActor; import com.akto.data_actor.DataActorFactory; import com.akto.dto.ApiInfo.ApiInfoKey; @@ -13,6 +14,7 @@ import com.akto.dto.RawApi; import com.akto.dto.monitoring.FilterConfig; import com.akto.dto.test_editor.YamlTemplate; +import com.akto.dto.threat_detection.DetectedThreatAlert; import com.akto.dto.traffic.SuspectSampleData; import com.akto.dto.type.URLMethods.Method; import com.akto.filters.aggregators.key_generator.SourceIPKeyGenerator; @@ -21,8 +23,6 @@ import com.akto.kafka.Kafka; import com.akto.log.LoggerMaker; import com.akto.log.LoggerMaker.LogDb; -import com.akto.malicious_request.Request; -import com.akto.malicious_request.notifier.SaveRedisNotifier; import com.akto.rules.TestPlugin; import com.akto.runtime.policies.ApiAccessTypePolicy; import com.akto.suspect_data.Message; @@ -49,8 +49,6 @@ public class HttpCallFilter { private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; - private final SaveRedisNotifier saveRedisNotifier; - public HttpCallFilter(RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { this.apiFilters = new HashMap<>(); this.lastFilterFetch = 0; @@ -67,8 +65,6 @@ public HttpCallFilter(RedisClient redisClient, int sync_threshold_count, int syn this.windowBasedThresholdNotifier = new WindowBasedThresholdNotifier( new RedisWriteBackCache<>(redisClient, "wbt"), new WindowBasedThresholdNotifier.Config(100, 10 * 60)); - - this.saveRedisNotifier = new SaveRedisNotifier(redisClient); } public void filterFunction(List responseParams) { @@ -118,18 +114,16 @@ public void filterFunction(List responseParams) { boolean thresholdBreached = this.windowBasedThresholdNotifier.shouldNotify(aggKey, responseParam); - if (!thresholdBreached) { - return; - } + // TODO: Add window id with each suspect sample data and alert + if (thresholdBreached) { + DetectedThreatAlert alert = new DetectedThreatAlert( + UUID.randomUUID().toString(), + apiFilter.getId(), + aggKey, + System.currentTimeMillis()); - Request request = new Request( - UUID.randomUUID().toString(), - apiFilter.getId(), - aggKey, - System.currentTimeMillis()); - - // For now just writing breached IP to redis - this.saveRedisNotifier.notifyRequest(request); + DetectedThreatAlertDao.instance.insertOne(alert); + } }); } } diff --git a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/MaliciousRequestsNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/MaliciousRequestsNotifier.java deleted file mode 100644 index ba57f06e4c..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/MaliciousRequestsNotifier.java +++ /dev/null @@ -1,11 +0,0 @@ -package com.akto.malicious_request.notifier; - -import com.akto.malicious_request.Request; - -public interface MaliciousRequestsNotifier { - - // This method is used to notify the system about a malicious request - // Notification can be of any type, like email, SMS, logging it somewhere - // or save it to a database - void notifyRequest(Request request); -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/SaveRedisNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/SaveRedisNotifier.java deleted file mode 100644 index bef0649889..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/notifier/SaveRedisNotifier.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.akto.malicious_request.notifier; - -import com.akto.malicious_request.Request; - -import io.lettuce.core.RedisClient; -import io.lettuce.core.api.StatefulRedisConnection; - -public class SaveRedisNotifier implements MaliciousRequestsNotifier { - - private final StatefulRedisConnection redis; - - public SaveRedisNotifier(RedisClient redisClient) { - this.redis = redisClient.connect(); - } - - @Override - public void notifyRequest(Request request) { - // Save the request to Redis - request.marshall().ifPresent(data -> this.redis.sync().hset("malicious_requests", request.getId(), data)); - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java index 1af9bf02eb..00e74c1bde 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -7,6 +7,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.akto.DaoInit; import com.akto.dao.context.Context; import com.akto.dto.HttpResponseParams; import com.akto.log.LoggerMaker; @@ -14,6 +15,7 @@ import com.akto.metrics.AllMetrics; import com.akto.runtime.utils.Utils; import com.akto.traffic.KafkaRunner; +import com.mongodb.ConnectionString; import io.lettuce.core.RedisClient; @@ -33,6 +35,11 @@ public class Main { private static final RedisClient redisClient = createRedisClient(); public static void main(String[] args) { + // We have a separate Mongo for storing threat detection data + // Metadata is stored in the main Mongo, which we call using API + // So we always need to enable hybrid mode for this module + String mongoURI = System.getenv("AKTO_THREAT_DETECTION_MONGO_CONN"); + DaoInit.init(new ConnectionString(mongoURI)); // Flush Messages task FlushMessagesTask.instance.init(); @@ -46,8 +53,8 @@ public static void main(String[] args) { topicName = defaultTopic; } - KafkaRunner recordRunner = new KafkaRunner(module); - recordRunner.consume( + KafkaRunner.consume( + module, Collections.singletonList(topicName), records -> { processRecords(records); @@ -110,7 +117,7 @@ public static void processRecords(ConsumerRecords records) { } public static RedisClient createRedisClient() { - String host = System.getenv("AKTO_PROTECTION_REDIS_HOST"); + String host = System.getenv().getOrDefault("AKTO_PROTECTION_REDIS_HOST", "localhost"); int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_PROTECTION_REDIS_PORT", "6379")); int database = Integer.parseInt(System.getenv().getOrDefault("AKTO_PROTECTION_REDIS_DB", "0")); diff --git a/libs/dao/src/main/java/com/akto/dao/threat_detection/DetectedThreatAlertDao.java b/libs/dao/src/main/java/com/akto/dao/threat_detection/DetectedThreatAlertDao.java new file mode 100644 index 0000000000..8bec8986e7 --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dao/threat_detection/DetectedThreatAlertDao.java @@ -0,0 +1,20 @@ +package com.akto.dao.threat_detection; + +import com.akto.dao.AccountsContextDao; +import com.akto.dto.threat_detection.DetectedThreatAlert; + +public class DetectedThreatAlertDao extends AccountsContextDao { + + public static final DetectedThreatAlertDao instance = new DetectedThreatAlertDao(); + + @Override + public String getCollName() { + return "detected_threat_alerts"; + } + + @Override + public Class getClassT() { + return DetectedThreatAlert.class; + } + +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/Request.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java similarity index 85% rename from apps/api-threat-detection/src/main/java/com/akto/malicious_request/Request.java rename to libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java index a13e0d0c39..cfaaa872ce 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/malicious_request/Request.java +++ b/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java @@ -1,11 +1,10 @@ -package com.akto.malicious_request; +package com.akto.dto.threat_detection; import java.util.Optional; import com.fasterxml.jackson.databind.ObjectMapper; -public class Request { - +public class DetectedThreatAlert { private String id; private String filterId; @@ -16,10 +15,10 @@ public class Request { private final ObjectMapper objectMapper = new ObjectMapper(); - public Request() { + public DetectedThreatAlert() { } - public Request(String id, String filterId, String actor, long detectedAt) { + public DetectedThreatAlert(String id, String filterId, String actor, long detectedAt) { this.id = id; this.filterId = filterId; this.detectedAt = detectedAt; diff --git a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java index c377714818..a331f90d67 100644 --- a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java +++ b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java @@ -24,24 +24,16 @@ import com.akto.log.LoggerMaker.LogDb; import com.akto.metrics.AllMetrics; import com.akto.runtime.utils.Utils; -import com.mongodb.ConnectionString; public class KafkaRunner { - private final Consumer consumer; - private final LogDb module; - private final LoggerMaker loggerMaker = new LoggerMaker(KafkaRunner.class, LogDb.RUNTIME); + private Consumer consumer; + private static final LoggerMaker loggerMaker = new LoggerMaker(KafkaRunner.class, LogDb.RUNTIME); private static final DataActor dataActor = DataActorFactory.fetchInstance(); private static final String KAFKA_GROUP_ID = "akto-threat-detection"; public static final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2); - public KafkaRunner(LogDb module) { - this.module = module; - this.loggerMaker.setDb(module); - this.consumer = new KafkaConsumer<>(generateKafkaProperties()); - } - - private Properties generateKafkaProperties() { + private static Properties generateKafkaProperties() { String kafkaBrokerUrl = "127.0.0.1:29092"; String isKubernetes = System.getenv("IS_KUBERNETES"); if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { @@ -54,30 +46,30 @@ private Properties generateKafkaProperties() { return Utils.configProperties(kafkaBrokerUrl, KAFKA_GROUP_ID, maxPollRecords); } - public void consume( - List topics, + public static void consume(LogDb module, List topics, FailableFunction, Void, Exception> recordProcessor) { + loggerMaker.setDb(module); + + final KafkaRunner main = new KafkaRunner(); + main.consumer = new KafkaConsumer<>(generateKafkaProperties()); + boolean hybridSaas = RuntimeMode.isHybridDeployment(); boolean connected = false; - if (hybridSaas) { - AccountSettings accountSettings = dataActor.fetchAccountSettings(); - if (accountSettings != null) { - int acc = accountSettings.getId(); - Context.accountId.set(acc); - connected = true; - } - } else { - String mongoURI = System.getenv("AKTO_MONGO_CONN"); - DaoInit.init(new ConnectionString(mongoURI)); - Context.accountId.set(1_000_000); + if (!hybridSaas) { + throw new RuntimeException("Hybrid mode is required for this module"); + } + + AccountSettings accountSettings = dataActor.fetchAccountSettings(); + if (accountSettings != null) { + int acc = accountSettings.getId(); + Context.accountId.set(acc); connected = true; } if (connected) { - loggerMaker.infoAndAddToDb( - String.format("Starting module for account : %d", Context.accountId.get())); - AllMetrics.instance.init(this.module); + loggerMaker.infoAndAddToDb(String.format("Starting module for account : %d", Context.accountId.get())); + AllMetrics.instance.init(module); } final Thread mainThread = Thread.currentThread(); @@ -85,7 +77,7 @@ public void consume( Runtime.getRuntime().addShutdownHook(new Thread() { public void run() { - consumer.wakeup(); + main.consumer.wakeup(); try { if (!exceptionOnCommitSync.get()) { mainThread.join(); @@ -99,17 +91,17 @@ public void run() { }); scheduler.scheduleAtFixedRate(() -> { - logKafkaMetrics(); + main.logKafkaMetrics(module); }, 0, 1, TimeUnit.MINUTES); try { - consumer.subscribe(topics); + main.consumer.subscribe(topics); loggerMaker.infoAndAddToDb( String.format("Consumer subscribed for topics : %s", topics.toString())); while (true) { - ConsumerRecords records = consumer.poll(Duration.ofMillis(10000)); + ConsumerRecords records = main.consumer.poll(Duration.ofMillis(10000)); try { - consumer.commitSync(); + main.consumer.commitSync(); } catch (Exception e) { throw e; } @@ -129,11 +121,11 @@ public void run() { e.printStackTrace(); System.exit(0); } finally { - consumer.close(); + main.consumer.close(); } } - public void logKafkaMetrics() { + public void logKafkaMetrics(LogDb module) { try { Map metrics = this.consumer.metrics(); for (Map.Entry entry : metrics.entrySet()) { @@ -168,10 +160,10 @@ public void logKafkaMetrics() { } } } catch (Exception e) { - this.loggerMaker.errorAndAddToDb( + loggerMaker.errorAndAddToDb( e, String.format( - "Failed to get kafka metrics for %s error: %s", this.module.name(), e)); + "Failed to get kafka metrics for %s error: %s", module.name(), e)); } } } From 6de974dba2a5c48a8b57e1f0e78e74a0ab82ff42 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 12 Nov 2024 15:48:00 +0530 Subject: [PATCH 05/73] fixed kafka host --- .../src/main/java/com/akto/filters/HttpCallFilter.java | 10 ++-------- .../java/com/akto/suspect_data/FlushMessagesTask.java | 7 +------ .../src/main/java/com/akto/threat/detection/Main.java | 6 +++--- .../src/main/java/com/akto/traffic/KafkaRunner.java | 7 +------ 4 files changed, 7 insertions(+), 23 deletions(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index b194ec68d3..227164fe2d 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -54,14 +54,8 @@ public HttpCallFilter(RedisClient redisClient, int sync_threshold_count, int syn this.lastFilterFetch = 0; this.httpCallParser = new HttpCallParser(sync_threshold_count, sync_threshold_time); - String kafkaBootstrapServers = "localhost:29092"; - String isKubernetes = System.getenv("IS_KUBERNETES"); - if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { - loggerMaker.infoAndAddToDb("is_kubernetes: true"); - kafkaBootstrapServers = "127.0.0.1:29092"; - } - - this.kafka = new Kafka(kafkaBootstrapServers, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); + String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); + this.kafka = new Kafka(kafkaBrokerUrl, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); this.windowBasedThresholdNotifier = new WindowBasedThresholdNotifier( new RedisWriteBackCache<>(redisClient, "wbt"), new WindowBasedThresholdNotifier.Config(100, 10 * 60)); diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java index 920d2b43f2..9e9a2da938 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java @@ -29,12 +29,7 @@ public class FlushMessagesTask { private final Consumer consumer; private FlushMessagesTask() { - String kafkaBrokerUrl = "127.0.0.1:29092"; - String isKubernetes = System.getenv("IS_KUBERNETES"); - if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { - kafkaBrokerUrl = "127.0.0.1:29092"; - } - + String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); String groupId = "akto-flush-malicious-messages"; Properties properties = Utils.configProperties(kafkaBrokerUrl, groupId, 100); diff --git a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java index 00e74c1bde..4cf060a657 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -117,9 +117,9 @@ public static void processRecords(ConsumerRecords records) { } public static RedisClient createRedisClient() { - String host = System.getenv().getOrDefault("AKTO_PROTECTION_REDIS_HOST", "localhost"); - int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_PROTECTION_REDIS_PORT", "6379")); - int database = Integer.parseInt(System.getenv().getOrDefault("AKTO_PROTECTION_REDIS_DB", "0")); + String host = System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_HOST", "localhost"); + int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_PORT", "6379")); + int database = Integer.parseInt(System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_DB", "0")); return RedisClient.create("redis://" + host + ":" + port + "/" + database); } diff --git a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java index a331f90d67..b81d3dc889 100644 --- a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java +++ b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java @@ -34,12 +34,7 @@ public class KafkaRunner { public static final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2); private static Properties generateKafkaProperties() { - String kafkaBrokerUrl = "127.0.0.1:29092"; - String isKubernetes = System.getenv("IS_KUBERNETES"); - if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { - loggerMaker.infoAndAddToDb("is_kubernetes: true"); - kafkaBrokerUrl = "127.0.0.1:29092"; - } + String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); int maxPollRecords = Integer.parseInt( System.getenv().getOrDefault("AKTO_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); From 6660653e98ea3ae92f38dbdd4e67d0ded04f7533 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 12 Nov 2024 19:15:43 +0530 Subject: [PATCH 06/73] fixed source ip generation logic --- .../aggregators/key_generator/SourceIPKeyGenerator.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java index cdefdfefeb..a70eed510a 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java @@ -1,7 +1,9 @@ package com.akto.filters.aggregators.key_generator; import com.akto.dto.HttpResponseParams; +import com.akto.runtime.policies.ApiAccessTypePolicy; +import java.util.List; import java.util.Optional; public class SourceIPKeyGenerator implements KeyGenerator { @@ -13,6 +15,11 @@ private SourceIPKeyGenerator() { @Override public Optional generate(HttpResponseParams responseParams) { - return Optional.of(responseParams.getSourceIP()); + List sourceIPs = ApiAccessTypePolicy.getSourceIps(responseParams); + if (sourceIPs.isEmpty()) { + return Optional.of(responseParams.getSourceIP()); + } + + return Optional.of(sourceIPs.get(0)); } } From 5d21d82554e08d1e7730de94e833604f95e620a4 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 13 Nov 2024 11:39:22 +0530 Subject: [PATCH 07/73] not syncing writes and read for RedisWriteBackCache at the same time --- .../src/main/java/com/akto/cache/RedisWriteBackCache.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java index 647bffea62..2152133fd2 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java @@ -62,7 +62,7 @@ public RedisWriteBackCache(RedisClient redisClient, String prefix) { this.scheduler.scheduleAtFixedRate( this::syncWrite, WRITE_SYNC_INTERVAL, WRITE_SYNC_INTERVAL, TimeUnit.SECONDS); this.scheduler.scheduleAtFixedRate( - this::syncRead, READ_SYNC_INTERVAL, READ_SYNC_INTERVAL, TimeUnit.SECONDS); + this::syncRead, 0, READ_SYNC_INTERVAL, TimeUnit.SECONDS); this.inMemoryCache = Caffeine.newBuilder() .maximumSize(MAX_CACHE_SIZE) @@ -95,7 +95,11 @@ public V getOrDefault(String key, V defaultValue) { } public long size() { - return inMemoryCache.estimatedSize(); + try { + return inMemoryCache.estimatedSize(); + } catch (Exception e) { + return 0L; + } } public void put(String key, V value) { From 534ad262d73eacb63daaa5730177edc7fe693072 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 13 Nov 2024 15:58:03 +0530 Subject: [PATCH 08/73] using redis backed increment counter cache --- .../java/com/akto/cache/CounterCache.java | 10 + .../java/com/akto/cache/LongValueCodec.java | 33 ++++ .../akto/cache/RedisBackedCounterCache.java | 90 +++++++++ .../com/akto/cache/RedisWriteBackCache.java | 173 ------------------ .../java/com/akto/cache/TypeValueCache.java | 18 -- .../java/com/akto/cache/TypeValueCode.java | 43 ----- .../java/com/akto/filters/HttpCallFilter.java | 4 +- .../WindowBasedThresholdNotifier.java | 66 ++----- .../WindowBasedThresholdNotifierTest.java | 73 +++----- .../java/com/akto/traffic/KafkaRunner.java | 1 - 10 files changed, 174 insertions(+), 337 deletions(-) create mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/LongValueCodec.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCache.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCode.java diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java new file mode 100644 index 0000000000..7786a05c25 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java @@ -0,0 +1,10 @@ +package com.akto.cache; + +public interface CounterCache { + + void incrementBy(String key, long val); + + void increment(String key); + + long get(String key); +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/LongValueCodec.java b/apps/api-threat-detection/src/main/java/com/akto/cache/LongValueCodec.java new file mode 100644 index 0000000000..2fce242dd5 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/LongValueCodec.java @@ -0,0 +1,33 @@ +package com.akto.cache; + +import io.lettuce.core.codec.RedisCodec; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +public class LongValueCodec implements RedisCodec { + + @Override + public String decodeKey(ByteBuffer bytes) { + return StandardCharsets.UTF_8.decode(bytes).toString(); + } + + @Override + public Long decodeValue(ByteBuffer bytes) { + if (!bytes.hasRemaining()) + return null; + return bytes.getLong(); + } + + @Override + public ByteBuffer encodeKey(String key) { + return StandardCharsets.UTF_8.encode(key); + } + + @Override + public ByteBuffer encodeValue(Long value) { + ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES); + buffer.putLong(value); + buffer.flip(); + return buffer; + } +} \ No newline at end of file diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java new file mode 100644 index 0000000000..fb316788f6 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java @@ -0,0 +1,90 @@ +package com.akto.cache; + +import io.lettuce.core.RedisClient; +import io.lettuce.core.api.StatefulRedisConnection; + +import java.util.Optional; +import java.util.concurrent.*; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; + +public class RedisBackedCounterCache implements CounterCache { + + class Op { + private final String key; + private final long value; + + public Op(String key, long value) { + this.key = key; + this.value = value; + } + + public String getKey() { + return key; + } + + public long getValue() { + return value; + } + } + + private final StatefulRedisConnection redis; + + private final Cache localCache; + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + private final ConcurrentLinkedQueue pendingOps; + private final String prefix; + + public RedisBackedCounterCache(RedisClient redisClient, String prefix) { + this.prefix = prefix; + this.redis = redisClient.connect(new LongValueCodec()); + this.localCache = Caffeine.newBuilder() + .maximumSize(100000) + .expireAfterWrite(1, TimeUnit.HOURS) + .build(); + this.executor.scheduleAtFixedRate(this::syncToRedis, 60, 1, TimeUnit.SECONDS); + this.pendingOps = new ConcurrentLinkedQueue<>(); + } + + private String getKey(String key) { + return new StringBuilder().append(prefix).append("|").append(key).toString(); + } + + @Override + public void increment(String key) { + incrementBy(key, 1); + } + + @Override + public void incrementBy(String key, long val) { + String _key = getKey(key); + localCache.asMap().merge(_key, val, Long::sum); + pendingOps.add(new Op(_key, val)); + } + + @Override + public long get(String key) { + return Optional.ofNullable(this.localCache.getIfPresent(getKey(key))).orElse(0L); + } + + private void syncToRedis() { + while (!pendingOps.isEmpty()) { + Op op = pendingOps.poll(); + String key = op.getKey(); + long val = op.getValue(); + redis.async() + .incrby(key, val) + .whenComplete( + (result, ex) -> { + if (ex != null) { + ex.printStackTrace(); + } + + if (result != null) { + localCache.asMap().put(key, result); + } + }); + } + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java deleted file mode 100644 index 2152133fd2..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisWriteBackCache.java +++ /dev/null @@ -1,173 +0,0 @@ -package com.akto.cache; - -import io.lettuce.core.KeyValue; -import io.lettuce.core.RedisClient; -import io.lettuce.core.api.StatefulRedisConnection; -import io.lettuce.core.api.async.RedisAsyncCommands; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.*; - -import org.slf4j.LoggerFactory; -import org.slf4j.Logger; - -import java.util.Optional; - -import com.github.benmanes.caffeine.cache.Caffeine; - -public class RedisWriteBackCache implements TypeValueCache { - - class InsertEntry { - String key; - V value; - - public InsertEntry(String key, V value) { - this.key = key; - this.value = value; - } - - public String getKey() { - return key; - } - - public V getValue() { - return value; - } - } - - private final com.github.benmanes.caffeine.cache.Cache inMemoryCache; - private final ConcurrentLinkedQueue insertQueue; - private final ScheduledExecutorService scheduler; - private final StatefulRedisConnection redis; - private final String prefix; - - private static final int WRITE_SYNC_INTERVAL = 10; // in seconds - private static final int READ_SYNC_INTERVAL = 10; // in seconds - - private static final int MAX_CACHE_SIZE = 100_000; - private static final int CACHE_EXPIRY_MINUTES = 60 * 2; - - private static final Logger logger = LoggerFactory.getLogger(RedisWriteBackCache.class); - - public RedisWriteBackCache(RedisClient redisClient, String prefix) { - this.redis = redisClient.connect(new TypeValueCode()); - this.prefix = prefix; - this.insertQueue = new ConcurrentLinkedQueue<>(); - - // Schedule a task to write the batch to Redis every minute - this.scheduler = Executors.newScheduledThreadPool(1); - - // Periodically sync the in-memory cache with Redis - this.scheduler.scheduleAtFixedRate( - this::syncWrite, WRITE_SYNC_INTERVAL, WRITE_SYNC_INTERVAL, TimeUnit.SECONDS); - this.scheduler.scheduleAtFixedRate( - this::syncRead, 0, READ_SYNC_INTERVAL, TimeUnit.SECONDS); - - this.inMemoryCache = Caffeine.newBuilder() - .maximumSize(MAX_CACHE_SIZE) - .expireAfterWrite(CACHE_EXPIRY_MINUTES, TimeUnit.MINUTES) - .build(); - } - - public boolean containsKey(String key) { - return inMemoryCache.asMap().containsKey(key); - } - - public Optional get(String key) { - try { - return Optional.ofNullable(inMemoryCache.getIfPresent(key)); - } catch (Exception e) { - e.printStackTrace(); - return Optional.empty(); - } - - } - - public V getOrDefault(String key, V defaultValue) { - try { - return this.get(key).orElse(defaultValue); - } catch (Exception e) { - e.printStackTrace(); - return defaultValue; - } - - } - - public long size() { - try { - return inMemoryCache.estimatedSize(); - } catch (Exception e) { - return 0L; - } - } - - public void put(String key, V value) { - inMemoryCache.put(key, value); - - // Add the key to the insert queue - insertQueue.add(new InsertEntry(key, value)); - } - - public void destroy() { - scheduler.shutdown(); - redis.close(); - } - - public void syncWrite() { - if (insertQueue.isEmpty()) { - logger.info("[REDIS-{}] No writes to sync", prefix); - return; - } - - try { - RedisAsyncCommands async = this.redis.async(); - Map map = new HashMap<>(); - for (InsertEntry entry : insertQueue) { - map.put(entry.getKey(), entry.getValue()); - } - - async.hset(this.prefix, map).whenComplete((result, ex) -> { - if (ex != null) { - logger.error("[REDIS-{}] Failed to write to Redis", prefix, ex); - return; - } - - this.insertQueue.clear(); - }); - } catch (Exception e) { - e.printStackTrace(); - } - } - - private void syncRead() { - try { - if (this.size() == 0L) { - logger.debug("[REDIS-{}] No reads to sync", prefix); - return; - } - - RedisAsyncCommands async = this.redis.async(); - String[] keys = this.inMemoryCache.asMap().keySet().toArray(new String[0]); - - // NOTE: We only read the keys that are present in local cache - async.hmget(prefix, keys).whenComplete((dlist, ex) -> { - if (ex != null) { - logger.error("[REDIS-{}] Failed to read from Redis", prefix, ex); - return; - } - - for (KeyValue kv : dlist) { - if (kv.hasValue()) { - logger.debug("[REDIS-{}] Syncing key: {}, value: {}", prefix, kv.getKey(), kv.getValue()); - inMemoryCache.put(kv.getKey(), kv.getValue()); - } - } - }); - } catch (Exception e) { - e.printStackTrace(); - } finally { - logger.debug("[REDIS-{}] Synced reads with Redis", prefix); - } - } -} \ No newline at end of file diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCache.java deleted file mode 100644 index e6379293cc..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCache.java +++ /dev/null @@ -1,18 +0,0 @@ -package com.akto.cache; - -import java.util.Optional; - -public interface TypeValueCache { - - Optional get(String key); - - V getOrDefault(String key, V defaultValue); - - boolean containsKey(String key); - - void put(String key, V value); - - long size(); - - void destroy(); -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCode.java b/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCode.java deleted file mode 100644 index 55172e2eef..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/TypeValueCode.java +++ /dev/null @@ -1,43 +0,0 @@ -package com.akto.cache; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import io.lettuce.core.codec.RedisCodec; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.charset.Charset; -import java.nio.charset.StandardCharsets; - -public class TypeValueCode implements RedisCodec { - private final Charset charset = StandardCharsets.UTF_8; - private final ObjectMapper mapper = new ObjectMapper(); - - public String decodeKey(ByteBuffer bytes) { - return charset.decode(bytes).toString(); - } - - public V decodeValue(ByteBuffer byteBuffer) { - try { - // Generate byte array from ByteBuffer - String jsonString = charset.decode(byteBuffer).toString(); - return this.mapper.readValue(jsonString, new TypeReference() { - }); - } catch (Exception e) { - System.out.println("Error decoding value: " + e); - return null; - } - } - - public ByteBuffer encodeKey(String key) { - return ByteBuffer.wrap(key.getBytes(charset)); - } - - public ByteBuffer encodeValue(V value) { - try { - byte[] bytes = this.mapper.writeValueAsBytes(value); - return ByteBuffer.wrap(bytes); - } catch (IOException e) { - return null; - } - } -} \ No newline at end of file diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 227164fe2d..2679b5222f 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -2,7 +2,7 @@ import java.util.*; -import com.akto.cache.RedisWriteBackCache; +import com.akto.cache.RedisBackedCounterCache; import com.akto.dao.context.Context; import com.akto.dao.monitoring.FilterYamlTemplateDao; import com.akto.dao.threat_detection.DetectedThreatAlertDao; @@ -57,7 +57,7 @@ public HttpCallFilter(RedisClient redisClient, int sync_threshold_count, int syn String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); this.kafka = new Kafka(kafkaBrokerUrl, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); this.windowBasedThresholdNotifier = new WindowBasedThresholdNotifier( - new RedisWriteBackCache<>(redisClient, "wbt"), + new RedisBackedCounterCache(redisClient, "wbt"), new WindowBasedThresholdNotifier.Config(100, 10 * 60)); } diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index 488fc76467..bd0d4f43b1 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -1,86 +1,52 @@ package com.akto.filters.aggregators.window_based; -import com.akto.cache.TypeValueCache; +import com.akto.cache.CounterCache; import com.akto.dto.HttpResponseParams; import com.akto.filters.aggregators.RealTimeThresholdNotifier; -import java.util.List; - public class WindowBasedThresholdNotifier extends RealTimeThresholdNotifier { private final Config config; public static class Config { private final int threshold; - private final int windowInSeconds; + private final int windowSizeInMinutes; public Config(int threshold, int windowInSeconds) { this.threshold = threshold; - this.windowInSeconds = windowInSeconds; + this.windowSizeInMinutes = windowInSeconds; } public int getThreshold() { return threshold; } - public int getWindowInSeconds() { - return windowInSeconds; + public int getWindowSizeInMinutes() { + return windowSizeInMinutes; } } - private final TypeValueCache cache; + private final CounterCache cache; - public WindowBasedThresholdNotifier(TypeValueCache cache, Config config) { + public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.cache = cache; this.config = config; } @Override - public boolean shouldNotify(String aggKey, HttpResponseParams responseParam) { - long now = System.currentTimeMillis(); - - Data data = this.cache.getOrDefault(aggKey, new Data()); - List requests = data.getRequests(); - requests.add(new Data.Request(responseParam.getTime() * 1000L)); - - // Check if the current request is in the same window as previous requests - boolean sameWindow = now - requests.get(0).getReceivedAt() <= this.config.getWindowInSeconds() * 1000L; + public boolean shouldNotify(String actor, HttpResponseParams responseParam) { + int requestTimeSeconds = responseParam.getTime(); - // Qualify request for notification pending last notified verification - boolean thresholdCrossedForWindow = false; + int minuteOfYear = (int) Math.ceil(requestTimeSeconds / (60L)); - if (!sameWindow) { - // Remove all the requests that are outside the window - while (!requests.isEmpty() - && now - requests.get(0).getReceivedAt() > this.config.getWindowInSeconds() * 1000L) { - requests.remove(0); - } - } else { - thresholdCrossedForWindow = requests.size() >= this.config.getThreshold(); - - // This is to ensure that we don't keep on adding requests to the list - // Eg: 10k requests in 1 second qualify. So we keep only last N requests - // where N is the threshold - while (requests.size() > this.config.getThreshold()) { - requests.remove(0); - } - } + String bucketKey = actor + "|" + minuteOfYear; + this.cache.increment(bucketKey); - boolean shouldNotify = thresholdCrossedForWindow; - // Note: This also has a dependency on the cache expiry. If cache expiry is less - // than notification cooldown, then this will not work as expected. - // Eg: If cache expiry is 1 minute and notification cooldown is 1 hour, then - // this will always notify. - if (thresholdCrossedForWindow) { - if (now - data.getLastNotifiedAt() >= NOTIFICATION_COOLDOWN_MINUTES * 60 * 1000L) { - data.setLastNotifiedAt(now); - } else { - shouldNotify = false; - } + long windowCount = 0L; + for (int i = minuteOfYear; i >= minuteOfYear - this.config.getWindowSizeInMinutes(); i--) { + windowCount += this.cache.get(actor + "|" + i); } - data.setRequests(requests); - this.cache.put(aggKey, data); - return shouldNotify; + return windowCount >= this.config.getThreshold(); } } diff --git a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java index 3be3863d92..dc1b609af7 100644 --- a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java +++ b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java @@ -1,57 +1,41 @@ package com.akto.filters.aggregators.window_based; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; - import java.util.HashMap; -import java.util.Optional; - +import java.util.Map; import com.akto.dto.HttpRequestParams; import com.akto.dto.HttpResponseParams; import java.util.concurrent.ConcurrentHashMap; import org.junit.Test; -import com.akto.cache.TypeValueCache; +import com.akto.cache.CounterCache; -class MemCache implements TypeValueCache { +class MemCache implements CounterCache { - private final ConcurrentHashMap cache; + private final ConcurrentHashMap cache; public MemCache() { this.cache = new ConcurrentHashMap<>(); } @Override - public Optional get(String key) { - return Optional.ofNullable(cache.get(key)); - } - - @Override - public V getOrDefault(String key, V defaultValue) { - return this.cache.getOrDefault(key, defaultValue); + public void incrementBy(String key, long val) { + cache.put(key, cache.getOrDefault(key, 0L) + val); } @Override - public boolean containsKey(String key) { - return this.cache.containsKey(key); + public void increment(String key) { + incrementBy(key, 1); } @Override - public void put(String key, V value) { - this.cache.put(key, value); + public long get(String key) { + return cache.getOrDefault(key, 0L); } - @Override - public long size() { - return this.cache.size(); - } - - @Override - public void destroy() { - this.cache.clear(); + public Map internalCache() { + return cache; } } @@ -81,7 +65,7 @@ private static HttpResponseParams generateResponseParamsForStatusCode(int status @Test public void testShouldNotify() throws InterruptedException { - MemCache cache = new MemCache<>(); + MemCache cache = new MemCache(); WindowBasedThresholdNotifier notifier = new WindowBasedThresholdNotifier( cache, new WindowBasedThresholdNotifier.Config(10, 1)); @@ -89,30 +73,19 @@ public void testShouldNotify() throws InterruptedException { String ip = "192.168.0.1"; for (int i = 0; i < 1000; i++) { - shouldNotify = shouldNotify - || notifier.shouldNotify( - ip, - WindowBasedThresholdNotifierTest - .generateResponseParamsForStatusCode(400)); + boolean _shouldNotify = notifier.shouldNotify( + ip, + WindowBasedThresholdNotifierTest + .generateResponseParamsForStatusCode(400)); + shouldNotify = shouldNotify || _shouldNotify; } - Data data = cache.get(ip).orElse(new Data()); - assertEquals(10, data.getRequests().size()); - - long lastNotifiedAt = data.getLastNotifiedAt(); - assertNotEquals(lastNotifiedAt, 0); - - assertTrue(shouldNotify); - - Thread.sleep(2000L); - - shouldNotify = notifier.shouldNotify( - ip, - WindowBasedThresholdNotifierTest.generateResponseParamsForStatusCode(401)); - assertFalse(shouldNotify); - - data = cache.get(ip).orElse(new Data()); + long count = 0; + for (Map.Entry entry : cache.internalCache().entrySet()) { + count += entry.getValue(); + } + assertEquals(1000, count); } } diff --git a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java index b81d3dc889..f31e35a496 100644 --- a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java +++ b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.errors.WakeupException; -import com.akto.DaoInit; import com.akto.RuntimeMode; import com.akto.dao.context.Context; import com.akto.data_actor.DataActor; From 44362c9c4db695c1946f53fb3171fb7d5275e7f0 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Thu, 14 Nov 2024 11:35:45 +0530 Subject: [PATCH 09/73] added group id to aggregate notifier --- .../akto/cache/RedisBackedCounterCache.java | 17 ++++++++------ .../java/com/akto/filters/HttpCallFilter.java | 4 +++- .../RealTimeThresholdNotifier.java | 22 ------------------- .../WindowBasedThresholdNotifier.java | 16 ++++++++------ .../WindowBasedThresholdNotifierTest.java | 1 + 5 files changed, 23 insertions(+), 37 deletions(-) delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/RealTimeThresholdNotifier.java diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java index fb316788f6..0b277d6519 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java @@ -32,23 +32,26 @@ public long getValue() { private final Cache localCache; - private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); private final ConcurrentLinkedQueue pendingOps; private final String prefix; public RedisBackedCounterCache(RedisClient redisClient, String prefix) { this.prefix = prefix; this.redis = redisClient.connect(new LongValueCodec()); - this.localCache = Caffeine.newBuilder() - .maximumSize(100000) - .expireAfterWrite(1, TimeUnit.HOURS) - .build(); - this.executor.scheduleAtFixedRate(this::syncToRedis, 60, 1, TimeUnit.SECONDS); + this.localCache = + Caffeine.newBuilder() + .maximumSize(100000) + .expireAfterWrite(1, TimeUnit.HOURS) + .build(); + + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + executor.scheduleAtFixedRate(this::syncToRedis, 60, 1, TimeUnit.SECONDS); + this.pendingOps = new ConcurrentLinkedQueue<>(); } private String getKey(String key) { - return new StringBuilder().append(prefix).append("|").append(key).toString(); + return prefix + "|" + key; } @Override diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 2679b5222f..d402b9ab93 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -105,7 +105,9 @@ public void filterFunction(List responseParams) { // we still push malicious requests to kafka SourceIPKeyGenerator.instance.generate(responseParam).ifPresent(aggKey -> { - boolean thresholdBreached = this.windowBasedThresholdNotifier.shouldNotify(aggKey, + boolean thresholdBreached = this.windowBasedThresholdNotifier.shouldNotify( + apiFilter.getId(), + aggKey, responseParam); // TODO: Add window id with each suspect sample data and alert diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/RealTimeThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/RealTimeThresholdNotifier.java deleted file mode 100644 index 07d0828248..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/RealTimeThresholdNotifier.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.akto.filters.aggregators; - -import com.akto.dto.HttpResponseParams; - -/* - * RealTimeThresholdNotifier is an abstract class that provides the basic structure for all the notifiers. - * It provides the basic structure for the notifiers to work with. - */ -public abstract class RealTimeThresholdNotifier { - - protected static final int NOTIFICATION_COOLDOWN_MINUTES = 60; - - /* - * Check if the aggregator should notify the system. - * Types of notifiers: - * Window based Threshold based notifier (for something like X bad requests in - * last Y minutes) - */ - public boolean shouldNotify(String aggKey, HttpResponseParams responseParam) { - throw new UnsupportedOperationException("Not implemented"); - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index bd0d4f43b1..f3690d8eb6 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -2,9 +2,8 @@ import com.akto.cache.CounterCache; import com.akto.dto.HttpResponseParams; -import com.akto.filters.aggregators.RealTimeThresholdNotifier; -public class WindowBasedThresholdNotifier extends RealTimeThresholdNotifier { +public class WindowBasedThresholdNotifier { private final Config config; @@ -33,18 +32,21 @@ public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.config = config; } - @Override - public boolean shouldNotify(String actor, HttpResponseParams responseParam) { + private static String getBucketKey(String actor, String groupKey, int minuteOfYear) { + return groupKey + "|" + actor + "|" + minuteOfYear; + } + + public boolean shouldNotify(String groupKey, String actor, HttpResponseParams responseParam) { int requestTimeSeconds = responseParam.getTime(); - int minuteOfYear = (int) Math.ceil(requestTimeSeconds / (60L)); + int minuteOfYear = requestTimeSeconds / 60; - String bucketKey = actor + "|" + minuteOfYear; + String bucketKey = getBucketKey(groupKey, actor, minuteOfYear); this.cache.increment(bucketKey); long windowCount = 0L; for (int i = minuteOfYear; i >= minuteOfYear - this.config.getWindowSizeInMinutes(); i--) { - windowCount += this.cache.get(actor + "|" + i); + windowCount += this.cache.get(getBucketKey(groupKey, actor, minuteOfYear)); } return windowCount >= this.config.getThreshold(); diff --git a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java index dc1b609af7..7c3c1ed391 100644 --- a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java +++ b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java @@ -75,6 +75,7 @@ public void testShouldNotify() throws InterruptedException { for (int i = 0; i < 1000; i++) { boolean _shouldNotify = notifier.shouldNotify( ip, + "4XX_FILTER", WindowBasedThresholdNotifierTest .generateResponseParamsForStatusCode(400)); shouldNotify = shouldNotify || _shouldNotify; From df11b6adda8e3095ab69ce18b09be57c32f39b7f Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Thu, 14 Nov 2024 11:58:30 +0530 Subject: [PATCH 10/73] added notification cooldown support --- .../java/com/akto/filters/HttpCallFilter.java | 4 +-- .../WindowBasedThresholdNotifier.java | 25 ++++++++++++++++++- 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index d402b9ab93..bc08d2439b 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -105,13 +105,13 @@ public void filterFunction(List responseParams) { // we still push malicious requests to kafka SourceIPKeyGenerator.instance.generate(responseParam).ifPresent(aggKey -> { - boolean thresholdBreached = this.windowBasedThresholdNotifier.shouldNotify( + boolean registerThreatAlert = this.windowBasedThresholdNotifier.shouldNotify( apiFilter.getId(), aggKey, responseParam); // TODO: Add window id with each suspect sample data and alert - if (thresholdBreached) { + if (registerThreatAlert) { DetectedThreatAlert alert = new DetectedThreatAlert( UUID.randomUUID().toString(), apiFilter.getId(), diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index f3690d8eb6..a62e4a6f93 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -1,5 +1,8 @@ package com.akto.filters.aggregators.window_based; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentHashMap; + import com.akto.cache.CounterCache; import com.akto.dto.HttpResponseParams; @@ -7,9 +10,16 @@ public class WindowBasedThresholdNotifier { private final Config config; + // We can use an in-memory cache for this, since we dont mind being notified + // more than once by multiple instances of the service. + // But on 1 instance, we should not notify more than once in the cooldown + // period. + private final ConcurrentMap notifiedMap; + public static class Config { private final int threshold; private final int windowSizeInMinutes; + private int notificationCooldownInSeconds = 60 * 30; // 30 mins public Config(int threshold, int windowInSeconds) { this.threshold = threshold; @@ -30,6 +40,7 @@ public int getWindowSizeInMinutes() { public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.cache = cache; this.config = config; + this.notifiedMap = new ConcurrentHashMap<>(); } private static String getBucketKey(String actor, String groupKey, int minuteOfYear) { @@ -49,6 +60,18 @@ public boolean shouldNotify(String groupKey, String actor, HttpResponseParams re windowCount += this.cache.get(getBucketKey(groupKey, actor, minuteOfYear)); } - return windowCount >= this.config.getThreshold(); + boolean thresholdBreached = windowCount >= this.config.getThreshold(); + + long now = System.currentTimeMillis() / 1000L; + long lastNotified = this.notifiedMap.getOrDefault(bucketKey, 0L); + + boolean cooldownBreached = (now - lastNotified) >= this.config.notificationCooldownInSeconds; + + if (thresholdBreached && cooldownBreached) { + this.notifiedMap.put(bucketKey, now); + return true; + } + + return false; } } From 8a344d3aa969d5a74ac489175a1b947ca6d49117 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Thu, 14 Nov 2024 12:14:25 +0530 Subject: [PATCH 11/73] added start and end bucket ids with each detected alerts --- .../java/com/akto/filters/HttpCallFilter.java | 11 ++++++-- .../WindowBasedThresholdNotifier.java | 4 +++ .../java/com/akto/suspect_data/Message.java | 12 ++++++++- .../threat_detection/DetectedThreatAlert.java | 25 ++++++++++++++++++- 4 files changed, 48 insertions(+), 4 deletions(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index bc08d2439b..4f9681626c 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -87,6 +87,8 @@ public void filterFunction(List responseParams) { List sourceIps = ApiAccessTypePolicy.getSourceIps(responseParam); Method method = Method.fromString(requestParams.getMethod()); + int currentBucket = (int) (responseParam.getTime() / 60); + maliciousSamples.add( new Message( responseParam.getAccountId(), @@ -97,7 +99,8 @@ public void filterFunction(List responseParams) { method, responseParam.getOrig(), Context.now(), - apiFilter.getId()))); + apiFilter.getId()), + currentBucket)); // Later we will also add aggregation support // Eg: 100 4xx requests in last 10 minutes. @@ -112,11 +115,15 @@ public void filterFunction(List responseParams) { // TODO: Add window id with each suspect sample data and alert if (registerThreatAlert) { + int windowStartBucket = currentBucket + - this.windowBasedThresholdNotifier.getConfig().getWindowSizeInMinutes(); DetectedThreatAlert alert = new DetectedThreatAlert( UUID.randomUUID().toString(), apiFilter.getId(), aggKey, - System.currentTimeMillis()); + System.currentTimeMillis(), + windowStartBucket, + currentBucket); DetectedThreatAlertDao.instance.insertOne(alert); } diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index a62e4a6f93..e8e862e765 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -35,6 +35,10 @@ public int getWindowSizeInMinutes() { } } + public Config getConfig() { + return config; + } + private final CounterCache cache; public WindowBasedThresholdNotifier(CounterCache cache, Config config) { diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java index 313f6c3968..bf0cfd994a 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java @@ -9,15 +9,17 @@ public class Message { private String accountId; private SuspectSampleData data; + private int bucketId; private static final ObjectMapper objectMapper = new ObjectMapper(); public Message() { } - public Message(String accountId, SuspectSampleData data) { + public Message(String accountId, SuspectSampleData data, int bucketId) { this.accountId = accountId; this.data = data; + this.bucketId = bucketId; } public String getAccountId() { @@ -36,6 +38,14 @@ public void setData(SuspectSampleData data) { this.data = data; } + public int getBucketId() { + return bucketId; + } + + public void setBucketId(int bucketId) { + this.bucketId = bucketId; + } + public static Optional marshall(Message m) { try { return Optional.of(objectMapper.writeValueAsString(m)); diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java index cfaaa872ce..bfca48592b 100644 --- a/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java +++ b/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java @@ -13,16 +13,23 @@ public class DetectedThreatAlert { private String actor; + private int bucketStart; + + private int bucketEnd; + private final ObjectMapper objectMapper = new ObjectMapper(); public DetectedThreatAlert() { } - public DetectedThreatAlert(String id, String filterId, String actor, long detectedAt) { + public DetectedThreatAlert(String id, String filterId, String actor, long detectedAt, int bucketStart, + int bucketEnd) { this.id = id; this.filterId = filterId; this.detectedAt = detectedAt; this.actor = actor; + this.bucketStart = bucketStart; + this.bucketEnd = bucketEnd; } public String getId() { @@ -57,6 +64,22 @@ public void setActor(String actor) { this.actor = actor; } + public int getBucketStart() { + return bucketStart; + } + + public void setBucketStart(int bucketStart) { + this.bucketStart = bucketStart; + } + + public int getBucketEnd() { + return bucketEnd; + } + + public void setBucketEnd(int bucketEnd) { + this.bucketEnd = bucketEnd; + } + public Optional marshall() { try { return Optional.of(this.objectMapper.writeValueAsString(this)); From 3862ca1d60b6c0a8fceea63c7ea6209c5c23c508 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Thu, 14 Nov 2024 14:29:30 +0530 Subject: [PATCH 12/73] added ttl for redis entries --- .../main/java/com/akto/cache/CounterCache.java | 2 ++ .../akto/cache/RedisBackedCounterCache.java | 18 +++++++++++++----- .../WindowBasedThresholdNotifier.java | 3 +++ 3 files changed, 18 insertions(+), 5 deletions(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java index 7786a05c25..26c7d395f4 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java @@ -7,4 +7,6 @@ public interface CounterCache { void increment(String key); long get(String key); + + void setExpiryIfNotSet(String key, long seconds); } diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java index 0b277d6519..b5091c4e12 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java @@ -1,5 +1,6 @@ package com.akto.cache; +import io.lettuce.core.ExpireArgs; import io.lettuce.core.RedisClient; import io.lettuce.core.api.StatefulRedisConnection; @@ -38,11 +39,10 @@ public long getValue() { public RedisBackedCounterCache(RedisClient redisClient, String prefix) { this.prefix = prefix; this.redis = redisClient.connect(new LongValueCodec()); - this.localCache = - Caffeine.newBuilder() - .maximumSize(100000) - .expireAfterWrite(1, TimeUnit.HOURS) - .build(); + this.localCache = Caffeine.newBuilder() + .maximumSize(100000) + .expireAfterWrite(1, TimeUnit.HOURS) + .build(); ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); executor.scheduleAtFixedRate(this::syncToRedis, 60, 1, TimeUnit.SECONDS); @@ -71,6 +71,14 @@ public long get(String key) { return Optional.ofNullable(this.localCache.getIfPresent(getKey(key))).orElse(0L); } + @Override + public void setExpiryIfNotSet(String key, long seconds) { + // We only set expiry for redis entry. For local cache we have lower expiry for + // all entries. + ExpireArgs args = ExpireArgs.Builder.nx(); + redis.async().expire(getKey(key), seconds, args); + } + private void syncToRedis() { while (!pendingOps.isEmpty()) { Op op = pendingOps.poll(); diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index e8e862e765..7589f123c2 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -59,6 +59,9 @@ public boolean shouldNotify(String groupKey, String actor, HttpResponseParams re String bucketKey = getBucketKey(groupKey, actor, minuteOfYear); this.cache.increment(bucketKey); + // Set expiry if not set (3 times window size) + this.cache.setExpiryIfNotSet(bucketKey, 3 * this.config.getWindowSizeInMinutes() * 60); + long windowCount = 0L; for (int i = minuteOfYear; i >= minuteOfYear - this.config.getWindowSizeInMinutes(); i--) { windowCount += this.cache.get(getBucketKey(groupKey, actor, minuteOfYear)); From d1f0f9a4869d06b460ffbc146b8d799f510ca5c0 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 15 Nov 2024 12:19:09 +0530 Subject: [PATCH 13/73] added new collections for sample malicious requests and detected alerts --- .../java/com/akto/cache/CounterCache.java | 2 +- .../akto/cache/RedisBackedCounterCache.java | 8 +- .../java/com/akto/filters/HttpCallFilter.java | 77 +++++++---------- .../WindowBasedThresholdNotifier.java | 71 +++++++++++----- .../akto/suspect_data/FlushMessagesTask.java | 59 +++++++------ .../java/com/akto/suspect_data/Message.java | 20 ++--- .../WindowBasedThresholdNotifierTest.java | 19 +++-- .../threat_detection/SampleRequestDao.java | 19 +++++ .../com/akto/dto/threat_detection/Bin.java | 19 +++++ .../threat_detection/DetectedThreatAlert.java | 31 +++---- .../dto/threat_detection/SampleRequest.java | 85 +++++++++++++++++++ 11 files changed, 276 insertions(+), 134 deletions(-) create mode 100644 libs/dao/src/main/java/com/akto/dao/threat_detection/SampleRequestDao.java create mode 100644 libs/dao/src/main/java/com/akto/dto/threat_detection/Bin.java create mode 100644 libs/dao/src/main/java/com/akto/dto/threat_detection/SampleRequest.java diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java index 26c7d395f4..cfc7266312 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java @@ -8,5 +8,5 @@ public interface CounterCache { long get(String key); - void setExpiryIfNotSet(String key, long seconds); + boolean exists(String key); } diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java index b5091c4e12..f481d256ac 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java @@ -64,6 +64,8 @@ public void incrementBy(String key, long val) { String _key = getKey(key); localCache.asMap().merge(_key, val, Long::sum); pendingOps.add(new Op(_key, val)); + + this.setExpiryIfNotSet(_key, 3 * 60 * 60); // added 3 hours expiry for now } @Override @@ -72,7 +74,11 @@ public long get(String key) { } @Override - public void setExpiryIfNotSet(String key, long seconds) { + public boolean exists(String key) { + return localCache.asMap().containsKey(getKey(key)); + } + + private void setExpiryIfNotSet(String key, long seconds) { // We only set expiry for redis entry. For local cache we have lower expiry for // all entries. ExpireArgs args = ExpireArgs.Builder.nx(); diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 4f9681626c..e140cf672e 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -9,13 +9,12 @@ import com.akto.data_actor.DataActor; import com.akto.data_actor.DataActorFactory; import com.akto.dto.ApiInfo.ApiInfoKey; -import com.akto.dto.HttpRequestParams; import com.akto.dto.HttpResponseParams; import com.akto.dto.RawApi; import com.akto.dto.monitoring.FilterConfig; import com.akto.dto.test_editor.YamlTemplate; import com.akto.dto.threat_detection.DetectedThreatAlert; -import com.akto.dto.traffic.SuspectSampleData; +import com.akto.dto.threat_detection.SampleRequest; import com.akto.dto.type.URLMethods.Method; import com.akto.filters.aggregators.key_generator.SourceIPKeyGenerator; import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier; @@ -24,7 +23,6 @@ import com.akto.log.LoggerMaker; import com.akto.log.LoggerMaker.LogDb; import com.akto.rules.TestPlugin; -import com.akto.runtime.policies.ApiAccessTypePolicy; import com.akto.suspect_data.Message; import com.akto.test_editor.execution.VariableResolver; import com.akto.test_editor.filter.data_operands_impl.ValidationResult; @@ -49,7 +47,8 @@ public class HttpCallFilter { private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; - public HttpCallFilter(RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { + public HttpCallFilter( + RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { this.apiFilters = new HashMap<>(); this.lastFilterFetch = 0; this.httpCallParser = new HttpCallParser(sync_threshold_count, sync_threshold_time); @@ -83,51 +82,41 @@ public void filterFunction(List responseParams) { // If a request passes any of the filter, then it's a malicious request, // and so we push it to kafka if (hasPassedFilter) { - HttpRequestParams requestParams = responseParam.getRequestParams(); - List sourceIps = ApiAccessTypePolicy.getSourceIps(responseParam); - Method method = Method.fromString(requestParams.getMethod()); - - int currentBucket = (int) (responseParam.getTime() / 60); - - maliciousSamples.add( - new Message( - responseParam.getAccountId(), - new SuspectSampleData( - sourceIps, - requestParams.getApiCollectionId(), - requestParams.getURL(), - method, - responseParam.getOrig(), - Context.now(), - apiFilter.getId()), - currentBucket)); - // Later we will also add aggregation support // Eg: 100 4xx requests in last 10 minutes. // But regardless of whether request falls in aggregation or not, // we still push malicious requests to kafka - SourceIPKeyGenerator.instance.generate(responseParam).ifPresent(aggKey -> { - boolean registerThreatAlert = this.windowBasedThresholdNotifier.shouldNotify( - apiFilter.getId(), - aggKey, - responseParam); - - // TODO: Add window id with each suspect sample data and alert - if (registerThreatAlert) { - int windowStartBucket = currentBucket - - this.windowBasedThresholdNotifier.getConfig().getWindowSizeInMinutes(); - DetectedThreatAlert alert = new DetectedThreatAlert( - UUID.randomUUID().toString(), - apiFilter.getId(), - aggKey, - System.currentTimeMillis(), - windowStartBucket, - currentBucket); - - DetectedThreatAlertDao.instance.insertOne(alert); - } - }); + SourceIPKeyGenerator.instance + .generate(responseParam) + .ifPresent( + actor -> { + String groupKey = apiFilter.getId(); + String aggKey = actor + "|" + groupKey; + SampleRequest sampleRequest = new SampleRequest( + apiFilter, + actor, + responseParam); + + maliciousSamples.add( + new Message( + responseParam.getAccountId(), + sampleRequest)); + + WindowBasedThresholdNotifier.Result result = this.windowBasedThresholdNotifier + .shouldNotify( + aggKey, sampleRequest); + + if (result.shouldNotify()) { + DetectedThreatAlert alert = new DetectedThreatAlert( + groupKey, + actor, + System.currentTimeMillis() / 1000L, + result.getBins()); + + DetectedThreatAlertDao.instance.insertOne(alert); + } + }); } } } diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index 7589f123c2..72414ae4bf 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -1,10 +1,14 @@ package com.akto.filters.aggregators.window_based; import java.util.concurrent.ConcurrentMap; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; import com.akto.cache.CounterCache; import com.akto.dto.HttpResponseParams; +import com.akto.dto.threat_detection.Bin; +import com.akto.dto.threat_detection.SampleRequest; public class WindowBasedThresholdNotifier { @@ -33,6 +37,28 @@ public int getThreshold() { public int getWindowSizeInMinutes() { return windowSizeInMinutes; } + + public int getNotificationCooldownInSeconds() { + return notificationCooldownInSeconds; + } + } + + public static class Result { + private final boolean shouldNotify; + private final List bins; + + public Result(boolean shouldNotify, List bins) { + this.shouldNotify = shouldNotify; + this.bins = bins; + } + + public boolean shouldNotify() { + return shouldNotify; + } + + public List getBins() { + return bins; + } } public Config getConfig() { @@ -47,38 +73,45 @@ public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.notifiedMap = new ConcurrentHashMap<>(); } - private static String getBucketKey(String actor, String groupKey, int minuteOfYear) { - return groupKey + "|" + actor + "|" + minuteOfYear; + public static int generateBinId(HttpResponseParams responseParams) { + return (int) (responseParams.getTime() / 60); } - public boolean shouldNotify(String groupKey, String actor, HttpResponseParams responseParam) { - int requestTimeSeconds = responseParam.getTime(); - - int minuteOfYear = requestTimeSeconds / 60; - - String bucketKey = getBucketKey(groupKey, actor, minuteOfYear); - this.cache.increment(bucketKey); - - // Set expiry if not set (3 times window size) - this.cache.setExpiryIfNotSet(bucketKey, 3 * this.config.getWindowSizeInMinutes() * 60); + public Result shouldNotify(String aggKey, SampleRequest sampleRequest) { + int binId = sampleRequest.getBinId(); + String cacheKey = aggKey + "|" + binId; + this.cache.increment(cacheKey); long windowCount = 0L; - for (int i = minuteOfYear; i >= minuteOfYear - this.config.getWindowSizeInMinutes(); i--) { - windowCount += this.cache.get(getBucketKey(groupKey, actor, minuteOfYear)); + List bins = getBins(aggKey, binId - this.config.getWindowSizeInMinutes() + 1, binId); + for (Bin data : bins) { + windowCount += data.getCount(); } boolean thresholdBreached = windowCount >= this.config.getThreshold(); long now = System.currentTimeMillis() / 1000L; - long lastNotified = this.notifiedMap.getOrDefault(bucketKey, 0L); + long lastNotified = this.notifiedMap.getOrDefault(aggKey, 0L); - boolean cooldownBreached = (now - lastNotified) >= this.config.notificationCooldownInSeconds; + boolean cooldownBreached = (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); if (thresholdBreached && cooldownBreached) { - this.notifiedMap.put(bucketKey, now); - return true; + this.notifiedMap.put(aggKey, now); + return new Result(true, bins); } - return false; + return new Result(false, bins); + } + + public List getBins(String aggKey, int binStart, int binEnd) { + List binData = new ArrayList<>(); + for (int i = binStart; i <= binEnd; i++) { + String key = aggKey + "|" + i; + if (!this.cache.exists(key)) { + continue; + } + binData.add(new Bin(i, this.cache.get(key))); + } + return binData; } } diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java index 9e9a2da938..84ee641c83 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java @@ -10,16 +10,16 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import com.akto.dao.threat_detection.SampleRequestDao; +import com.akto.dto.threat_detection.SampleRequest; +import com.mongodb.client.model.BulkWriteOptions; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; -import com.akto.dao.SuspectSampleDataDao; import com.akto.dao.context.Context; -import com.akto.dto.traffic.SuspectSampleData; import com.akto.runtime.utils.Utils; -import com.mongodb.client.model.BulkWriteOptions; import com.mongodb.client.model.InsertOneModel; import com.mongodb.client.model.WriteModel; @@ -40,42 +40,49 @@ private FlushMessagesTask() { public void init() { consumer.subscribe(Collections.singletonList("akto.malicious")); - pollingExecutor.execute(new Runnable() { - @Override - public void run() { - while (true) { - try { - ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); - processRecords(records); - } catch (Exception e) { - e.printStackTrace(); - consumer.close(); + pollingExecutor.execute( + new Runnable() { + @Override + public void run() { + while (true) { + try { + ConsumerRecords records = + consumer.poll(Duration.ofMillis(100)); + processRecords(records); + } catch (Exception e) { + e.printStackTrace(); + consumer.close(); + } + } } - } - } - }); + }); } public void processRecords(ConsumerRecords records) { - Map> accWiseMessages = new HashMap<>(); + Map> accWiseMessages = new HashMap<>(); for (ConsumerRecord record : records) { String msgStr = record.value(); - Message.unmarshall(msgStr).ifPresent(msg -> { - accWiseMessages.computeIfAbsent(msg.getAccountId(), k -> new ArrayList<>()).add(msg.getData()); - }); + Message.unmarshall(msgStr) + .ifPresent( + msg -> { + accWiseMessages + .computeIfAbsent(msg.getAccountId(), k -> new ArrayList<>()) + .add(msg.getData()); + }); } - for (Map.Entry> entry : accWiseMessages.entrySet()) { + for (Map.Entry> entry : accWiseMessages.entrySet()) { String accountId = entry.getKey(); - List sampleDatas = entry.getValue(); + List sampleDatas = entry.getValue(); Context.accountId.set(Integer.parseInt(accountId)); try { - List> bulkUpdates = new ArrayList<>(); - sampleDatas - .forEach(sampleData -> bulkUpdates.add(new InsertOneModel<>(sampleData))); + List> bulkUpdates = new ArrayList<>(); + sampleDatas.forEach( + sampleData -> bulkUpdates.add(new InsertOneModel<>(sampleData))); - SuspectSampleDataDao.instance.bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); + SampleRequestDao.instance.bulkWrite( + bulkUpdates, new BulkWriteOptions().ordered(false)); } catch (Exception e) { e.printStackTrace(); } diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java index bf0cfd994a..e28f894ae8 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java @@ -1,6 +1,6 @@ package com.akto.suspect_data; -import com.akto.dto.traffic.SuspectSampleData; +import com.akto.dto.threat_detection.SampleRequest; import com.fasterxml.jackson.databind.ObjectMapper; import java.util.Optional; @@ -8,18 +8,16 @@ // Kafka Message Wrapper for suspect data public class Message { private String accountId; - private SuspectSampleData data; - private int bucketId; + private SampleRequest data; private static final ObjectMapper objectMapper = new ObjectMapper(); public Message() { } - public Message(String accountId, SuspectSampleData data, int bucketId) { + public Message(String accountId, SampleRequest data) { this.accountId = accountId; this.data = data; - this.bucketId = bucketId; } public String getAccountId() { @@ -30,22 +28,14 @@ public void setAccountId(String accountId) { this.accountId = accountId; } - public SuspectSampleData getData() { + public SampleRequest getData() { return data; } - public void setData(SuspectSampleData data) { + public void setData(SampleRequest data) { this.data = data; } - public int getBucketId() { - return bucketId; - } - - public void setBucketId(int bucketId) { - this.bucketId = bucketId; - } - public static Optional marshall(Message m) { try { return Optional.of(objectMapper.writeValueAsString(m)); diff --git a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java index 7c3c1ed391..fda9329181 100644 --- a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java +++ b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java @@ -5,6 +5,10 @@ import java.util.Map; import com.akto.dto.HttpRequestParams; import com.akto.dto.HttpResponseParams; +import com.akto.dto.monitoring.FilterConfig; +import com.akto.dto.threat_detection.SampleRequest; +import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier.Result; + import java.util.concurrent.ConcurrentHashMap; import org.junit.Test; @@ -72,13 +76,15 @@ public void testShouldNotify() throws InterruptedException { boolean shouldNotify = false; String ip = "192.168.0.1"; + FilterConfig filterConfig = new FilterConfig(); + filterConfig.setId("4XX_FILTER"); + for (int i = 0; i < 1000; i++) { - boolean _shouldNotify = notifier.shouldNotify( - ip, - "4XX_FILTER", - WindowBasedThresholdNotifierTest - .generateResponseParamsForStatusCode(400)); - shouldNotify = shouldNotify || _shouldNotify; + Result res = notifier.shouldNotify( + ip + "|" + "4XX_FILTER", + new SampleRequest( + filterConfig, ip, generateResponseParamsForStatusCode(400))); + shouldNotify = shouldNotify || res.shouldNotify(); } long count = 0; @@ -88,5 +94,4 @@ public void testShouldNotify() throws InterruptedException { assertEquals(1000, count); } - } diff --git a/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleRequestDao.java b/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleRequestDao.java new file mode 100644 index 0000000000..0607c243c0 --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleRequestDao.java @@ -0,0 +1,19 @@ +package com.akto.dao.threat_detection; + +import com.akto.dao.AccountsContextDao; +import com.akto.dto.threat_detection.SampleRequest; + +public class SampleRequestDao extends AccountsContextDao { + + public static final SampleRequestDao instance = new SampleRequestDao(); + + @Override + public String getCollName() { + return "sample_malicious_requests"; + } + + @Override + public Class getClassT() { + return SampleRequest.class; + } +} diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/Bin.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/Bin.java new file mode 100644 index 0000000000..df181406f2 --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dto/threat_detection/Bin.java @@ -0,0 +1,19 @@ +package com.akto.dto.threat_detection; + +public class Bin { + int binId; + long count; + + public Bin(int binId, long count) { + this.binId = binId; + this.count = count; + } + + public int getBinId() { + return binId; + } + + public long getCount() { + return count; + } +} \ No newline at end of file diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java index bfca48592b..2a1bcbd9d3 100644 --- a/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java +++ b/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java @@ -1,6 +1,8 @@ package com.akto.dto.threat_detection; +import java.util.List; import java.util.Optional; +import java.util.UUID; import com.fasterxml.jackson.databind.ObjectMapper; @@ -13,23 +15,19 @@ public class DetectedThreatAlert { private String actor; - private int bucketStart; - - private int bucketEnd; + private List bins; private final ObjectMapper objectMapper = new ObjectMapper(); public DetectedThreatAlert() { } - public DetectedThreatAlert(String id, String filterId, String actor, long detectedAt, int bucketStart, - int bucketEnd) { - this.id = id; + public DetectedThreatAlert(String filterId, String actor, long detectedAt, List bins) { + this.id = UUID.randomUUID().toString(); this.filterId = filterId; this.detectedAt = detectedAt; this.actor = actor; - this.bucketStart = bucketStart; - this.bucketEnd = bucketEnd; + this.bins = bins; } public String getId() { @@ -64,20 +62,12 @@ public void setActor(String actor) { this.actor = actor; } - public int getBucketStart() { - return bucketStart; - } - - public void setBucketStart(int bucketStart) { - this.bucketStart = bucketStart; + public List getBins() { + return bins; } - public int getBucketEnd() { - return bucketEnd; - } - - public void setBucketEnd(int bucketEnd) { - this.bucketEnd = bucketEnd; + public void setBins(List bins) { + this.bins = bins; } public Optional marshall() { @@ -88,5 +78,4 @@ public Optional marshall() { return Optional.empty(); } } - } diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleRequest.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleRequest.java new file mode 100644 index 0000000000..3c186775fe --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleRequest.java @@ -0,0 +1,85 @@ +package com.akto.dto.threat_detection; + +import com.akto.dto.HttpResponseParams; +import com.akto.dto.monitoring.FilterConfig; + +import java.util.UUID; + +public class SampleRequest { + + private String id; + private String filterId; + private String actor; + private String data; + private int binId; + private int expiry; + + public SampleRequest() { + } + + public SampleRequest(FilterConfig filter, String actor, HttpResponseParams responseParams) { + int now = (int) (System.currentTimeMillis() / 1000L); + this.id = UUID.randomUUID().toString(); + this.filterId = filter.getId(); + this.actor = actor; + this.data = responseParams.getOrig(); + this.binId = generateBinId(responseParams); + + // For now we are hardcoding it to 1 hr. + // But later we will read it through FilterConfig + this.expiry = now + (1 * 60 * 60); + } + + public static int generateBinId(HttpResponseParams responseParam) { + return responseParam.getTime() / 60; + } + + public String getId() { + return id; + } + + public String getFilterId() { + return filterId; + } + + public String getActor() { + return actor; + } + + public String getData() { + return data; + } + + public int getBinId() { + return binId; + } + + public int getExpiry() { + return expiry; + } + + public void setId(String id) { + this.id = id; + } + + public void setFilterId(String filterId) { + this.filterId = filterId; + } + + public void setActor(String actor) { + this.actor = actor; + } + + public void setData(String data) { + this.data = data; + } + + public void setBinId(int binId) { + this.binId = binId; + } + + public void setExpiry(int expiry) { + this.expiry = expiry; + } + +} From f99611d609ab2c76f756ce9a1ffda5e937a939f3 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Sat, 16 Nov 2024 12:47:14 +0530 Subject: [PATCH 14/73] addded cleanup sample malicious records task --- .../java/com/akto/filters/HttpCallFilter.java | 51 ++++---- .../WindowBasedThresholdNotifier.java | 14 +-- .../com/akto/suspect_data/CleanUpTask.java | 109 ++++++++++++++++++ .../akto/suspect_data/FlushMessagesTask.java | 14 +-- .../java/com/akto/suspect_data/Message.java | 13 +-- .../java/com/akto/threat/detection/Main.java | 17 ++- .../WindowBasedThresholdNotifierTest.java | 21 ++-- .../dao/threat_detection/CleanupAuditDao.java | 29 +++++ .../SampleMaliciousRequestDao.java | 19 +++ .../threat_detection/SampleRequestDao.java | 19 --- .../dto/threat_detection/CleanupAudit.java | 42 +++++++ ...quest.java => SampleMaliciousRequest.java} | 19 +-- 12 files changed, 283 insertions(+), 84 deletions(-) create mode 100644 apps/api-threat-detection/src/main/java/com/akto/suspect_data/CleanUpTask.java create mode 100644 libs/dao/src/main/java/com/akto/dao/threat_detection/CleanupAuditDao.java create mode 100644 libs/dao/src/main/java/com/akto/dao/threat_detection/SampleMaliciousRequestDao.java delete mode 100644 libs/dao/src/main/java/com/akto/dao/threat_detection/SampleRequestDao.java create mode 100644 libs/dao/src/main/java/com/akto/dto/threat_detection/CleanupAudit.java rename libs/dao/src/main/java/com/akto/dto/threat_detection/{SampleRequest.java => SampleMaliciousRequest.java} (74%) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index e140cf672e..58c94bcdaf 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -14,7 +14,7 @@ import com.akto.dto.monitoring.FilterConfig; import com.akto.dto.test_editor.YamlTemplate; import com.akto.dto.threat_detection.DetectedThreatAlert; -import com.akto.dto.threat_detection.SampleRequest; +import com.akto.dto.threat_detection.SampleMaliciousRequest; import com.akto.dto.type.URLMethods.Method; import com.akto.filters.aggregators.key_generator.SourceIPKeyGenerator; import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier; @@ -30,7 +30,8 @@ import io.lettuce.core.RedisClient; public class HttpCallFilter { - private static final LoggerMaker loggerMaker = new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); + private static final LoggerMaker loggerMaker = + new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); private Map apiFilters; private final HttpCallParser httpCallParser; @@ -55,9 +56,10 @@ public HttpCallFilter( String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); this.kafka = new Kafka(kafkaBrokerUrl, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); - this.windowBasedThresholdNotifier = new WindowBasedThresholdNotifier( - new RedisBackedCounterCache(redisClient, "wbt"), - new WindowBasedThresholdNotifier.Config(100, 10 * 60)); + this.windowBasedThresholdNotifier = + new WindowBasedThresholdNotifier( + new RedisBackedCounterCache(redisClient, "wbt"), + new WindowBasedThresholdNotifier.Config(100, 10 * 60)); } public void filterFunction(List responseParams) { @@ -93,26 +95,26 @@ public void filterFunction(List responseParams) { actor -> { String groupKey = apiFilter.getId(); String aggKey = actor + "|" + groupKey; - SampleRequest sampleRequest = new SampleRequest( - apiFilter, - actor, - responseParam); + SampleMaliciousRequest sampleMaliciousRequest = + new SampleMaliciousRequest( + apiFilter, actor, responseParam); maliciousSamples.add( new Message( responseParam.getAccountId(), - sampleRequest)); + sampleMaliciousRequest)); - WindowBasedThresholdNotifier.Result result = this.windowBasedThresholdNotifier - .shouldNotify( - aggKey, sampleRequest); + WindowBasedThresholdNotifier.Result result = + this.windowBasedThresholdNotifier.shouldNotify( + aggKey, sampleMaliciousRequest); if (result.shouldNotify()) { - DetectedThreatAlert alert = new DetectedThreatAlert( - groupKey, - actor, - System.currentTimeMillis() / 1000L, - result.getBins()); + DetectedThreatAlert alert = + new DetectedThreatAlert( + groupKey, + actor, + System.currentTimeMillis() / 1000L, + result.getBins()); DetectedThreatAlertDao.instance.insertOne(alert); } @@ -157,12 +159,13 @@ private boolean validateFilterForRequest( }, apiInfoKey); String filterExecutionLogId = UUID.randomUUID().toString(); - ValidationResult res = TestPlugin.validateFilter( - apiFilter.getFilter().getNode(), - rawApi, - apiInfoKey, - varMap, - filterExecutionLogId); + ValidationResult res = + TestPlugin.validateFilter( + apiFilter.getFilter().getNode(), + rawApi, + apiInfoKey, + varMap, + filterExecutionLogId); return res.getIsValid(); } catch (Exception e) { diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index 72414ae4bf..a16d1e5ec9 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -6,9 +6,8 @@ import java.util.concurrent.ConcurrentHashMap; import com.akto.cache.CounterCache; -import com.akto.dto.HttpResponseParams; import com.akto.dto.threat_detection.Bin; -import com.akto.dto.threat_detection.SampleRequest; +import com.akto.dto.threat_detection.SampleMaliciousRequest; public class WindowBasedThresholdNotifier { @@ -73,12 +72,8 @@ public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.notifiedMap = new ConcurrentHashMap<>(); } - public static int generateBinId(HttpResponseParams responseParams) { - return (int) (responseParams.getTime() / 60); - } - - public Result shouldNotify(String aggKey, SampleRequest sampleRequest) { - int binId = sampleRequest.getBinId(); + public Result shouldNotify(String aggKey, SampleMaliciousRequest sampleMaliciousRequest) { + int binId = sampleMaliciousRequest.getBinId(); String cacheKey = aggKey + "|" + binId; this.cache.increment(cacheKey); @@ -93,7 +88,8 @@ public Result shouldNotify(String aggKey, SampleRequest sampleRequest) { long now = System.currentTimeMillis() / 1000L; long lastNotified = this.notifiedMap.getOrDefault(aggKey, 0L); - boolean cooldownBreached = (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); + boolean cooldownBreached = + (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); if (thresholdBreached && cooldownBreached) { this.notifiedMap.put(aggKey, now); diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/CleanUpTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/CleanUpTask.java new file mode 100644 index 0000000000..dd40bc0f20 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/CleanUpTask.java @@ -0,0 +1,109 @@ +package com.akto.suspect_data; + +import java.util.*; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +import com.akto.dao.threat_detection.CleanupAuditDao; +import com.akto.dao.threat_detection.DetectedThreatAlertDao; +import com.akto.dao.threat_detection.SampleMaliciousRequestDao; +import com.akto.dto.Account; +import com.akto.dto.threat_detection.CleanupAudit; +import com.akto.util.AccountTask; +import com.mongodb.BasicDBList; +import com.mongodb.BasicDBObject; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.model.Accumulators; +import com.mongodb.client.model.Aggregates; +import com.mongodb.client.model.Filters; +import com.mongodb.client.model.Projections; +import org.bson.conversions.Bson; + +public class CleanUpTask { + private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(2); + + private CleanUpTask() {} + + public static CleanUpTask instance = new CleanUpTask(); + + public void init() { + this.executor.scheduleAtFixedRate(this::cleanUp, 0, 5, TimeUnit.HOURS); + } + + public void cleanUp() { + AccountTask.instance.executeTask( + new Consumer() { + @Override + public void accept(Account account) { + cleanUpForAccount(); + } + }, + "cleanup-malicious-requests"); + } + + public void cleanUpForAccount() { + // Remove all the requests that have passed their expiry. + // AND those requests whose actor and filter that don't have any alerts associated with + // them. + long now = System.currentTimeMillis() / 1000L; + + // Get the latest cleanup audit if exists + Optional audit = CleanupAuditDao.instance.getLatestEntry(); + long start = audit.map(CleanupAudit::getAlertWindowEnd).orElse(0L); + + List pipeline = + Arrays.asList( + Aggregates.match( + Filters.and( + Filters.gte("detectedAt", start), + Filters.lt("detectedAt", now))), + Aggregates.group( + 0, + Accumulators.addToSet( + "validFilters", + new BasicDBObject("filterId", "$filterId") + .append("actor", "$actor"))), + Aggregates.project( + Projections.fields( + Projections.include("validFilters"), + Projections.excludeId()))); + + try (MongoCursor result = + DetectedThreatAlertDao.instance + .getMCollection() + .aggregate(pipeline, BasicDBObject.class) + .cursor()) { + + BasicDBObject validFilters = result.tryNext(); + if (validFilters == null) { + return; + } + + BasicDBList filters = (BasicDBList) validFilters.get("validFilters"); + List filterList = new ArrayList<>(); + for (Object filter : filters) { + BasicDBObject filterObj = (BasicDBObject) filter; + filterList.add( + new BasicDBObject("filterId", filterObj.getString("filterId")) + .append("actor", filterObj.getString("actor"))); + } + + // Remove all the requests that have passed their expiry. + SampleMaliciousRequestDao.instance + .getMCollection() + .deleteMany( + Filters.and( + Filters.lt("expiry", now), + Filters.nor( + filterList.stream() + .map(Filters::and) + .toArray(Bson[]::new)))); + + // TODO: For any given filter, only keep last 1000 requests + + CleanupAuditDao.instance.insertOne(new CleanupAudit(start, now)); + } + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java index 84ee641c83..53d021157a 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java @@ -10,8 +10,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import com.akto.dao.threat_detection.SampleRequestDao; -import com.akto.dto.threat_detection.SampleRequest; +import com.akto.dao.threat_detection.SampleMaliciousRequestDao; +import com.akto.dto.threat_detection.SampleMaliciousRequest; import com.mongodb.client.model.BulkWriteOptions; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -59,7 +59,7 @@ public void run() { } public void processRecords(ConsumerRecords records) { - Map> accWiseMessages = new HashMap<>(); + Map> accWiseMessages = new HashMap<>(); for (ConsumerRecord record : records) { String msgStr = record.value(); Message.unmarshall(msgStr) @@ -71,17 +71,17 @@ public void processRecords(ConsumerRecords records) { }); } - for (Map.Entry> entry : accWiseMessages.entrySet()) { + for (Map.Entry> entry : accWiseMessages.entrySet()) { String accountId = entry.getKey(); - List sampleDatas = entry.getValue(); + List sampleDatas = entry.getValue(); Context.accountId.set(Integer.parseInt(accountId)); try { - List> bulkUpdates = new ArrayList<>(); + List> bulkUpdates = new ArrayList<>(); sampleDatas.forEach( sampleData -> bulkUpdates.add(new InsertOneModel<>(sampleData))); - SampleRequestDao.instance.bulkWrite( + SampleMaliciousRequestDao.instance.bulkWrite( bulkUpdates, new BulkWriteOptions().ordered(false)); } catch (Exception e) { e.printStackTrace(); diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java index e28f894ae8..508d4a35e5 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java @@ -1,6 +1,6 @@ package com.akto.suspect_data; -import com.akto.dto.threat_detection.SampleRequest; +import com.akto.dto.threat_detection.SampleMaliciousRequest; import com.fasterxml.jackson.databind.ObjectMapper; import java.util.Optional; @@ -8,14 +8,13 @@ // Kafka Message Wrapper for suspect data public class Message { private String accountId; - private SampleRequest data; + private SampleMaliciousRequest data; private static final ObjectMapper objectMapper = new ObjectMapper(); - public Message() { - } + public Message() {} - public Message(String accountId, SampleRequest data) { + public Message(String accountId, SampleMaliciousRequest data) { this.accountId = accountId; this.data = data; } @@ -28,11 +27,11 @@ public void setAccountId(String accountId) { this.accountId = accountId; } - public SampleRequest getData() { + public SampleMaliciousRequest getData() { return data; } - public void setData(SampleRequest data) { + public void setData(SampleMaliciousRequest data) { this.data = data; } diff --git a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java index 4cf060a657..016428da3e 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -1,6 +1,8 @@ package com.akto.threat.detection; import java.util.*; + +import com.akto.suspect_data.CleanUpTask; import com.akto.suspect_data.FlushMessagesTask; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -44,6 +46,9 @@ public static void main(String[] args) { // Flush Messages task FlushMessagesTask.instance.init(); + // Clean up sample requests for which no alert is generated + CleanUpTask.instance.init(); + String topicName = System.getenv("AKTO_KAFKA_TOPIC_NAME"); if (topicName == null) { String defaultTopic = "akto.api.protection"; @@ -102,8 +107,8 @@ public static void processRecords(ConsumerRecords records) { Context.accountId.set(accountIdInt); if (!httpCallFilterMap.containsKey(accountId)) { - HttpCallFilter filter = new HttpCallFilter(redisClient, sync_threshold_count, - sync_threshold_time); + HttpCallFilter filter = + new HttpCallFilter(redisClient, sync_threshold_count, sync_threshold_time); httpCallFilterMap.put(accountId, filter); loggerMaker.infoAndAddToDb("New filter created for account: " + accountId); } @@ -118,8 +123,12 @@ public static void processRecords(ConsumerRecords records) { public static RedisClient createRedisClient() { String host = System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_HOST", "localhost"); - int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_PORT", "6379")); - int database = Integer.parseInt(System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_DB", "0")); + int port = + Integer.parseInt( + System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_PORT", "6379")); + int database = + Integer.parseInt( + System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_DB", "0")); return RedisClient.create("redis://" + host + ":" + port + "/" + database); } diff --git a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java index fda9329181..adc4e529bb 100644 --- a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java +++ b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java @@ -6,7 +6,7 @@ import com.akto.dto.HttpRequestParams; import com.akto.dto.HttpResponseParams; import com.akto.dto.monitoring.FilterConfig; -import com.akto.dto.threat_detection.SampleRequest; +import com.akto.dto.threat_detection.SampleMaliciousRequest; import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier.Result; import java.util.concurrent.ConcurrentHashMap; @@ -38,6 +38,11 @@ public long get(String key) { return cache.getOrDefault(key, 0L); } + @Override + public boolean exists(String key) { + return cache.containsKey(key); + } + public Map internalCache() { return cache; } @@ -70,8 +75,9 @@ private static HttpResponseParams generateResponseParamsForStatusCode(int status public void testShouldNotify() throws InterruptedException { MemCache cache = new MemCache(); - WindowBasedThresholdNotifier notifier = new WindowBasedThresholdNotifier( - cache, new WindowBasedThresholdNotifier.Config(10, 1)); + WindowBasedThresholdNotifier notifier = + new WindowBasedThresholdNotifier( + cache, new WindowBasedThresholdNotifier.Config(10, 1)); boolean shouldNotify = false; String ip = "192.168.0.1"; @@ -80,10 +86,11 @@ public void testShouldNotify() throws InterruptedException { filterConfig.setId("4XX_FILTER"); for (int i = 0; i < 1000; i++) { - Result res = notifier.shouldNotify( - ip + "|" + "4XX_FILTER", - new SampleRequest( - filterConfig, ip, generateResponseParamsForStatusCode(400))); + Result res = + notifier.shouldNotify( + ip + "|" + "4XX_FILTER", + new SampleMaliciousRequest( + filterConfig, ip, generateResponseParamsForStatusCode(400))); shouldNotify = shouldNotify || res.shouldNotify(); } diff --git a/libs/dao/src/main/java/com/akto/dao/threat_detection/CleanupAuditDao.java b/libs/dao/src/main/java/com/akto/dao/threat_detection/CleanupAuditDao.java new file mode 100644 index 0000000000..90e8444480 --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dao/threat_detection/CleanupAuditDao.java @@ -0,0 +1,29 @@ +package com.akto.dao.threat_detection; + +import com.akto.dao.AccountsContextDao; +import com.akto.dto.threat_detection.CleanupAudit; +import com.mongodb.BasicDBObject; + +import java.util.Optional; + +public class CleanupAuditDao extends AccountsContextDao { + + public static final CleanupAuditDao instance = new CleanupAuditDao(); + + @Override + public String getCollName() { + return "cleanup_malicious_requests_audit"; + } + + @Override + public Class getClassT() { + return CleanupAudit.class; + } + + public Optional getLatestEntry() { + return Optional.ofNullable( + findOne( + new BasicDBObject(), + new BasicDBObject("sort", new BasicDBObject("alertWindowEnd", -1)))); + } +} diff --git a/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleMaliciousRequestDao.java b/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleMaliciousRequestDao.java new file mode 100644 index 0000000000..23fa950efa --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleMaliciousRequestDao.java @@ -0,0 +1,19 @@ +package com.akto.dao.threat_detection; + +import com.akto.dao.AccountsContextDao; +import com.akto.dto.threat_detection.SampleMaliciousRequest; + +public class SampleMaliciousRequestDao extends AccountsContextDao { + + public static final SampleMaliciousRequestDao instance = new SampleMaliciousRequestDao(); + + @Override + public String getCollName() { + return "sample_malicious_requests"; + } + + @Override + public Class getClassT() { + return SampleMaliciousRequest.class; + } +} diff --git a/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleRequestDao.java b/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleRequestDao.java deleted file mode 100644 index 0607c243c0..0000000000 --- a/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleRequestDao.java +++ /dev/null @@ -1,19 +0,0 @@ -package com.akto.dao.threat_detection; - -import com.akto.dao.AccountsContextDao; -import com.akto.dto.threat_detection.SampleRequest; - -public class SampleRequestDao extends AccountsContextDao { - - public static final SampleRequestDao instance = new SampleRequestDao(); - - @Override - public String getCollName() { - return "sample_malicious_requests"; - } - - @Override - public Class getClassT() { - return SampleRequest.class; - } -} diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/CleanupAudit.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/CleanupAudit.java new file mode 100644 index 0000000000..a705ae6eeb --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dto/threat_detection/CleanupAudit.java @@ -0,0 +1,42 @@ +package com.akto.dto.threat_detection; + +import org.bson.types.ObjectId; + +public class CleanupAudit { + + private String id; + private long alertWindowStart; + private long alertWindowEnd; + + public CleanupAudit() {} + + public CleanupAudit(long alertWindowStart, long alertWindowEnd) { + this.id = new ObjectId().toString(); + this.alertWindowStart = alertWindowStart; + this.alertWindowEnd = alertWindowEnd; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public long getAlertWindowStart() { + return alertWindowStart; + } + + public void setAlertWindowStart(long alertWindowStart) { + this.alertWindowStart = alertWindowStart; + } + + public long getAlertWindowEnd() { + return alertWindowEnd; + } + + public void setAlertWindowEnd(long alertWindowEnd) { + this.alertWindowEnd = alertWindowEnd; + } +} diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleRequest.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java similarity index 74% rename from libs/dao/src/main/java/com/akto/dto/threat_detection/SampleRequest.java rename to libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java index 3c186775fe..a0c502b6f0 100644 --- a/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleRequest.java +++ b/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java @@ -3,31 +3,37 @@ import com.akto.dto.HttpResponseParams; import com.akto.dto.monitoring.FilterConfig; +import com.akto.dto.type.URLMethods.Method; + import java.util.UUID; -public class SampleRequest { +public class SampleMaliciousRequest { private String id; private String filterId; private String actor; + private String url; + private Method method; private String data; private int binId; private int expiry; - public SampleRequest() { - } + public SampleMaliciousRequest() {} - public SampleRequest(FilterConfig filter, String actor, HttpResponseParams responseParams) { + public SampleMaliciousRequest( + FilterConfig filter, String actor, HttpResponseParams responseParams) { int now = (int) (System.currentTimeMillis() / 1000L); this.id = UUID.randomUUID().toString(); this.filterId = filter.getId(); this.actor = actor; this.data = responseParams.getOrig(); this.binId = generateBinId(responseParams); + this.url = responseParams.getRequestParams().getURL(); + this.method = Method.fromString(responseParams.getRequestParams().getMethod()); - // For now we are hardcoding it to 1 hr. + // For now, we are hardcoding it to 1 hr. // But later we will read it through FilterConfig - this.expiry = now + (1 * 60 * 60); + this.expiry = now + (60 * 60); } public static int generateBinId(HttpResponseParams responseParam) { @@ -81,5 +87,4 @@ public void setBinId(int binId) { public void setExpiry(int expiry) { this.expiry = expiry; } - } From a2031736c9062d0202ff6ac01db38abc9bd01738 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Mon, 18 Nov 2024 13:35:22 +0530 Subject: [PATCH 15/73] refactor code --- .../malicious_event/v1/MaliciousEvent.java | 1534 +++++++++++++++++ .../v1/MaliciousEventOrBuilder.java | 106 ++ .../malicious_event/v1/MessageProto.java | 81 + .../sample_request/v1/MessageProto.java | 73 + .../v1/SampleMaliciousRequest.java | 1178 +++++++++++++ .../v1/SampleMaliciousRequestOrBuilder.java | 83 + .../v1/DashboardServiceGrpc.java | 367 ++++ .../v1/FetchAlertFiltersRequest.java | 358 ++++ .../v1/FetchAlertFiltersRequestOrBuilder.java | 11 + .../v1/FetchAlertFiltersResponse.java | 743 ++++++++ .../FetchAlertFiltersResponseOrBuilder.java | 61 + .../v1/ListMaliciousRequestsRequest.java | 538 ++++++ ...ListMaliciousRequestsRequestOrBuilder.java | 32 + .../v1/ListMaliciousRequestsResponse.java | 851 +++++++++ ...istMaliciousRequestsResponseOrBuilder.java | 47 + .../v1/MaliciousRequest.java | 1520 ++++++++++++++++ .../v1/MaliciousRequestOrBuilder.java | 113 ++ .../dashboard_service/v1/ServiceProto.java | 136 ++ .../v1/MaliciousEventServiceGrpc.java | 293 ++++ .../v1/RecordMaliciousEventRequest.java | 913 ++++++++++ .../RecordMaliciousEventRequestOrBuilder.java | 50 + .../v1/RecordMaliciousEventResponse.java | 358 ++++ ...RecordMaliciousEventResponseOrBuilder.java | 11 + .../v1/ServiceProto.java | 99 ++ 24 files changed, 9556 insertions(+) create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/MessageProto.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/DashboardServiceGrpc.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponse.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponseOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponse.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponseOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ServiceProto.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousEventServiceGrpc.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponse.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponseOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ServiceProto.java diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java new file mode 100644 index 0000000000..362daeb9d8 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java @@ -0,0 +1,1534 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/malicious_event/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.malicious_event.v1; + +/** + * Protobuf type {@code threat_protection.message.malicious_event.v1.MaliciousEvent} + */ +public final class MaliciousEvent extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.message.malicious_event.v1.MaliciousEvent) + MaliciousEventOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + MaliciousEvent.class.getName()); + } + // Use MaliciousEvent.newBuilder() to construct. + private MaliciousEvent(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private MaliciousEvent() { + actor_ = ""; + filterId_ = ""; + latestApiIp_ = ""; + latestApiEndpoint_ = ""; + latestApiMethod_ = ""; + latestApiPayload_ = ""; + eventType_ = 0; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.class, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder.class); + } + + /** + * Protobuf enum {@code threat_protection.message.malicious_event.v1.MaliciousEvent.EventType} + */ + public enum EventType + implements com.google.protobuf.ProtocolMessageEnum { + /** + * EVENT_TYPE_UNSPECIFIED = 0; + */ + EVENT_TYPE_UNSPECIFIED(0), + /** + * EVENT_TYPE_SINGLE = 1; + */ + EVENT_TYPE_SINGLE(1), + /** + * EVENT_TYPE_AGGREGATED = 2; + */ + EVENT_TYPE_AGGREGATED(2), + UNRECOGNIZED(-1), + ; + + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + EventType.class.getName()); + } + /** + * EVENT_TYPE_UNSPECIFIED = 0; + */ + public static final int EVENT_TYPE_UNSPECIFIED_VALUE = 0; + /** + * EVENT_TYPE_SINGLE = 1; + */ + public static final int EVENT_TYPE_SINGLE_VALUE = 1; + /** + * EVENT_TYPE_AGGREGATED = 2; + */ + public static final int EVENT_TYPE_AGGREGATED_VALUE = 2; + + + public final int getNumber() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalArgumentException( + "Can't get the number of an unknown enum value."); + } + return value; + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static EventType valueOf(int value) { + return forNumber(value); + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + */ + public static EventType forNumber(int value) { + switch (value) { + case 0: return EVENT_TYPE_UNSPECIFIED; + case 1: return EVENT_TYPE_SINGLE; + case 2: return EVENT_TYPE_AGGREGATED; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static final com.google.protobuf.Internal.EnumLiteMap< + EventType> internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public EventType findValueByNumber(int number) { + return EventType.forNumber(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalStateException( + "Can't get the descriptor of an unrecognized enum value."); + } + return getDescriptor().getValues().get(ordinal()); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDescriptor().getEnumTypes().get(0); + } + + private static final EventType[] VALUES = values(); + + public static EventType valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + if (desc.getIndex() == -1) { + return UNRECOGNIZED; + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private EventType(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:threat_protection.message.malicious_event.v1.MaliciousEvent.EventType) + } + + public static final int ACTOR_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + @java.lang.Override + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DETECTED_AT_FIELD_NUMBER = 3; + private long detectedAt_ = 0L; + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + + public static final int LATEST_API_IP_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object latestApiIp_ = ""; + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @return The latestApiIp. + */ + @java.lang.Override + public java.lang.String getLatestApiIp() { + java.lang.Object ref = latestApiIp_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + latestApiIp_ = s; + return s; + } + } + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @return The bytes for latestApiIp. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getLatestApiIpBytes() { + java.lang.Object ref = latestApiIp_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + latestApiIp_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int LATEST_API_ENDPOINT_FIELD_NUMBER = 5; + @SuppressWarnings("serial") + private volatile java.lang.Object latestApiEndpoint_ = ""; + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @return The latestApiEndpoint. + */ + @java.lang.Override + public java.lang.String getLatestApiEndpoint() { + java.lang.Object ref = latestApiEndpoint_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + latestApiEndpoint_ = s; + return s; + } + } + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @return The bytes for latestApiEndpoint. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getLatestApiEndpointBytes() { + java.lang.Object ref = latestApiEndpoint_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + latestApiEndpoint_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int LATEST_API_METHOD_FIELD_NUMBER = 6; + @SuppressWarnings("serial") + private volatile java.lang.Object latestApiMethod_ = ""; + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @return The latestApiMethod. + */ + @java.lang.Override + public java.lang.String getLatestApiMethod() { + java.lang.Object ref = latestApiMethod_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + latestApiMethod_ = s; + return s; + } + } + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @return The bytes for latestApiMethod. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getLatestApiMethodBytes() { + java.lang.Object ref = latestApiMethod_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + latestApiMethod_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int LATEST_API_COLLECTION_ID_FIELD_NUMBER = 7; + private int latestApiCollectionId_ = 0; + /** + * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; + * @return The latestApiCollectionId. + */ + @java.lang.Override + public int getLatestApiCollectionId() { + return latestApiCollectionId_; + } + + public static final int LATEST_API_PAYLOAD_FIELD_NUMBER = 8; + @SuppressWarnings("serial") + private volatile java.lang.Object latestApiPayload_ = ""; + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @return The latestApiPayload. + */ + @java.lang.Override + public java.lang.String getLatestApiPayload() { + java.lang.Object ref = latestApiPayload_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + latestApiPayload_ = s; + return s; + } + } + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @return The bytes for latestApiPayload. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getLatestApiPayloadBytes() { + java.lang.Object ref = latestApiPayload_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + latestApiPayload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int EVENT_TYPE_FIELD_NUMBER = 9; + private int eventType_ = 0; + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @return The enum numeric value on the wire for eventType. + */ + @java.lang.Override public int getEventTypeValue() { + return eventType_; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @return The eventType. + */ + @java.lang.Override public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType getEventType() { + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType result = com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.forNumber(eventType_); + return result == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.UNRECOGNIZED : result; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); + } + if (detectedAt_ != 0L) { + output.writeInt64(3, detectedAt_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiIp_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 4, latestApiIp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiEndpoint_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 5, latestApiEndpoint_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiMethod_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 6, latestApiMethod_); + } + if (latestApiCollectionId_ != 0) { + output.writeInt32(7, latestApiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiPayload_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 8, latestApiPayload_); + } + if (eventType_ != com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.EVENT_TYPE_UNSPECIFIED.getNumber()) { + output.writeEnum(9, eventType_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); + } + if (detectedAt_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(3, detectedAt_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiIp_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(4, latestApiIp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiEndpoint_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(5, latestApiEndpoint_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiMethod_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(6, latestApiMethod_); + } + if (latestApiCollectionId_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(7, latestApiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiPayload_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(8, latestApiPayload_); + } + if (eventType_ != com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.EVENT_TYPE_UNSPECIFIED.getNumber()) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(9, eventType_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent other = (com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent) obj; + + if (!getActor() + .equals(other.getActor())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (getDetectedAt() + != other.getDetectedAt()) return false; + if (!getLatestApiIp() + .equals(other.getLatestApiIp())) return false; + if (!getLatestApiEndpoint() + .equals(other.getLatestApiEndpoint())) return false; + if (!getLatestApiMethod() + .equals(other.getLatestApiMethod())) return false; + if (getLatestApiCollectionId() + != other.getLatestApiCollectionId()) return false; + if (!getLatestApiPayload() + .equals(other.getLatestApiPayload())) return false; + if (eventType_ != other.eventType_) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACTOR_FIELD_NUMBER; + hash = (53 * hash) + getActor().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getDetectedAt()); + hash = (37 * hash) + LATEST_API_IP_FIELD_NUMBER; + hash = (53 * hash) + getLatestApiIp().hashCode(); + hash = (37 * hash) + LATEST_API_ENDPOINT_FIELD_NUMBER; + hash = (53 * hash) + getLatestApiEndpoint().hashCode(); + hash = (37 * hash) + LATEST_API_METHOD_FIELD_NUMBER; + hash = (53 * hash) + getLatestApiMethod().hashCode(); + hash = (37 * hash) + LATEST_API_COLLECTION_ID_FIELD_NUMBER; + hash = (53 * hash) + getLatestApiCollectionId(); + hash = (37 * hash) + LATEST_API_PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getLatestApiPayload().hashCode(); + hash = (37 * hash) + EVENT_TYPE_FIELD_NUMBER; + hash = (53 * hash) + eventType_; + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.message.malicious_event.v1.MaliciousEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.message.malicious_event.v1.MaliciousEvent) + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.class, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + actor_ = ""; + filterId_ = ""; + detectedAt_ = 0L; + latestApiIp_ = ""; + latestApiEndpoint_ = ""; + latestApiMethod_ = ""; + latestApiCollectionId_ = 0; + latestApiPayload_ = ""; + eventType_ = 0; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getDefaultInstanceForType() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent build() { + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent buildPartial() { + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result = new com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.actor_ = actor_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.filterId_ = filterId_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.detectedAt_ = detectedAt_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.latestApiIp_ = latestApiIp_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.latestApiEndpoint_ = latestApiEndpoint_; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.latestApiMethod_ = latestApiMethod_; + } + if (((from_bitField0_ & 0x00000040) != 0)) { + result.latestApiCollectionId_ = latestApiCollectionId_; + } + if (((from_bitField0_ & 0x00000080) != 0)) { + result.latestApiPayload_ = latestApiPayload_; + } + if (((from_bitField0_ & 0x00000100) != 0)) { + result.eventType_ = eventType_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent) { + return mergeFrom((com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent other) { + if (other == com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance()) return this; + if (!other.getActor().isEmpty()) { + actor_ = other.actor_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (other.getDetectedAt() != 0L) { + setDetectedAt(other.getDetectedAt()); + } + if (!other.getLatestApiIp().isEmpty()) { + latestApiIp_ = other.latestApiIp_; + bitField0_ |= 0x00000008; + onChanged(); + } + if (!other.getLatestApiEndpoint().isEmpty()) { + latestApiEndpoint_ = other.latestApiEndpoint_; + bitField0_ |= 0x00000010; + onChanged(); + } + if (!other.getLatestApiMethod().isEmpty()) { + latestApiMethod_ = other.latestApiMethod_; + bitField0_ |= 0x00000020; + onChanged(); + } + if (other.getLatestApiCollectionId() != 0) { + setLatestApiCollectionId(other.getLatestApiCollectionId()); + } + if (!other.getLatestApiPayload().isEmpty()) { + latestApiPayload_ = other.latestApiPayload_; + bitField0_ |= 0x00000080; + onChanged(); + } + if (other.eventType_ != 0) { + setEventTypeValue(other.getEventTypeValue()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + actor_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + detectedAt_ = input.readInt64(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 34: { + latestApiIp_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 42: { + latestApiEndpoint_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000010; + break; + } // case 42 + case 50: { + latestApiMethod_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000020; + break; + } // case 50 + case 56: { + latestApiCollectionId_ = input.readInt32(); + bitField0_ |= 0x00000040; + break; + } // case 56 + case 66: { + latestApiPayload_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000080; + break; + } // case 66 + case 72: { + eventType_ = input.readEnum(); + bitField0_ |= 0x00000100; + break; + } // case 72 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The actor to set. + * @return This builder for chaining. + */ + public Builder setActor( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @return This builder for chaining. + */ + public Builder clearActor() { + actor_ = getDefaultInstance().getActor(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The bytes for actor to set. + * @return This builder for chaining. + */ + public Builder setActorBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private long detectedAt_ ; + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @param value The detectedAt to set. + * @return This builder for chaining. + */ + public Builder setDetectedAt(long value) { + + detectedAt_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return This builder for chaining. + */ + public Builder clearDetectedAt() { + bitField0_ = (bitField0_ & ~0x00000004); + detectedAt_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object latestApiIp_ = ""; + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @return The latestApiIp. + */ + public java.lang.String getLatestApiIp() { + java.lang.Object ref = latestApiIp_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + latestApiIp_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @return The bytes for latestApiIp. + */ + public com.google.protobuf.ByteString + getLatestApiIpBytes() { + java.lang.Object ref = latestApiIp_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + latestApiIp_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @param value The latestApiIp to set. + * @return This builder for chaining. + */ + public Builder setLatestApiIp( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + latestApiIp_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @return This builder for chaining. + */ + public Builder clearLatestApiIp() { + latestApiIp_ = getDefaultInstance().getLatestApiIp(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @param value The bytes for latestApiIp to set. + * @return This builder for chaining. + */ + public Builder setLatestApiIpBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + latestApiIp_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + + private java.lang.Object latestApiEndpoint_ = ""; + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @return The latestApiEndpoint. + */ + public java.lang.String getLatestApiEndpoint() { + java.lang.Object ref = latestApiEndpoint_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + latestApiEndpoint_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @return The bytes for latestApiEndpoint. + */ + public com.google.protobuf.ByteString + getLatestApiEndpointBytes() { + java.lang.Object ref = latestApiEndpoint_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + latestApiEndpoint_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @param value The latestApiEndpoint to set. + * @return This builder for chaining. + */ + public Builder setLatestApiEndpoint( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + latestApiEndpoint_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @return This builder for chaining. + */ + public Builder clearLatestApiEndpoint() { + latestApiEndpoint_ = getDefaultInstance().getLatestApiEndpoint(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + return this; + } + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @param value The bytes for latestApiEndpoint to set. + * @return This builder for chaining. + */ + public Builder setLatestApiEndpointBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + latestApiEndpoint_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + + private java.lang.Object latestApiMethod_ = ""; + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @return The latestApiMethod. + */ + public java.lang.String getLatestApiMethod() { + java.lang.Object ref = latestApiMethod_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + latestApiMethod_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @return The bytes for latestApiMethod. + */ + public com.google.protobuf.ByteString + getLatestApiMethodBytes() { + java.lang.Object ref = latestApiMethod_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + latestApiMethod_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @param value The latestApiMethod to set. + * @return This builder for chaining. + */ + public Builder setLatestApiMethod( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + latestApiMethod_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @return This builder for chaining. + */ + public Builder clearLatestApiMethod() { + latestApiMethod_ = getDefaultInstance().getLatestApiMethod(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + return this; + } + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @param value The bytes for latestApiMethod to set. + * @return This builder for chaining. + */ + public Builder setLatestApiMethodBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + latestApiMethod_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + + private int latestApiCollectionId_ ; + /** + * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; + * @return The latestApiCollectionId. + */ + @java.lang.Override + public int getLatestApiCollectionId() { + return latestApiCollectionId_; + } + /** + * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; + * @param value The latestApiCollectionId to set. + * @return This builder for chaining. + */ + public Builder setLatestApiCollectionId(int value) { + + latestApiCollectionId_ = value; + bitField0_ |= 0x00000040; + onChanged(); + return this; + } + /** + * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; + * @return This builder for chaining. + */ + public Builder clearLatestApiCollectionId() { + bitField0_ = (bitField0_ & ~0x00000040); + latestApiCollectionId_ = 0; + onChanged(); + return this; + } + + private java.lang.Object latestApiPayload_ = ""; + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @return The latestApiPayload. + */ + public java.lang.String getLatestApiPayload() { + java.lang.Object ref = latestApiPayload_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + latestApiPayload_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @return The bytes for latestApiPayload. + */ + public com.google.protobuf.ByteString + getLatestApiPayloadBytes() { + java.lang.Object ref = latestApiPayload_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + latestApiPayload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @param value The latestApiPayload to set. + * @return This builder for chaining. + */ + public Builder setLatestApiPayload( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + latestApiPayload_ = value; + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @return This builder for chaining. + */ + public Builder clearLatestApiPayload() { + latestApiPayload_ = getDefaultInstance().getLatestApiPayload(); + bitField0_ = (bitField0_ & ~0x00000080); + onChanged(); + return this; + } + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @param value The bytes for latestApiPayload to set. + * @return This builder for chaining. + */ + public Builder setLatestApiPayloadBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + latestApiPayload_ = value; + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + + private int eventType_ = 0; + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @return The enum numeric value on the wire for eventType. + */ + @java.lang.Override public int getEventTypeValue() { + return eventType_; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @param value The enum numeric value on the wire for eventType to set. + * @return This builder for chaining. + */ + public Builder setEventTypeValue(int value) { + eventType_ = value; + bitField0_ |= 0x00000100; + onChanged(); + return this; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @return The eventType. + */ + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType getEventType() { + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType result = com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.forNumber(eventType_); + return result == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.UNRECOGNIZED : result; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @param value The eventType to set. + * @return This builder for chaining. + */ + public Builder setEventType(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000100; + eventType_ = value.getNumber(); + onChanged(); + return this; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @return This builder for chaining. + */ + public Builder clearEventType() { + bitField0_ = (bitField0_ & ~0x00000100); + eventType_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.message.malicious_event.v1.MaliciousEvent) + } + + // @@protoc_insertion_point(class_scope:threat_protection.message.malicious_event.v1.MaliciousEvent) + private static final com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent(); + } + + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MaliciousEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java new file mode 100644 index 0000000000..b37b92a7ec --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java @@ -0,0 +1,106 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/malicious_event/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.malicious_event.v1; + +public interface MaliciousEventOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.message.malicious_event.v1.MaliciousEvent) + com.google.protobuf.MessageOrBuilder { + + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + java.lang.String getActor(); + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + com.google.protobuf.ByteString + getActorBytes(); + + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + java.lang.String getFilterId(); + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + com.google.protobuf.ByteString + getFilterIdBytes(); + + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + long getDetectedAt(); + + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @return The latestApiIp. + */ + java.lang.String getLatestApiIp(); + /** + * string latest_api_ip = 4 [json_name = "latestApiIp"]; + * @return The bytes for latestApiIp. + */ + com.google.protobuf.ByteString + getLatestApiIpBytes(); + + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @return The latestApiEndpoint. + */ + java.lang.String getLatestApiEndpoint(); + /** + * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; + * @return The bytes for latestApiEndpoint. + */ + com.google.protobuf.ByteString + getLatestApiEndpointBytes(); + + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @return The latestApiMethod. + */ + java.lang.String getLatestApiMethod(); + /** + * string latest_api_method = 6 [json_name = "latestApiMethod"]; + * @return The bytes for latestApiMethod. + */ + com.google.protobuf.ByteString + getLatestApiMethodBytes(); + + /** + * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; + * @return The latestApiCollectionId. + */ + int getLatestApiCollectionId(); + + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @return The latestApiPayload. + */ + java.lang.String getLatestApiPayload(); + /** + * string latest_api_payload = 8 [json_name = "latestApiPayload"]; + * @return The bytes for latestApiPayload. + */ + com.google.protobuf.ByteString + getLatestApiPayloadBytes(); + + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @return The enum numeric value on the wire for eventType. + */ + int getEventTypeValue(); + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; + * @return The eventType. + */ + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType getEventType(); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java new file mode 100644 index 0000000000..5a1f6fb606 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java @@ -0,0 +1,81 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/malicious_event/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.malicious_event.v1; + +public final class MessageProto { + private MessageProto() {} + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + MessageProto.class.getName()); + } + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n:threat_protection/message/malicious_ev" + + "ent/v1/message.proto\022,threat_protection." + + "message.malicious_event.v1\"\215\004\n\016Malicious" + + "Event\022\024\n\005actor\030\001 \001(\tR\005actor\022\033\n\tfilter_id" + + "\030\002 \001(\tR\010filterId\022\037\n\013detected_at\030\003 \001(\003R\nd" + + "etectedAt\022\"\n\rlatest_api_ip\030\004 \001(\tR\013latest" + + "ApiIp\022.\n\023latest_api_endpoint\030\005 \001(\tR\021late" + + "stApiEndpoint\022*\n\021latest_api_method\030\006 \001(\t" + + "R\017latestApiMethod\0227\n\030latest_api_collecti" + + "on_id\030\007 \001(\005R\025latestApiCollectionId\022,\n\022la" + + "test_api_payload\030\010 \001(\tR\020latestApiPayload" + + "\022e\n\nevent_type\030\t \001(\0162F.threat_protection" + + ".message.malicious_event.v1.MaliciousEve" + + "nt.EventTypeR\teventType\"Y\n\tEventType\022\032\n\026" + + "EVENT_TYPE_UNSPECIFIED\020\000\022\025\n\021EVENT_TYPE_S" + + "INGLE\020\001\022\031\n\025EVENT_TYPE_AGGREGATED\020\002B\226\002\n;c" + + "om.akto.proto.threat_protection.message." + + "malicious_event.v1B\014MessageProtoP\001\242\002\003TMM" + + "\252\002*ThreatProtection.Message.MaliciousEve" + + "nt.V1\312\002*ThreatProtection\\Message\\Malicio" + + "usEvent\\V1\342\0026ThreatProtection\\Message\\Ma" + + "liciousEvent\\V1\\GPBMetadata\352\002-ThreatProt" + + "ection::Message::MaliciousEvent::V1b\006pro" + + "to3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor, + new java.lang.String[] { "Actor", "FilterId", "DetectedAt", "LatestApiIp", "LatestApiEndpoint", "LatestApiMethod", "LatestApiCollectionId", "LatestApiPayload", "EventType", }); + descriptor.resolveAllFeaturesImmutable(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/MessageProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/MessageProto.java new file mode 100644 index 0000000000..828d016696 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/MessageProto.java @@ -0,0 +1,73 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/sample_request/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.sample_request.v1; + +public final class MessageProto { + private MessageProto() {} + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + MessageProto.class.getName()); + } + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n9threat_protection/message/sample_reque" + + "st/v1/message.proto\022+threat_protection.m" + + "essage.sample_request.v1\"\323\001\n\026SampleMalic" + + "iousRequest\022\016\n\002ip\030\001 \001(\tR\002ip\022\034\n\ttimestamp" + + "\030\002 \001(\003R\ttimestamp\022\020\n\003url\030\003 \001(\tR\003url\022\026\n\006m" + + "ethod\030\004 \001(\tR\006method\022*\n\021api_collection_id" + + "\030\005 \001(\005R\017apiCollectionId\022\030\n\007payload\030\006 \001(\t" + + "R\007payload\022\033\n\tfilter_id\030\007 \001(\tR\010filterIdB\221" + + "\002\n:com.akto.proto.threat_protection.mess" + + "age.sample_request.v1B\014MessageProtoP\001\242\002\003" + + "TMS\252\002)ThreatProtection.Message.SampleReq" + + "uest.V1\312\002)ThreatProtection\\Message\\Sampl" + + "eRequest\\V1\342\0025ThreatProtection\\Message\\S" + + "ampleRequest\\V1\\GPBMetadata\352\002,ThreatProt" + + "ection::Message::SampleRequest::V1b\006prot" + + "o3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor, + new java.lang.String[] { "Ip", "Timestamp", "Url", "Method", "ApiCollectionId", "Payload", "FilterId", }); + descriptor.resolveAllFeaturesImmutable(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequest.java new file mode 100644 index 0000000000..c2cb45ef84 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequest.java @@ -0,0 +1,1178 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/sample_request/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.sample_request.v1; + +/** + * Protobuf type {@code threat_protection.message.sample_request.v1.SampleMaliciousRequest} + */ +public final class SampleMaliciousRequest extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.message.sample_request.v1.SampleMaliciousRequest) + SampleMaliciousRequestOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SampleMaliciousRequest.class.getName()); + } + // Use SampleMaliciousRequest.newBuilder() to construct. + private SampleMaliciousRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SampleMaliciousRequest() { + ip_ = ""; + url_ = ""; + method_ = ""; + payload_ = ""; + filterId_ = ""; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.class, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder.class); + } + + public static final int IP_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object ip_ = ""; + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + @java.lang.Override + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TIMESTAMP_FIELD_NUMBER = 2; + private long timestamp_ = 0L; + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + + public static final int URL_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object url_ = ""; + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + @java.lang.Override + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } + } + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int METHOD_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object method_ = ""; + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + @java.lang.Override + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } + } + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int API_COLLECTION_ID_FIELD_NUMBER = 5; + private int apiCollectionId_ = 0; + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + + public static final int PAYLOAD_FIELD_NUMBER = 6; + @SuppressWarnings("serial") + private volatile java.lang.Object payload_ = ""; + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + @java.lang.Override + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 7; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 7 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 7 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, ip_); + } + if (timestamp_ != 0L) { + output.writeInt64(2, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 3, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 4, method_); + } + if (apiCollectionId_ != 0) { + output.writeInt32(5, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 6, payload_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 7, filterId_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, ip_); + } + if (timestamp_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(2, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(3, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(4, method_); + } + if (apiCollectionId_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(5, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(6, payload_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(7, filterId_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest other = (com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest) obj; + + if (!getIp() + .equals(other.getIp())) return false; + if (getTimestamp() + != other.getTimestamp()) return false; + if (!getUrl() + .equals(other.getUrl())) return false; + if (!getMethod() + .equals(other.getMethod())) return false; + if (getApiCollectionId() + != other.getApiCollectionId()) return false; + if (!getPayload() + .equals(other.getPayload())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + IP_FIELD_NUMBER; + hash = (53 * hash) + getIp().hashCode(); + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getTimestamp()); + hash = (37 * hash) + URL_FIELD_NUMBER; + hash = (53 * hash) + getUrl().hashCode(); + hash = (37 * hash) + METHOD_FIELD_NUMBER; + hash = (53 * hash) + getMethod().hashCode(); + hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; + hash = (53 * hash) + getApiCollectionId(); + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.message.sample_request.v1.SampleMaliciousRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.message.sample_request.v1.SampleMaliciousRequest) + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.class, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ip_ = ""; + timestamp_ = 0L; + url_ = ""; + method_ = ""; + apiCollectionId_ = 0; + payload_ = ""; + filterId_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest build() { + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest buildPartial() { + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest result = new com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ip_ = ip_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.timestamp_ = timestamp_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.url_ = url_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.method_ = method_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.apiCollectionId_ = apiCollectionId_; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.payload_ = payload_; + } + if (((from_bitField0_ & 0x00000040) != 0)) { + result.filterId_ = filterId_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest) { + return mergeFrom((com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest other) { + if (other == com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.getDefaultInstance()) return this; + if (!other.getIp().isEmpty()) { + ip_ = other.ip_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.getTimestamp() != 0L) { + setTimestamp(other.getTimestamp()); + } + if (!other.getUrl().isEmpty()) { + url_ = other.url_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (!other.getMethod().isEmpty()) { + method_ = other.method_; + bitField0_ |= 0x00000008; + onChanged(); + } + if (other.getApiCollectionId() != 0) { + setApiCollectionId(other.getApiCollectionId()); + } + if (!other.getPayload().isEmpty()) { + payload_ = other.payload_; + bitField0_ |= 0x00000020; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000040; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + ip_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 26: { + url_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + method_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 40: { + apiCollectionId_ = input.readInt32(); + bitField0_ |= 0x00000010; + break; + } // case 40 + case 50: { + payload_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000020; + break; + } // case 50 + case 58: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000040; + break; + } // case 58 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object ip_ = ""; + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @param value The ip to set. + * @return This builder for chaining. + */ + public Builder setIp( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ip_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string ip = 1 [json_name = "ip"]; + * @return This builder for chaining. + */ + public Builder clearIp() { + ip_ = getDefaultInstance().getIp(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string ip = 1 [json_name = "ip"]; + * @param value The bytes for ip to set. + * @return This builder for chaining. + */ + public Builder setIpBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ip_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private long timestamp_ ; + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + + timestamp_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000002); + timestamp_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object url_ = ""; + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string url = 3 [json_name = "url"]; + * @param value The url to set. + * @return This builder for chaining. + */ + public Builder setUrl( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + url_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string url = 3 [json_name = "url"]; + * @return This builder for chaining. + */ + public Builder clearUrl() { + url_ = getDefaultInstance().getUrl(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string url = 3 [json_name = "url"]; + * @param value The bytes for url to set. + * @return This builder for chaining. + */ + public Builder setUrlBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + url_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private java.lang.Object method_ = ""; + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string method = 4 [json_name = "method"]; + * @param value The method to set. + * @return This builder for chaining. + */ + public Builder setMethod( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + method_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * string method = 4 [json_name = "method"]; + * @return This builder for chaining. + */ + public Builder clearMethod() { + method_ = getDefaultInstance().getMethod(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * string method = 4 [json_name = "method"]; + * @param value The bytes for method to set. + * @return This builder for chaining. + */ + public Builder setMethodBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + method_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + + private int apiCollectionId_ ; + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @param value The apiCollectionId to set. + * @return This builder for chaining. + */ + public Builder setApiCollectionId(int value) { + + apiCollectionId_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return This builder for chaining. + */ + public Builder clearApiCollectionId() { + bitField0_ = (bitField0_ & ~0x00000010); + apiCollectionId_ = 0; + onChanged(); + return this; + } + + private java.lang.Object payload_ = ""; + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @param value The payload to set. + * @return This builder for chaining. + */ + public Builder setPayload( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + payload_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * string payload = 6 [json_name = "payload"]; + * @return This builder for chaining. + */ + public Builder clearPayload() { + payload_ = getDefaultInstance().getPayload(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + return this; + } + /** + * string payload = 6 [json_name = "payload"]; + * @param value The bytes for payload to set. + * @return This builder for chaining. + */ + public Builder setPayloadBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + payload_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 7 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 7 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 7 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000040; + onChanged(); + return this; + } + /** + * string filter_id = 7 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000040); + onChanged(); + return this; + } + /** + * string filter_id = 7 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000040; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.message.sample_request.v1.SampleMaliciousRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.message.sample_request.v1.SampleMaliciousRequest) + private static final com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest(); + } + + public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SampleMaliciousRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequestOrBuilder.java new file mode 100644 index 0000000000..5080d4b4ae --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequestOrBuilder.java @@ -0,0 +1,83 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/sample_request/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.sample_request.v1; + +public interface SampleMaliciousRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.message.sample_request.v1.SampleMaliciousRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + java.lang.String getIp(); + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + com.google.protobuf.ByteString + getIpBytes(); + + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + long getTimestamp(); + + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + java.lang.String getUrl(); + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + com.google.protobuf.ByteString + getUrlBytes(); + + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + java.lang.String getMethod(); + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + com.google.protobuf.ByteString + getMethodBytes(); + + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + int getApiCollectionId(); + + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + java.lang.String getPayload(); + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + com.google.protobuf.ByteString + getPayloadBytes(); + + /** + * string filter_id = 7 [json_name = "filterId"]; + * @return The filterId. + */ + java.lang.String getFilterId(); + /** + * string filter_id = 7 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + com.google.protobuf.ByteString + getFilterIdBytes(); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/DashboardServiceGrpc.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/DashboardServiceGrpc.java new file mode 100644 index 0000000000..002d8e8a71 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/DashboardServiceGrpc.java @@ -0,0 +1,367 @@ +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +import static io.grpc.MethodDescriptor.generateFullMethodName; + +/** + */ +@javax.annotation.Generated( + value = "by gRPC proto compiler (version 1.68.1)", + comments = "Source: threat_protection/service/dashboard_service/v1/service.proto") +@io.grpc.stub.annotations.GrpcGenerated +public final class DashboardServiceGrpc { + + private DashboardServiceGrpc() {} + + public static final java.lang.String SERVICE_NAME = "threat_protection.service.dashboard_service.v1.DashboardService"; + + // Static method descriptors that strictly reflect the proto. + private static volatile io.grpc.MethodDescriptor getListMaliciousRequestsMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "ListMaliciousRequests", + requestType = com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.class, + responseType = com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getListMaliciousRequestsMethod() { + io.grpc.MethodDescriptor getListMaliciousRequestsMethod; + if ((getListMaliciousRequestsMethod = DashboardServiceGrpc.getListMaliciousRequestsMethod) == null) { + synchronized (DashboardServiceGrpc.class) { + if ((getListMaliciousRequestsMethod = DashboardServiceGrpc.getListMaliciousRequestsMethod) == null) { + DashboardServiceGrpc.getListMaliciousRequestsMethod = getListMaliciousRequestsMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "ListMaliciousRequests")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.getDefaultInstance())) + .setSchemaDescriptor(new DashboardServiceMethodDescriptorSupplier("ListMaliciousRequests")) + .build(); + } + } + } + return getListMaliciousRequestsMethod; + } + + private static volatile io.grpc.MethodDescriptor getFetchAlertFiltersMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "FetchAlertFilters", + requestType = com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.class, + responseType = com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getFetchAlertFiltersMethod() { + io.grpc.MethodDescriptor getFetchAlertFiltersMethod; + if ((getFetchAlertFiltersMethod = DashboardServiceGrpc.getFetchAlertFiltersMethod) == null) { + synchronized (DashboardServiceGrpc.class) { + if ((getFetchAlertFiltersMethod = DashboardServiceGrpc.getFetchAlertFiltersMethod) == null) { + DashboardServiceGrpc.getFetchAlertFiltersMethod = getFetchAlertFiltersMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "FetchAlertFilters")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.getDefaultInstance())) + .setSchemaDescriptor(new DashboardServiceMethodDescriptorSupplier("FetchAlertFilters")) + .build(); + } + } + } + return getFetchAlertFiltersMethod; + } + + /** + * Creates a new async stub that supports all call types for the service + */ + public static DashboardServiceStub newStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public DashboardServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new DashboardServiceStub(channel, callOptions); + } + }; + return DashboardServiceStub.newStub(factory, channel); + } + + /** + * Creates a new blocking-style stub that supports unary and streaming output calls on the service + */ + public static DashboardServiceBlockingStub newBlockingStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public DashboardServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new DashboardServiceBlockingStub(channel, callOptions); + } + }; + return DashboardServiceBlockingStub.newStub(factory, channel); + } + + /** + * Creates a new ListenableFuture-style stub that supports unary calls on the service + */ + public static DashboardServiceFutureStub newFutureStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public DashboardServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new DashboardServiceFutureStub(channel, callOptions); + } + }; + return DashboardServiceFutureStub.newStub(factory, channel); + } + + /** + */ + public interface AsyncService { + + /** + */ + default void listMaliciousRequests(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getListMaliciousRequestsMethod(), responseObserver); + } + + /** + */ + default void fetchAlertFilters(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getFetchAlertFiltersMethod(), responseObserver); + } + } + + /** + * Base class for the server implementation of the service DashboardService. + */ + public static abstract class DashboardServiceImplBase + implements io.grpc.BindableService, AsyncService { + + @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { + return DashboardServiceGrpc.bindService(this); + } + } + + /** + * A stub to allow clients to do asynchronous rpc calls to service DashboardService. + */ + public static final class DashboardServiceStub + extends io.grpc.stub.AbstractAsyncStub { + private DashboardServiceStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected DashboardServiceStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new DashboardServiceStub(channel, callOptions); + } + + /** + */ + public void listMaliciousRequests(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getListMaliciousRequestsMethod(), getCallOptions()), request, responseObserver); + } + + /** + */ + public void fetchAlertFilters(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getFetchAlertFiltersMethod(), getCallOptions()), request, responseObserver); + } + } + + /** + * A stub to allow clients to do synchronous rpc calls to service DashboardService. + */ + public static final class DashboardServiceBlockingStub + extends io.grpc.stub.AbstractBlockingStub { + private DashboardServiceBlockingStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected DashboardServiceBlockingStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new DashboardServiceBlockingStub(channel, callOptions); + } + + /** + */ + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse listMaliciousRequests(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getListMaliciousRequestsMethod(), getCallOptions(), request); + } + + /** + */ + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse fetchAlertFilters(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getFetchAlertFiltersMethod(), getCallOptions(), request); + } + } + + /** + * A stub to allow clients to do ListenableFuture-style rpc calls to service DashboardService. + */ + public static final class DashboardServiceFutureStub + extends io.grpc.stub.AbstractFutureStub { + private DashboardServiceFutureStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected DashboardServiceFutureStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new DashboardServiceFutureStub(channel, callOptions); + } + + /** + */ + public com.google.common.util.concurrent.ListenableFuture listMaliciousRequests( + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getListMaliciousRequestsMethod(), getCallOptions()), request); + } + + /** + */ + public com.google.common.util.concurrent.ListenableFuture fetchAlertFilters( + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getFetchAlertFiltersMethod(), getCallOptions()), request); + } + } + + private static final int METHODID_LIST_MALICIOUS_REQUESTS = 0; + private static final int METHODID_FETCH_ALERT_FILTERS = 1; + + private static final class MethodHandlers implements + io.grpc.stub.ServerCalls.UnaryMethod, + io.grpc.stub.ServerCalls.ServerStreamingMethod, + io.grpc.stub.ServerCalls.ClientStreamingMethod, + io.grpc.stub.ServerCalls.BidiStreamingMethod { + private final AsyncService serviceImpl; + private final int methodId; + + MethodHandlers(AsyncService serviceImpl, int methodId) { + this.serviceImpl = serviceImpl; + this.methodId = methodId; + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + case METHODID_LIST_MALICIOUS_REQUESTS: + serviceImpl.listMaliciousRequests((com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + case METHODID_FETCH_ALERT_FILTERS: + serviceImpl.fetchAlertFilters((com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + default: + throw new AssertionError(); + } + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public io.grpc.stub.StreamObserver invoke( + io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + default: + throw new AssertionError(); + } + } + } + + public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { + return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) + .addMethod( + getListMaliciousRequestsMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest, + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse>( + service, METHODID_LIST_MALICIOUS_REQUESTS))) + .addMethod( + getFetchAlertFiltersMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest, + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse>( + service, METHODID_FETCH_ALERT_FILTERS))) + .build(); + } + + private static abstract class DashboardServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { + DashboardServiceBaseDescriptorSupplier() {} + + @java.lang.Override + public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.getDescriptor(); + } + + @java.lang.Override + public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { + return getFileDescriptor().findServiceByName("DashboardService"); + } + } + + private static final class DashboardServiceFileDescriptorSupplier + extends DashboardServiceBaseDescriptorSupplier { + DashboardServiceFileDescriptorSupplier() {} + } + + private static final class DashboardServiceMethodDescriptorSupplier + extends DashboardServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { + private final java.lang.String methodName; + + DashboardServiceMethodDescriptorSupplier(java.lang.String methodName) { + this.methodName = methodName; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { + return getServiceDescriptor().findMethodByName(methodName); + } + } + + private static volatile io.grpc.ServiceDescriptor serviceDescriptor; + + public static io.grpc.ServiceDescriptor getServiceDescriptor() { + io.grpc.ServiceDescriptor result = serviceDescriptor; + if (result == null) { + synchronized (DashboardServiceGrpc.class) { + result = serviceDescriptor; + if (result == null) { + serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) + .setSchemaDescriptor(new DashboardServiceFileDescriptorSupplier()) + .addMethod(getListMaliciousRequestsMethod()) + .addMethod(getFetchAlertFiltersMethod()) + .build(); + } + } + } + return result; + } +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequest.java new file mode 100644 index 0000000000..13696e35ee --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequest.java @@ -0,0 +1,358 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +/** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest} + */ +public final class FetchAlertFiltersRequest extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) + FetchAlertFiltersRequestOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + FetchAlertFiltersRequest.class.getName()); + } + // Use FetchAlertFiltersRequest.newBuilder() to construct. + private FetchAlertFiltersRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private FetchAlertFiltersRequest() { + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.Builder.class); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest other = (com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) obj; + + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest build() { + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest buildPartial() { + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest result = new com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) { + return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest other) { + if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) + private static final com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest(); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public FetchAlertFiltersRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequestOrBuilder.java new file mode 100644 index 0000000000..c408feed12 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequestOrBuilder.java @@ -0,0 +1,11 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +public interface FetchAlertFiltersRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) + com.google.protobuf.MessageOrBuilder { +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponse.java new file mode 100644 index 0000000000..ff75ad076b --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponse.java @@ -0,0 +1,743 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +/** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse} + */ +public final class FetchAlertFiltersResponse extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) + FetchAlertFiltersResponseOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + FetchAlertFiltersResponse.class.getName()); + } + // Use FetchAlertFiltersResponse.newBuilder() to construct. + private FetchAlertFiltersResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private FetchAlertFiltersResponse() { + actors_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + urls_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.class, com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.Builder.class); + } + + public static final int ACTORS_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private com.google.protobuf.LazyStringArrayList actors_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + /** + * repeated string actors = 1 [json_name = "actors"]; + * @return A list containing the actors. + */ + public com.google.protobuf.ProtocolStringList + getActorsList() { + return actors_; + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @return The count of actors. + */ + public int getActorsCount() { + return actors_.size(); + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param index The index of the element to return. + * @return The actors at the given index. + */ + public java.lang.String getActors(int index) { + return actors_.get(index); + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param index The index of the value to return. + * @return The bytes of the actors at the given index. + */ + public com.google.protobuf.ByteString + getActorsBytes(int index) { + return actors_.getByteString(index); + } + + public static final int URLS_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private com.google.protobuf.LazyStringArrayList urls_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + /** + * repeated string urls = 2 [json_name = "urls"]; + * @return A list containing the urls. + */ + public com.google.protobuf.ProtocolStringList + getUrlsList() { + return urls_; + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @return The count of urls. + */ + public int getUrlsCount() { + return urls_.size(); + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param index The index of the element to return. + * @return The urls at the given index. + */ + public java.lang.String getUrls(int index) { + return urls_.get(index); + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param index The index of the value to return. + * @return The bytes of the urls at the given index. + */ + public com.google.protobuf.ByteString + getUrlsBytes(int index) { + return urls_.getByteString(index); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < actors_.size(); i++) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, actors_.getRaw(i)); + } + for (int i = 0; i < urls_.size(); i++) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, urls_.getRaw(i)); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + { + int dataSize = 0; + for (int i = 0; i < actors_.size(); i++) { + dataSize += computeStringSizeNoTag(actors_.getRaw(i)); + } + size += dataSize; + size += 1 * getActorsList().size(); + } + { + int dataSize = 0; + for (int i = 0; i < urls_.size(); i++) { + dataSize += computeStringSizeNoTag(urls_.getRaw(i)); + } + size += dataSize; + size += 1 * getUrlsList().size(); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse other = (com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) obj; + + if (!getActorsList() + .equals(other.getActorsList())) return false; + if (!getUrlsList() + .equals(other.getUrlsList())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (getActorsCount() > 0) { + hash = (37 * hash) + ACTORS_FIELD_NUMBER; + hash = (53 * hash) + getActorsList().hashCode(); + } + if (getUrlsCount() > 0) { + hash = (37 * hash) + URLS_FIELD_NUMBER; + hash = (53 * hash) + getUrlsList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.class, com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + actors_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + urls_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse build() { + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse buildPartial() { + com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse result = new com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + actors_.makeImmutable(); + result.actors_ = actors_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + urls_.makeImmutable(); + result.urls_ = urls_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) { + return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse other) { + if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.getDefaultInstance()) return this; + if (!other.actors_.isEmpty()) { + if (actors_.isEmpty()) { + actors_ = other.actors_; + bitField0_ |= 0x00000001; + } else { + ensureActorsIsMutable(); + actors_.addAll(other.actors_); + } + onChanged(); + } + if (!other.urls_.isEmpty()) { + if (urls_.isEmpty()) { + urls_ = other.urls_; + bitField0_ |= 0x00000002; + } else { + ensureUrlsIsMutable(); + urls_.addAll(other.urls_); + } + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + ensureActorsIsMutable(); + actors_.add(s); + break; + } // case 10 + case 18: { + java.lang.String s = input.readStringRequireUtf8(); + ensureUrlsIsMutable(); + urls_.add(s); + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private com.google.protobuf.LazyStringArrayList actors_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + private void ensureActorsIsMutable() { + if (!actors_.isModifiable()) { + actors_ = new com.google.protobuf.LazyStringArrayList(actors_); + } + bitField0_ |= 0x00000001; + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @return A list containing the actors. + */ + public com.google.protobuf.ProtocolStringList + getActorsList() { + actors_.makeImmutable(); + return actors_; + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @return The count of actors. + */ + public int getActorsCount() { + return actors_.size(); + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param index The index of the element to return. + * @return The actors at the given index. + */ + public java.lang.String getActors(int index) { + return actors_.get(index); + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param index The index of the value to return. + * @return The bytes of the actors at the given index. + */ + public com.google.protobuf.ByteString + getActorsBytes(int index) { + return actors_.getByteString(index); + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param index The index to set the value at. + * @param value The actors to set. + * @return This builder for chaining. + */ + public Builder setActors( + int index, java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ensureActorsIsMutable(); + actors_.set(index, value); + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param value The actors to add. + * @return This builder for chaining. + */ + public Builder addActors( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ensureActorsIsMutable(); + actors_.add(value); + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param values The actors to add. + * @return This builder for chaining. + */ + public Builder addAllActors( + java.lang.Iterable values) { + ensureActorsIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, actors_); + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @return This builder for chaining. + */ + public Builder clearActors() { + actors_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001);; + onChanged(); + return this; + } + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param value The bytes of the actors to add. + * @return This builder for chaining. + */ + public Builder addActorsBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ensureActorsIsMutable(); + actors_.add(value); + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private com.google.protobuf.LazyStringArrayList urls_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + private void ensureUrlsIsMutable() { + if (!urls_.isModifiable()) { + urls_ = new com.google.protobuf.LazyStringArrayList(urls_); + } + bitField0_ |= 0x00000002; + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @return A list containing the urls. + */ + public com.google.protobuf.ProtocolStringList + getUrlsList() { + urls_.makeImmutable(); + return urls_; + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @return The count of urls. + */ + public int getUrlsCount() { + return urls_.size(); + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param index The index of the element to return. + * @return The urls at the given index. + */ + public java.lang.String getUrls(int index) { + return urls_.get(index); + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param index The index of the value to return. + * @return The bytes of the urls at the given index. + */ + public com.google.protobuf.ByteString + getUrlsBytes(int index) { + return urls_.getByteString(index); + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param index The index to set the value at. + * @param value The urls to set. + * @return This builder for chaining. + */ + public Builder setUrls( + int index, java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ensureUrlsIsMutable(); + urls_.set(index, value); + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param value The urls to add. + * @return This builder for chaining. + */ + public Builder addUrls( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ensureUrlsIsMutable(); + urls_.add(value); + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param values The urls to add. + * @return This builder for chaining. + */ + public Builder addAllUrls( + java.lang.Iterable values) { + ensureUrlsIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, urls_); + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @return This builder for chaining. + */ + public Builder clearUrls() { + urls_ = + com.google.protobuf.LazyStringArrayList.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002);; + onChanged(); + return this; + } + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param value The bytes of the urls to add. + * @return This builder for chaining. + */ + public Builder addUrlsBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ensureUrlsIsMutable(); + urls_.add(value); + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) + private static final com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse(); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public FetchAlertFiltersResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponseOrBuilder.java new file mode 100644 index 0000000000..9d9cc50f6a --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponseOrBuilder.java @@ -0,0 +1,61 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +public interface FetchAlertFiltersResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) + com.google.protobuf.MessageOrBuilder { + + /** + * repeated string actors = 1 [json_name = "actors"]; + * @return A list containing the actors. + */ + java.util.List + getActorsList(); + /** + * repeated string actors = 1 [json_name = "actors"]; + * @return The count of actors. + */ + int getActorsCount(); + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param index The index of the element to return. + * @return The actors at the given index. + */ + java.lang.String getActors(int index); + /** + * repeated string actors = 1 [json_name = "actors"]; + * @param index The index of the value to return. + * @return The bytes of the actors at the given index. + */ + com.google.protobuf.ByteString + getActorsBytes(int index); + + /** + * repeated string urls = 2 [json_name = "urls"]; + * @return A list containing the urls. + */ + java.util.List + getUrlsList(); + /** + * repeated string urls = 2 [json_name = "urls"]; + * @return The count of urls. + */ + int getUrlsCount(); + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param index The index of the element to return. + * @return The urls at the given index. + */ + java.lang.String getUrls(int index); + /** + * repeated string urls = 2 [json_name = "urls"]; + * @param index The index of the value to return. + * @return The bytes of the urls at the given index. + */ + com.google.protobuf.ByteString + getUrlsBytes(int index); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequest.java new file mode 100644 index 0000000000..dfbacb88cd --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequest.java @@ -0,0 +1,538 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +/** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest} + */ +public final class ListMaliciousRequestsRequest extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) + ListMaliciousRequestsRequestOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + ListMaliciousRequestsRequest.class.getName()); + } + // Use ListMaliciousRequestsRequest.newBuilder() to construct. + private ListMaliciousRequestsRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private ListMaliciousRequestsRequest() { + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.Builder.class); + } + + private int bitField0_; + public static final int LIMIT_FIELD_NUMBER = 3; + private int limit_ = 0; + /** + *
+   * The number of alerts to return
+   * 
+ * + * int32 limit = 3 [json_name = "limit"]; + * @return The limit. + */ + @java.lang.Override + public int getLimit() { + return limit_; + } + + public static final int PAGE_FIELD_NUMBER = 4; + private int page_ = 0; + /** + * optional int32 page = 4 [json_name = "page"]; + * @return Whether the page field is set. + */ + @java.lang.Override + public boolean hasPage() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional int32 page = 4 [json_name = "page"]; + * @return The page. + */ + @java.lang.Override + public int getPage() { + return page_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (limit_ != 0) { + output.writeInt32(3, limit_); + } + if (((bitField0_ & 0x00000001) != 0)) { + output.writeInt32(4, page_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (limit_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(3, limit_); + } + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(4, page_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest other = (com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) obj; + + if (getLimit() + != other.getLimit()) return false; + if (hasPage() != other.hasPage()) return false; + if (hasPage()) { + if (getPage() + != other.getPage()) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + LIMIT_FIELD_NUMBER; + hash = (53 * hash) + getLimit(); + if (hasPage()) { + hash = (37 * hash) + PAGE_FIELD_NUMBER; + hash = (53 * hash) + getPage(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + limit_ = 0; + page_ = 0; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest build() { + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest buildPartial() { + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest result = new com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.limit_ = limit_; + } + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000002) != 0)) { + result.page_ = page_; + to_bitField0_ |= 0x00000001; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) { + return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest other) { + if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.getDefaultInstance()) return this; + if (other.getLimit() != 0) { + setLimit(other.getLimit()); + } + if (other.hasPage()) { + setPage(other.getPage()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 24: { + limit_ = input.readInt32(); + bitField0_ |= 0x00000001; + break; + } // case 24 + case 32: { + page_ = input.readInt32(); + bitField0_ |= 0x00000002; + break; + } // case 32 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private int limit_ ; + /** + *
+     * The number of alerts to return
+     * 
+ * + * int32 limit = 3 [json_name = "limit"]; + * @return The limit. + */ + @java.lang.Override + public int getLimit() { + return limit_; + } + /** + *
+     * The number of alerts to return
+     * 
+ * + * int32 limit = 3 [json_name = "limit"]; + * @param value The limit to set. + * @return This builder for chaining. + */ + public Builder setLimit(int value) { + + limit_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + *
+     * The number of alerts to return
+     * 
+ * + * int32 limit = 3 [json_name = "limit"]; + * @return This builder for chaining. + */ + public Builder clearLimit() { + bitField0_ = (bitField0_ & ~0x00000001); + limit_ = 0; + onChanged(); + return this; + } + + private int page_ ; + /** + * optional int32 page = 4 [json_name = "page"]; + * @return Whether the page field is set. + */ + @java.lang.Override + public boolean hasPage() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional int32 page = 4 [json_name = "page"]; + * @return The page. + */ + @java.lang.Override + public int getPage() { + return page_; + } + /** + * optional int32 page = 4 [json_name = "page"]; + * @param value The page to set. + * @return This builder for chaining. + */ + public Builder setPage(int value) { + + page_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * optional int32 page = 4 [json_name = "page"]; + * @return This builder for chaining. + */ + public Builder clearPage() { + bitField0_ = (bitField0_ & ~0x00000002); + page_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) + private static final com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest(); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public ListMaliciousRequestsRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequestOrBuilder.java new file mode 100644 index 0000000000..654dd72745 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequestOrBuilder.java @@ -0,0 +1,32 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +public interface ListMaliciousRequestsRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) + com.google.protobuf.MessageOrBuilder { + + /** + *
+   * The number of alerts to return
+   * 
+ * + * int32 limit = 3 [json_name = "limit"]; + * @return The limit. + */ + int getLimit(); + + /** + * optional int32 page = 4 [json_name = "page"]; + * @return Whether the page field is set. + */ + boolean hasPage(); + /** + * optional int32 page = 4 [json_name = "page"]; + * @return The page. + */ + int getPage(); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponse.java new file mode 100644 index 0000000000..c7e6041b01 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponse.java @@ -0,0 +1,851 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +/** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse} + */ +public final class ListMaliciousRequestsResponse extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) + ListMaliciousRequestsResponseOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + ListMaliciousRequestsResponse.class.getName()); + } + // Use ListMaliciousRequestsResponse.newBuilder() to construct. + private ListMaliciousRequestsResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private ListMaliciousRequestsResponse() { + maliciousRequests_ = java.util.Collections.emptyList(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.class, com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.Builder.class); + } + + public static final int MALICIOUS_REQUESTS_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private java.util.List maliciousRequests_; + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + @java.lang.Override + public java.util.List getMaliciousRequestsList() { + return maliciousRequests_; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + @java.lang.Override + public java.util.List + getMaliciousRequestsOrBuilderList() { + return maliciousRequests_; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + @java.lang.Override + public int getMaliciousRequestsCount() { + return maliciousRequests_.size(); + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getMaliciousRequests(int index) { + return maliciousRequests_.get(index); + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder getMaliciousRequestsOrBuilder( + int index) { + return maliciousRequests_.get(index); + } + + public static final int TOTAL_FIELD_NUMBER = 2; + private int total_ = 0; + /** + * int32 total = 2 [json_name = "total"]; + * @return The total. + */ + @java.lang.Override + public int getTotal() { + return total_; + } + + public static final int PAGE_FIELD_NUMBER = 3; + private int page_ = 0; + /** + * int32 page = 3 [json_name = "page"]; + * @return The page. + */ + @java.lang.Override + public int getPage() { + return page_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < maliciousRequests_.size(); i++) { + output.writeMessage(1, maliciousRequests_.get(i)); + } + if (total_ != 0) { + output.writeInt32(2, total_); + } + if (page_ != 0) { + output.writeInt32(3, page_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < maliciousRequests_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, maliciousRequests_.get(i)); + } + if (total_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(2, total_); + } + if (page_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(3, page_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse other = (com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) obj; + + if (!getMaliciousRequestsList() + .equals(other.getMaliciousRequestsList())) return false; + if (getTotal() + != other.getTotal()) return false; + if (getPage() + != other.getPage()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (getMaliciousRequestsCount() > 0) { + hash = (37 * hash) + MALICIOUS_REQUESTS_FIELD_NUMBER; + hash = (53 * hash) + getMaliciousRequestsList().hashCode(); + } + hash = (37 * hash) + TOTAL_FIELD_NUMBER; + hash = (53 * hash) + getTotal(); + hash = (37 * hash) + PAGE_FIELD_NUMBER; + hash = (53 * hash) + getPage(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.class, com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + if (maliciousRequestsBuilder_ == null) { + maliciousRequests_ = java.util.Collections.emptyList(); + } else { + maliciousRequests_ = null; + maliciousRequestsBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + total_ = 0; + page_ = 0; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse build() { + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse buildPartial() { + com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse result = new com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse(this); + buildPartialRepeatedFields(result); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartialRepeatedFields(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse result) { + if (maliciousRequestsBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0)) { + maliciousRequests_ = java.util.Collections.unmodifiableList(maliciousRequests_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.maliciousRequests_ = maliciousRequests_; + } else { + result.maliciousRequests_ = maliciousRequestsBuilder_.build(); + } + } + + private void buildPartial0(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000002) != 0)) { + result.total_ = total_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.page_ = page_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) { + return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse other) { + if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.getDefaultInstance()) return this; + if (maliciousRequestsBuilder_ == null) { + if (!other.maliciousRequests_.isEmpty()) { + if (maliciousRequests_.isEmpty()) { + maliciousRequests_ = other.maliciousRequests_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.addAll(other.maliciousRequests_); + } + onChanged(); + } + } else { + if (!other.maliciousRequests_.isEmpty()) { + if (maliciousRequestsBuilder_.isEmpty()) { + maliciousRequestsBuilder_.dispose(); + maliciousRequestsBuilder_ = null; + maliciousRequests_ = other.maliciousRequests_; + bitField0_ = (bitField0_ & ~0x00000001); + maliciousRequestsBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getMaliciousRequestsFieldBuilder() : null; + } else { + maliciousRequestsBuilder_.addAllMessages(other.maliciousRequests_); + } + } + } + if (other.getTotal() != 0) { + setTotal(other.getTotal()); + } + if (other.getPage() != 0) { + setPage(other.getPage()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest m = + input.readMessage( + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.parser(), + extensionRegistry); + if (maliciousRequestsBuilder_ == null) { + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.add(m); + } else { + maliciousRequestsBuilder_.addMessage(m); + } + break; + } // case 10 + case 16: { + total_ = input.readInt32(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 24: { + page_ = input.readInt32(); + bitField0_ |= 0x00000004; + break; + } // case 24 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.util.List maliciousRequests_ = + java.util.Collections.emptyList(); + private void ensureMaliciousRequestsIsMutable() { + if (!((bitField0_ & 0x00000001) != 0)) { + maliciousRequests_ = new java.util.ArrayList(maliciousRequests_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder> maliciousRequestsBuilder_; + + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public java.util.List getMaliciousRequestsList() { + if (maliciousRequestsBuilder_ == null) { + return java.util.Collections.unmodifiableList(maliciousRequests_); + } else { + return maliciousRequestsBuilder_.getMessageList(); + } + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public int getMaliciousRequestsCount() { + if (maliciousRequestsBuilder_ == null) { + return maliciousRequests_.size(); + } else { + return maliciousRequestsBuilder_.getCount(); + } + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getMaliciousRequests(int index) { + if (maliciousRequestsBuilder_ == null) { + return maliciousRequests_.get(index); + } else { + return maliciousRequestsBuilder_.getMessage(index); + } + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder setMaliciousRequests( + int index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest value) { + if (maliciousRequestsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.set(index, value); + onChanged(); + } else { + maliciousRequestsBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder setMaliciousRequests( + int index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder builderForValue) { + if (maliciousRequestsBuilder_ == null) { + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.set(index, builderForValue.build()); + onChanged(); + } else { + maliciousRequestsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder addMaliciousRequests(com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest value) { + if (maliciousRequestsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.add(value); + onChanged(); + } else { + maliciousRequestsBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder addMaliciousRequests( + int index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest value) { + if (maliciousRequestsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.add(index, value); + onChanged(); + } else { + maliciousRequestsBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder addMaliciousRequests( + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder builderForValue) { + if (maliciousRequestsBuilder_ == null) { + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.add(builderForValue.build()); + onChanged(); + } else { + maliciousRequestsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder addMaliciousRequests( + int index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder builderForValue) { + if (maliciousRequestsBuilder_ == null) { + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.add(index, builderForValue.build()); + onChanged(); + } else { + maliciousRequestsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder addAllMaliciousRequests( + java.lang.Iterable values) { + if (maliciousRequestsBuilder_ == null) { + ensureMaliciousRequestsIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, maliciousRequests_); + onChanged(); + } else { + maliciousRequestsBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder clearMaliciousRequests() { + if (maliciousRequestsBuilder_ == null) { + maliciousRequests_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + maliciousRequestsBuilder_.clear(); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public Builder removeMaliciousRequests(int index) { + if (maliciousRequestsBuilder_ == null) { + ensureMaliciousRequestsIsMutable(); + maliciousRequests_.remove(index); + onChanged(); + } else { + maliciousRequestsBuilder_.remove(index); + } + return this; + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder getMaliciousRequestsBuilder( + int index) { + return getMaliciousRequestsFieldBuilder().getBuilder(index); + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder getMaliciousRequestsOrBuilder( + int index) { + if (maliciousRequestsBuilder_ == null) { + return maliciousRequests_.get(index); } else { + return maliciousRequestsBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public java.util.List + getMaliciousRequestsOrBuilderList() { + if (maliciousRequestsBuilder_ != null) { + return maliciousRequestsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(maliciousRequests_); + } + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder addMaliciousRequestsBuilder() { + return getMaliciousRequestsFieldBuilder().addBuilder( + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.getDefaultInstance()); + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder addMaliciousRequestsBuilder( + int index) { + return getMaliciousRequestsFieldBuilder().addBuilder( + index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.getDefaultInstance()); + } + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + public java.util.List + getMaliciousRequestsBuilderList() { + return getMaliciousRequestsFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder> + getMaliciousRequestsFieldBuilder() { + if (maliciousRequestsBuilder_ == null) { + maliciousRequestsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder>( + maliciousRequests_, + ((bitField0_ & 0x00000001) != 0), + getParentForChildren(), + isClean()); + maliciousRequests_ = null; + } + return maliciousRequestsBuilder_; + } + + private int total_ ; + /** + * int32 total = 2 [json_name = "total"]; + * @return The total. + */ + @java.lang.Override + public int getTotal() { + return total_; + } + /** + * int32 total = 2 [json_name = "total"]; + * @param value The total to set. + * @return This builder for chaining. + */ + public Builder setTotal(int value) { + + total_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * int32 total = 2 [json_name = "total"]; + * @return This builder for chaining. + */ + public Builder clearTotal() { + bitField0_ = (bitField0_ & ~0x00000002); + total_ = 0; + onChanged(); + return this; + } + + private int page_ ; + /** + * int32 page = 3 [json_name = "page"]; + * @return The page. + */ + @java.lang.Override + public int getPage() { + return page_; + } + /** + * int32 page = 3 [json_name = "page"]; + * @param value The page to set. + * @return This builder for chaining. + */ + public Builder setPage(int value) { + + page_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int32 page = 3 [json_name = "page"]; + * @return This builder for chaining. + */ + public Builder clearPage() { + bitField0_ = (bitField0_ & ~0x00000004); + page_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) + private static final com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse(); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public ListMaliciousRequestsResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponseOrBuilder.java new file mode 100644 index 0000000000..e020064c48 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponseOrBuilder.java @@ -0,0 +1,47 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +public interface ListMaliciousRequestsResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) + com.google.protobuf.MessageOrBuilder { + + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + java.util.List + getMaliciousRequestsList(); + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getMaliciousRequests(int index); + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + int getMaliciousRequestsCount(); + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + java.util.List + getMaliciousRequestsOrBuilderList(); + /** + * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; + */ + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder getMaliciousRequestsOrBuilder( + int index); + + /** + * int32 total = 2 [json_name = "total"]; + * @return The total. + */ + int getTotal(); + + /** + * int32 page = 3 [json_name = "page"]; + * @return The page. + */ + int getPage(); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequest.java new file mode 100644 index 0000000000..fdcc670c24 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequest.java @@ -0,0 +1,1520 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +/** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.MaliciousRequest} + */ +public final class MaliciousRequest extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.MaliciousRequest) + MaliciousRequestOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + MaliciousRequest.class.getName()); + } + // Use MaliciousRequest.newBuilder() to construct. + private MaliciousRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private MaliciousRequest() { + id_ = ""; + actor_ = ""; + filterId_ = ""; + url_ = ""; + method_ = ""; + orig_ = ""; + ip_ = ""; + country_ = ""; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder.class); + } + + public static final int ID_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object id_ = ""; + /** + * string id = 1 [json_name = "id"]; + * @return The id. + */ + @java.lang.Override + public java.lang.String getId() { + java.lang.Object ref = id_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + id_ = s; + return s; + } + } + /** + * string id = 1 [json_name = "id"]; + * @return The bytes for id. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getIdBytes() { + java.lang.Object ref = id_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + id_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int ACTOR_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object actor_ = ""; + /** + * string actor = 2 [json_name = "actor"]; + * @return The actor. + */ + @java.lang.Override + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } + } + /** + * string actor = 2 [json_name = "actor"]; + * @return The bytes for actor. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 3 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 3 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int URL_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object url_ = ""; + /** + * string url = 4 [json_name = "url"]; + * @return The url. + */ + @java.lang.Override + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } + } + /** + * string url = 4 [json_name = "url"]; + * @return The bytes for url. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int METHOD_FIELD_NUMBER = 5; + @SuppressWarnings("serial") + private volatile java.lang.Object method_ = ""; + /** + * string method = 5 [json_name = "method"]; + * @return The method. + */ + @java.lang.Override + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } + } + /** + * string method = 5 [json_name = "method"]; + * @return The bytes for method. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int ORIG_FIELD_NUMBER = 6; + @SuppressWarnings("serial") + private volatile java.lang.Object orig_ = ""; + /** + * string orig = 6 [json_name = "orig"]; + * @return The orig. + */ + @java.lang.Override + public java.lang.String getOrig() { + java.lang.Object ref = orig_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + orig_ = s; + return s; + } + } + /** + * string orig = 6 [json_name = "orig"]; + * @return The bytes for orig. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getOrigBytes() { + java.lang.Object ref = orig_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + orig_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int IP_FIELD_NUMBER = 7; + @SuppressWarnings("serial") + private volatile java.lang.Object ip_ = ""; + /** + * string ip = 7 [json_name = "ip"]; + * @return The ip. + */ + @java.lang.Override + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } + } + /** + * string ip = 7 [json_name = "ip"]; + * @return The bytes for ip. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int COUNTRY_FIELD_NUMBER = 8; + @SuppressWarnings("serial") + private volatile java.lang.Object country_ = ""; + /** + * string country = 8 [json_name = "country"]; + * @return The country. + */ + @java.lang.Override + public java.lang.String getCountry() { + java.lang.Object ref = country_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + country_ = s; + return s; + } + } + /** + * string country = 8 [json_name = "country"]; + * @return The bytes for country. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getCountryBytes() { + java.lang.Object ref = country_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + country_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TIMESTAMP_FIELD_NUMBER = 9; + private long timestamp_ = 0L; + /** + * int64 timestamp = 9 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(id_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, id_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 3, filterId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 4, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 5, method_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(orig_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 6, orig_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 7, ip_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(country_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 8, country_); + } + if (timestamp_ != 0L) { + output.writeInt64(9, timestamp_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(id_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, id_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(2, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(3, filterId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(4, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(5, method_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(orig_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(6, orig_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(7, ip_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(country_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(8, country_); + } + if (timestamp_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(9, timestamp_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest other = (com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest) obj; + + if (!getId() + .equals(other.getId())) return false; + if (!getActor() + .equals(other.getActor())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (!getUrl() + .equals(other.getUrl())) return false; + if (!getMethod() + .equals(other.getMethod())) return false; + if (!getOrig() + .equals(other.getOrig())) return false; + if (!getIp() + .equals(other.getIp())) return false; + if (!getCountry() + .equals(other.getCountry())) return false; + if (getTimestamp() + != other.getTimestamp()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ID_FIELD_NUMBER; + hash = (53 * hash) + getId().hashCode(); + hash = (37 * hash) + ACTOR_FIELD_NUMBER; + hash = (53 * hash) + getActor().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (37 * hash) + URL_FIELD_NUMBER; + hash = (53 * hash) + getUrl().hashCode(); + hash = (37 * hash) + METHOD_FIELD_NUMBER; + hash = (53 * hash) + getMethod().hashCode(); + hash = (37 * hash) + ORIG_FIELD_NUMBER; + hash = (53 * hash) + getOrig().hashCode(); + hash = (37 * hash) + IP_FIELD_NUMBER; + hash = (53 * hash) + getIp().hashCode(); + hash = (37 * hash) + COUNTRY_FIELD_NUMBER; + hash = (53 * hash) + getCountry().hashCode(); + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getTimestamp()); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.dashboard_service.v1.MaliciousRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.MaliciousRequest) + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + id_ = ""; + actor_ = ""; + filterId_ = ""; + url_ = ""; + method_ = ""; + orig_ = ""; + ip_ = ""; + country_ = ""; + timestamp_ = 0L; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest build() { + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest buildPartial() { + com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest result = new com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.id_ = id_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.actor_ = actor_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.filterId_ = filterId_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.url_ = url_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.method_ = method_; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.orig_ = orig_; + } + if (((from_bitField0_ & 0x00000040) != 0)) { + result.ip_ = ip_; + } + if (((from_bitField0_ & 0x00000080) != 0)) { + result.country_ = country_; + } + if (((from_bitField0_ & 0x00000100) != 0)) { + result.timestamp_ = timestamp_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest) { + return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest other) { + if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.getDefaultInstance()) return this; + if (!other.getId().isEmpty()) { + id_ = other.id_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getActor().isEmpty()) { + actor_ = other.actor_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (!other.getUrl().isEmpty()) { + url_ = other.url_; + bitField0_ |= 0x00000008; + onChanged(); + } + if (!other.getMethod().isEmpty()) { + method_ = other.method_; + bitField0_ |= 0x00000010; + onChanged(); + } + if (!other.getOrig().isEmpty()) { + orig_ = other.orig_; + bitField0_ |= 0x00000020; + onChanged(); + } + if (!other.getIp().isEmpty()) { + ip_ = other.ip_; + bitField0_ |= 0x00000040; + onChanged(); + } + if (!other.getCountry().isEmpty()) { + country_ = other.country_; + bitField0_ |= 0x00000080; + onChanged(); + } + if (other.getTimestamp() != 0L) { + setTimestamp(other.getTimestamp()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + id_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + actor_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + url_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 42: { + method_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000010; + break; + } // case 42 + case 50: { + orig_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000020; + break; + } // case 50 + case 58: { + ip_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000040; + break; + } // case 58 + case 66: { + country_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000080; + break; + } // case 66 + case 72: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000100; + break; + } // case 72 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object id_ = ""; + /** + * string id = 1 [json_name = "id"]; + * @return The id. + */ + public java.lang.String getId() { + java.lang.Object ref = id_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + id_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string id = 1 [json_name = "id"]; + * @return The bytes for id. + */ + public com.google.protobuf.ByteString + getIdBytes() { + java.lang.Object ref = id_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + id_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string id = 1 [json_name = "id"]; + * @param value The id to set. + * @return This builder for chaining. + */ + public Builder setId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + id_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string id = 1 [json_name = "id"]; + * @return This builder for chaining. + */ + public Builder clearId() { + id_ = getDefaultInstance().getId(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string id = 1 [json_name = "id"]; + * @param value The bytes for id to set. + * @return This builder for chaining. + */ + public Builder setIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + id_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object actor_ = ""; + /** + * string actor = 2 [json_name = "actor"]; + * @return The actor. + */ + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string actor = 2 [json_name = "actor"]; + * @return The bytes for actor. + */ + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string actor = 2 [json_name = "actor"]; + * @param value The actor to set. + * @return This builder for chaining. + */ + public Builder setActor( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + actor_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string actor = 2 [json_name = "actor"]; + * @return This builder for chaining. + */ + public Builder clearActor() { + actor_ = getDefaultInstance().getActor(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string actor = 2 [json_name = "actor"]; + * @param value The bytes for actor to set. + * @return This builder for chaining. + */ + public Builder setActorBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + actor_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 3 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 3 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 3 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string filter_id = 3 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string filter_id = 3 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private java.lang.Object url_ = ""; + /** + * string url = 4 [json_name = "url"]; + * @return The url. + */ + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string url = 4 [json_name = "url"]; + * @return The bytes for url. + */ + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string url = 4 [json_name = "url"]; + * @param value The url to set. + * @return This builder for chaining. + */ + public Builder setUrl( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + url_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * string url = 4 [json_name = "url"]; + * @return This builder for chaining. + */ + public Builder clearUrl() { + url_ = getDefaultInstance().getUrl(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * string url = 4 [json_name = "url"]; + * @param value The bytes for url to set. + * @return This builder for chaining. + */ + public Builder setUrlBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + url_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + + private java.lang.Object method_ = ""; + /** + * string method = 5 [json_name = "method"]; + * @return The method. + */ + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string method = 5 [json_name = "method"]; + * @return The bytes for method. + */ + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string method = 5 [json_name = "method"]; + * @param value The method to set. + * @return This builder for chaining. + */ + public Builder setMethod( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + method_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * string method = 5 [json_name = "method"]; + * @return This builder for chaining. + */ + public Builder clearMethod() { + method_ = getDefaultInstance().getMethod(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + return this; + } + /** + * string method = 5 [json_name = "method"]; + * @param value The bytes for method to set. + * @return This builder for chaining. + */ + public Builder setMethodBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + method_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + + private java.lang.Object orig_ = ""; + /** + * string orig = 6 [json_name = "orig"]; + * @return The orig. + */ + public java.lang.String getOrig() { + java.lang.Object ref = orig_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + orig_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string orig = 6 [json_name = "orig"]; + * @return The bytes for orig. + */ + public com.google.protobuf.ByteString + getOrigBytes() { + java.lang.Object ref = orig_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + orig_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string orig = 6 [json_name = "orig"]; + * @param value The orig to set. + * @return This builder for chaining. + */ + public Builder setOrig( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + orig_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * string orig = 6 [json_name = "orig"]; + * @return This builder for chaining. + */ + public Builder clearOrig() { + orig_ = getDefaultInstance().getOrig(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + return this; + } + /** + * string orig = 6 [json_name = "orig"]; + * @param value The bytes for orig to set. + * @return This builder for chaining. + */ + public Builder setOrigBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + orig_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + + private java.lang.Object ip_ = ""; + /** + * string ip = 7 [json_name = "ip"]; + * @return The ip. + */ + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string ip = 7 [json_name = "ip"]; + * @return The bytes for ip. + */ + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string ip = 7 [json_name = "ip"]; + * @param value The ip to set. + * @return This builder for chaining. + */ + public Builder setIp( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ip_ = value; + bitField0_ |= 0x00000040; + onChanged(); + return this; + } + /** + * string ip = 7 [json_name = "ip"]; + * @return This builder for chaining. + */ + public Builder clearIp() { + ip_ = getDefaultInstance().getIp(); + bitField0_ = (bitField0_ & ~0x00000040); + onChanged(); + return this; + } + /** + * string ip = 7 [json_name = "ip"]; + * @param value The bytes for ip to set. + * @return This builder for chaining. + */ + public Builder setIpBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ip_ = value; + bitField0_ |= 0x00000040; + onChanged(); + return this; + } + + private java.lang.Object country_ = ""; + /** + * string country = 8 [json_name = "country"]; + * @return The country. + */ + public java.lang.String getCountry() { + java.lang.Object ref = country_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + country_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string country = 8 [json_name = "country"]; + * @return The bytes for country. + */ + public com.google.protobuf.ByteString + getCountryBytes() { + java.lang.Object ref = country_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + country_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string country = 8 [json_name = "country"]; + * @param value The country to set. + * @return This builder for chaining. + */ + public Builder setCountry( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + country_ = value; + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + /** + * string country = 8 [json_name = "country"]; + * @return This builder for chaining. + */ + public Builder clearCountry() { + country_ = getDefaultInstance().getCountry(); + bitField0_ = (bitField0_ & ~0x00000080); + onChanged(); + return this; + } + /** + * string country = 8 [json_name = "country"]; + * @param value The bytes for country to set. + * @return This builder for chaining. + */ + public Builder setCountryBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + country_ = value; + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + + private long timestamp_ ; + /** + * int64 timestamp = 9 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + /** + * int64 timestamp = 9 [json_name = "timestamp"]; + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + + timestamp_ = value; + bitField0_ |= 0x00000100; + onChanged(); + return this; + } + /** + * int64 timestamp = 9 [json_name = "timestamp"]; + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000100); + timestamp_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.MaliciousRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.MaliciousRequest) + private static final com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest(); + } + + public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MaliciousRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequestOrBuilder.java new file mode 100644 index 0000000000..943f4f2e0e --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequestOrBuilder.java @@ -0,0 +1,113 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +public interface MaliciousRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.MaliciousRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * string id = 1 [json_name = "id"]; + * @return The id. + */ + java.lang.String getId(); + /** + * string id = 1 [json_name = "id"]; + * @return The bytes for id. + */ + com.google.protobuf.ByteString + getIdBytes(); + + /** + * string actor = 2 [json_name = "actor"]; + * @return The actor. + */ + java.lang.String getActor(); + /** + * string actor = 2 [json_name = "actor"]; + * @return The bytes for actor. + */ + com.google.protobuf.ByteString + getActorBytes(); + + /** + * string filter_id = 3 [json_name = "filterId"]; + * @return The filterId. + */ + java.lang.String getFilterId(); + /** + * string filter_id = 3 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + com.google.protobuf.ByteString + getFilterIdBytes(); + + /** + * string url = 4 [json_name = "url"]; + * @return The url. + */ + java.lang.String getUrl(); + /** + * string url = 4 [json_name = "url"]; + * @return The bytes for url. + */ + com.google.protobuf.ByteString + getUrlBytes(); + + /** + * string method = 5 [json_name = "method"]; + * @return The method. + */ + java.lang.String getMethod(); + /** + * string method = 5 [json_name = "method"]; + * @return The bytes for method. + */ + com.google.protobuf.ByteString + getMethodBytes(); + + /** + * string orig = 6 [json_name = "orig"]; + * @return The orig. + */ + java.lang.String getOrig(); + /** + * string orig = 6 [json_name = "orig"]; + * @return The bytes for orig. + */ + com.google.protobuf.ByteString + getOrigBytes(); + + /** + * string ip = 7 [json_name = "ip"]; + * @return The ip. + */ + java.lang.String getIp(); + /** + * string ip = 7 [json_name = "ip"]; + * @return The bytes for ip. + */ + com.google.protobuf.ByteString + getIpBytes(); + + /** + * string country = 8 [json_name = "country"]; + * @return The country. + */ + java.lang.String getCountry(); + /** + * string country = 8 [json_name = "country"]; + * @return The bytes for country. + */ + com.google.protobuf.ByteString + getCountryBytes(); + + /** + * int64 timestamp = 9 [json_name = "timestamp"]; + * @return The timestamp. + */ + long getTimestamp(); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ServiceProto.java new file mode 100644 index 0000000000..4fec5af487 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ServiceProto.java @@ -0,0 +1,136 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/dashboard_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.dashboard_service.v1; + +public final class ServiceProto { + private ServiceProto() {} + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + ServiceProto.class.getName()); + } + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n getRecordMaliciousEventMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "RecordMaliciousEvent", + requestType = com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.class, + responseType = com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getRecordMaliciousEventMethod() { + io.grpc.MethodDescriptor getRecordMaliciousEventMethod; + if ((getRecordMaliciousEventMethod = MaliciousEventServiceGrpc.getRecordMaliciousEventMethod) == null) { + synchronized (MaliciousEventServiceGrpc.class) { + if ((getRecordMaliciousEventMethod = MaliciousEventServiceGrpc.getRecordMaliciousEventMethod) == null) { + MaliciousEventServiceGrpc.getRecordMaliciousEventMethod = getRecordMaliciousEventMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "RecordMaliciousEvent")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.getDefaultInstance())) + .setSchemaDescriptor(new MaliciousEventServiceMethodDescriptorSupplier("RecordMaliciousEvent")) + .build(); + } + } + } + return getRecordMaliciousEventMethod; + } + + /** + * Creates a new async stub that supports all call types for the service + */ + public static MaliciousEventServiceStub newStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public MaliciousEventServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousEventServiceStub(channel, callOptions); + } + }; + return MaliciousEventServiceStub.newStub(factory, channel); + } + + /** + * Creates a new blocking-style stub that supports unary and streaming output calls on the service + */ + public static MaliciousEventServiceBlockingStub newBlockingStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public MaliciousEventServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousEventServiceBlockingStub(channel, callOptions); + } + }; + return MaliciousEventServiceBlockingStub.newStub(factory, channel); + } + + /** + * Creates a new ListenableFuture-style stub that supports unary calls on the service + */ + public static MaliciousEventServiceFutureStub newFutureStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public MaliciousEventServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousEventServiceFutureStub(channel, callOptions); + } + }; + return MaliciousEventServiceFutureStub.newStub(factory, channel); + } + + /** + */ + public interface AsyncService { + + /** + */ + default void recordMaliciousEvent(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getRecordMaliciousEventMethod(), responseObserver); + } + } + + /** + * Base class for the server implementation of the service MaliciousEventService. + */ + public static abstract class MaliciousEventServiceImplBase + implements io.grpc.BindableService, AsyncService { + + @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { + return MaliciousEventServiceGrpc.bindService(this); + } + } + + /** + * A stub to allow clients to do asynchronous rpc calls to service MaliciousEventService. + */ + public static final class MaliciousEventServiceStub + extends io.grpc.stub.AbstractAsyncStub { + private MaliciousEventServiceStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected MaliciousEventServiceStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousEventServiceStub(channel, callOptions); + } + + /** + */ + public void recordMaliciousEvent(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getRecordMaliciousEventMethod(), getCallOptions()), request, responseObserver); + } + } + + /** + * A stub to allow clients to do synchronous rpc calls to service MaliciousEventService. + */ + public static final class MaliciousEventServiceBlockingStub + extends io.grpc.stub.AbstractBlockingStub { + private MaliciousEventServiceBlockingStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected MaliciousEventServiceBlockingStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousEventServiceBlockingStub(channel, callOptions); + } + + /** + */ + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse recordMaliciousEvent(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getRecordMaliciousEventMethod(), getCallOptions(), request); + } + } + + /** + * A stub to allow clients to do ListenableFuture-style rpc calls to service MaliciousEventService. + */ + public static final class MaliciousEventServiceFutureStub + extends io.grpc.stub.AbstractFutureStub { + private MaliciousEventServiceFutureStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected MaliciousEventServiceFutureStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousEventServiceFutureStub(channel, callOptions); + } + + /** + */ + public com.google.common.util.concurrent.ListenableFuture recordMaliciousEvent( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getRecordMaliciousEventMethod(), getCallOptions()), request); + } + } + + private static final int METHODID_RECORD_MALICIOUS_EVENT = 0; + + private static final class MethodHandlers implements + io.grpc.stub.ServerCalls.UnaryMethod, + io.grpc.stub.ServerCalls.ServerStreamingMethod, + io.grpc.stub.ServerCalls.ClientStreamingMethod, + io.grpc.stub.ServerCalls.BidiStreamingMethod { + private final AsyncService serviceImpl; + private final int methodId; + + MethodHandlers(AsyncService serviceImpl, int methodId) { + this.serviceImpl = serviceImpl; + this.methodId = methodId; + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + case METHODID_RECORD_MALICIOUS_EVENT: + serviceImpl.recordMaliciousEvent((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + default: + throw new AssertionError(); + } + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public io.grpc.stub.StreamObserver invoke( + io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + default: + throw new AssertionError(); + } + } + } + + public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { + return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) + .addMethod( + getRecordMaliciousEventMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest, + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse>( + service, METHODID_RECORD_MALICIOUS_EVENT))) + .build(); + } + + private static abstract class MaliciousEventServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { + MaliciousEventServiceBaseDescriptorSupplier() {} + + @java.lang.Override + public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.getDescriptor(); + } + + @java.lang.Override + public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { + return getFileDescriptor().findServiceByName("MaliciousEventService"); + } + } + + private static final class MaliciousEventServiceFileDescriptorSupplier + extends MaliciousEventServiceBaseDescriptorSupplier { + MaliciousEventServiceFileDescriptorSupplier() {} + } + + private static final class MaliciousEventServiceMethodDescriptorSupplier + extends MaliciousEventServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { + private final java.lang.String methodName; + + MaliciousEventServiceMethodDescriptorSupplier(java.lang.String methodName) { + this.methodName = methodName; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { + return getServiceDescriptor().findMethodByName(methodName); + } + } + + private static volatile io.grpc.ServiceDescriptor serviceDescriptor; + + public static io.grpc.ServiceDescriptor getServiceDescriptor() { + io.grpc.ServiceDescriptor result = serviceDescriptor; + if (result == null) { + synchronized (MaliciousEventServiceGrpc.class) { + result = serviceDescriptor; + if (result == null) { + serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) + .setSchemaDescriptor(new MaliciousEventServiceFileDescriptorSupplier()) + .addMethod(getRecordMaliciousEventMethod()) + .build(); + } + } + } + return result; + } +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequest.java new file mode 100644 index 0000000000..08df258112 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequest.java @@ -0,0 +1,913 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +/** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest} + */ +public final class RecordMaliciousEventRequest extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) + RecordMaliciousEventRequestOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + RecordMaliciousEventRequest.class.getName()); + } + // Use RecordMaliciousEventRequest.newBuilder() to construct. + private RecordMaliciousEventRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private RecordMaliciousEventRequest() { + sampleRequests_ = java.util.Collections.emptyList(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.Builder.class); + } + + private int bitField0_; + public static final int MALICIOUS_EVENT_FIELD_NUMBER = 1; + private com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent maliciousEvent_; + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + * @return Whether the maliciousEvent field is set. + */ + @java.lang.Override + public boolean hasMaliciousEvent() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + * @return The maliciousEvent. + */ + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getMaliciousEvent() { + return maliciousEvent_ == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance() : maliciousEvent_; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder getMaliciousEventOrBuilder() { + return maliciousEvent_ == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance() : maliciousEvent_; + } + + public static final int SAMPLE_REQUESTS_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private java.util.List sampleRequests_; + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + @java.lang.Override + public java.util.List getSampleRequestsList() { + return sampleRequests_; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + @java.lang.Override + public java.util.List + getSampleRequestsOrBuilderList() { + return sampleRequests_; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + @java.lang.Override + public int getSampleRequestsCount() { + return sampleRequests_.size(); + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getSampleRequests(int index) { + return sampleRequests_.get(index); + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder getSampleRequestsOrBuilder( + int index) { + return sampleRequests_.get(index); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getMaliciousEvent()); + } + for (int i = 0; i < sampleRequests_.size(); i++) { + output.writeMessage(2, sampleRequests_.get(i)); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getMaliciousEvent()); + } + for (int i = 0; i < sampleRequests_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, sampleRequests_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) obj; + + if (hasMaliciousEvent() != other.hasMaliciousEvent()) return false; + if (hasMaliciousEvent()) { + if (!getMaliciousEvent() + .equals(other.getMaliciousEvent())) return false; + } + if (!getSampleRequestsList() + .equals(other.getSampleRequestsList())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasMaliciousEvent()) { + hash = (37 * hash) + MALICIOUS_EVENT_FIELD_NUMBER; + hash = (53 * hash) + getMaliciousEvent().hashCode(); + } + if (getSampleRequestsCount() > 0) { + hash = (37 * hash) + SAMPLE_REQUESTS_FIELD_NUMBER; + hash = (53 * hash) + getSampleRequestsList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage + .alwaysUseFieldBuilders) { + getMaliciousEventFieldBuilder(); + getSampleRequestsFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + maliciousEvent_ = null; + if (maliciousEventBuilder_ != null) { + maliciousEventBuilder_.dispose(); + maliciousEventBuilder_ = null; + } + if (sampleRequestsBuilder_ == null) { + sampleRequests_ = java.util.Collections.emptyList(); + } else { + sampleRequests_ = null; + sampleRequestsBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest build() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest buildPartial() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest(this); + buildPartialRepeatedFields(result); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartialRepeatedFields(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest result) { + if (sampleRequestsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0)) { + sampleRequests_ = java.util.Collections.unmodifiableList(sampleRequests_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.sampleRequests_ = sampleRequests_; + } else { + result.sampleRequests_ = sampleRequestsBuilder_.build(); + } + } + + private void buildPartial0(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.maliciousEvent_ = maliciousEventBuilder_ == null + ? maliciousEvent_ + : maliciousEventBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) { + return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest other) { + if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.getDefaultInstance()) return this; + if (other.hasMaliciousEvent()) { + mergeMaliciousEvent(other.getMaliciousEvent()); + } + if (sampleRequestsBuilder_ == null) { + if (!other.sampleRequests_.isEmpty()) { + if (sampleRequests_.isEmpty()) { + sampleRequests_ = other.sampleRequests_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureSampleRequestsIsMutable(); + sampleRequests_.addAll(other.sampleRequests_); + } + onChanged(); + } + } else { + if (!other.sampleRequests_.isEmpty()) { + if (sampleRequestsBuilder_.isEmpty()) { + sampleRequestsBuilder_.dispose(); + sampleRequestsBuilder_ = null; + sampleRequests_ = other.sampleRequests_; + bitField0_ = (bitField0_ & ~0x00000002); + sampleRequestsBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getSampleRequestsFieldBuilder() : null; + } else { + sampleRequestsBuilder_.addAllMessages(other.sampleRequests_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getMaliciousEventFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest m = + input.readMessage( + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.parser(), + extensionRegistry); + if (sampleRequestsBuilder_ == null) { + ensureSampleRequestsIsMutable(); + sampleRequests_.add(m); + } else { + sampleRequestsBuilder_.addMessage(m); + } + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent maliciousEvent_; + private com.google.protobuf.SingleFieldBuilder< + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder> maliciousEventBuilder_; + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + * @return Whether the maliciousEvent field is set. + */ + public boolean hasMaliciousEvent() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + * @return The maliciousEvent. + */ + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getMaliciousEvent() { + if (maliciousEventBuilder_ == null) { + return maliciousEvent_ == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance() : maliciousEvent_; + } else { + return maliciousEventBuilder_.getMessage(); + } + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + public Builder setMaliciousEvent(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent value) { + if (maliciousEventBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + maliciousEvent_ = value; + } else { + maliciousEventBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + public Builder setMaliciousEvent( + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder builderForValue) { + if (maliciousEventBuilder_ == null) { + maliciousEvent_ = builderForValue.build(); + } else { + maliciousEventBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + public Builder mergeMaliciousEvent(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent value) { + if (maliciousEventBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + maliciousEvent_ != null && + maliciousEvent_ != com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance()) { + getMaliciousEventBuilder().mergeFrom(value); + } else { + maliciousEvent_ = value; + } + } else { + maliciousEventBuilder_.mergeFrom(value); + } + if (maliciousEvent_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + public Builder clearMaliciousEvent() { + bitField0_ = (bitField0_ & ~0x00000001); + maliciousEvent_ = null; + if (maliciousEventBuilder_ != null) { + maliciousEventBuilder_.dispose(); + maliciousEventBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder getMaliciousEventBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getMaliciousEventFieldBuilder().getBuilder(); + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder getMaliciousEventOrBuilder() { + if (maliciousEventBuilder_ != null) { + return maliciousEventBuilder_.getMessageOrBuilder(); + } else { + return maliciousEvent_ == null ? + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance() : maliciousEvent_; + } + } + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + private com.google.protobuf.SingleFieldBuilder< + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder> + getMaliciousEventFieldBuilder() { + if (maliciousEventBuilder_ == null) { + maliciousEventBuilder_ = new com.google.protobuf.SingleFieldBuilder< + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder>( + getMaliciousEvent(), + getParentForChildren(), + isClean()); + maliciousEvent_ = null; + } + return maliciousEventBuilder_; + } + + private java.util.List sampleRequests_ = + java.util.Collections.emptyList(); + private void ensureSampleRequestsIsMutable() { + if (!((bitField0_ & 0x00000002) != 0)) { + sampleRequests_ = new java.util.ArrayList(sampleRequests_); + bitField0_ |= 0x00000002; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder> sampleRequestsBuilder_; + + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public java.util.List getSampleRequestsList() { + if (sampleRequestsBuilder_ == null) { + return java.util.Collections.unmodifiableList(sampleRequests_); + } else { + return sampleRequestsBuilder_.getMessageList(); + } + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public int getSampleRequestsCount() { + if (sampleRequestsBuilder_ == null) { + return sampleRequests_.size(); + } else { + return sampleRequestsBuilder_.getCount(); + } + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getSampleRequests(int index) { + if (sampleRequestsBuilder_ == null) { + return sampleRequests_.get(index); + } else { + return sampleRequestsBuilder_.getMessage(index); + } + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder setSampleRequests( + int index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest value) { + if (sampleRequestsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleRequestsIsMutable(); + sampleRequests_.set(index, value); + onChanged(); + } else { + sampleRequestsBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder setSampleRequests( + int index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder builderForValue) { + if (sampleRequestsBuilder_ == null) { + ensureSampleRequestsIsMutable(); + sampleRequests_.set(index, builderForValue.build()); + onChanged(); + } else { + sampleRequestsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder addSampleRequests(com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest value) { + if (sampleRequestsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleRequestsIsMutable(); + sampleRequests_.add(value); + onChanged(); + } else { + sampleRequestsBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder addSampleRequests( + int index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest value) { + if (sampleRequestsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleRequestsIsMutable(); + sampleRequests_.add(index, value); + onChanged(); + } else { + sampleRequestsBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder addSampleRequests( + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder builderForValue) { + if (sampleRequestsBuilder_ == null) { + ensureSampleRequestsIsMutable(); + sampleRequests_.add(builderForValue.build()); + onChanged(); + } else { + sampleRequestsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder addSampleRequests( + int index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder builderForValue) { + if (sampleRequestsBuilder_ == null) { + ensureSampleRequestsIsMutable(); + sampleRequests_.add(index, builderForValue.build()); + onChanged(); + } else { + sampleRequestsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder addAllSampleRequests( + java.lang.Iterable values) { + if (sampleRequestsBuilder_ == null) { + ensureSampleRequestsIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, sampleRequests_); + onChanged(); + } else { + sampleRequestsBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder clearSampleRequests() { + if (sampleRequestsBuilder_ == null) { + sampleRequests_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + } else { + sampleRequestsBuilder_.clear(); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public Builder removeSampleRequests(int index) { + if (sampleRequestsBuilder_ == null) { + ensureSampleRequestsIsMutable(); + sampleRequests_.remove(index); + onChanged(); + } else { + sampleRequestsBuilder_.remove(index); + } + return this; + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder getSampleRequestsBuilder( + int index) { + return getSampleRequestsFieldBuilder().getBuilder(index); + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder getSampleRequestsOrBuilder( + int index) { + if (sampleRequestsBuilder_ == null) { + return sampleRequests_.get(index); } else { + return sampleRequestsBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public java.util.List + getSampleRequestsOrBuilderList() { + if (sampleRequestsBuilder_ != null) { + return sampleRequestsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(sampleRequests_); + } + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder addSampleRequestsBuilder() { + return getSampleRequestsFieldBuilder().addBuilder( + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.getDefaultInstance()); + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder addSampleRequestsBuilder( + int index) { + return getSampleRequestsFieldBuilder().addBuilder( + index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.getDefaultInstance()); + } + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + public java.util.List + getSampleRequestsBuilderList() { + return getSampleRequestsFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder> + getSampleRequestsFieldBuilder() { + if (sampleRequestsBuilder_ == null) { + sampleRequestsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder>( + sampleRequests_, + ((bitField0_ & 0x00000002) != 0), + getParentForChildren(), + isClean()); + sampleRequests_ = null; + } + return sampleRequestsBuilder_; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) + private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest(); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RecordMaliciousEventRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequestOrBuilder.java new file mode 100644 index 0000000000..b36835dcd6 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequestOrBuilder.java @@ -0,0 +1,50 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +public interface RecordMaliciousEventRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + * @return Whether the maliciousEvent field is set. + */ + boolean hasMaliciousEvent(); + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + * @return The maliciousEvent. + */ + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getMaliciousEvent(); + /** + * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; + */ + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder getMaliciousEventOrBuilder(); + + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + java.util.List + getSampleRequestsList(); + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getSampleRequests(int index); + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + int getSampleRequestsCount(); + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + java.util.List + getSampleRequestsOrBuilderList(); + /** + * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; + */ + com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder getSampleRequestsOrBuilder( + int index); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponse.java new file mode 100644 index 0000000000..5e58989b1a --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponse.java @@ -0,0 +1,358 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +/** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse} + */ +public final class RecordMaliciousEventResponse extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) + RecordMaliciousEventResponseOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + RecordMaliciousEventResponse.class.getName()); + } + // Use RecordMaliciousEventResponse.newBuilder() to construct. + private RecordMaliciousEventResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private RecordMaliciousEventResponse() { + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.Builder.class); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) obj; + + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse build() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse buildPartial() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) { + return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse other) { + if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) + private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse(); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RecordMaliciousEventResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponseOrBuilder.java new file mode 100644 index 0000000000..b9dc97a1dc --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponseOrBuilder.java @@ -0,0 +1,11 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +public interface RecordMaliciousEventResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) + com.google.protobuf.MessageOrBuilder { +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ServiceProto.java new file mode 100644 index 0000000000..a97ab01ca9 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ServiceProto.java @@ -0,0 +1,99 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +public final class ServiceProto { + private ServiceProto() {} + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + ServiceProto.class.getName()); + } + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\nBthreat_protection/service/malicious_al" + + "ert_service/v1/service.proto\0224threat_pro" + + "tection.service.malicious_alert_service." + + "v1\032:threat_protection/message/malicious_" + + "event/v1/message.proto\0329threat_protectio" + + "n/message/sample_request/v1/message.prot" + + "o\"\036\n\034RecordMaliciousEventResponse\"\362\001\n\033Re" + + "cordMaliciousEventRequest\022e\n\017malicious_e" + + "vent\030\001 \001(\0132<.threat_protection.message.m" + + "alicious_event.v1.MaliciousEventR\016malici" + + "ousEvent\022l\n\017sample_requests\030\002 \003(\0132C.thre" + + "at_protection.message.sample_request.v1." + + "SampleMaliciousRequestR\016sampleRequests2\331" + + "\001\n\025MaliciousEventService\022\277\001\n\024RecordMalic" + + "iousEvent\022Q.threat_protection.service.ma" + + "licious_alert_service.v1.RecordMalicious" + + "EventRequest\032R.threat_protection.service" + + ".malicious_alert_service.v1.RecordMalici" + + "ousEventResponse\"\000B\272\002\nCcom.akto.proto.th" + + "reat_protection.service.malicious_alert_" + + "service.v1B\014ServiceProtoP\001\242\002\003TSM\252\0021Threa" + + "tProtection.Service.MaliciousAlertServic" + + "e.V1\312\0021ThreatProtection\\Service\\Maliciou" + + "sAlertService\\V1\342\002=ThreatProtection\\Serv" + + "ice\\MaliciousAlertService\\V1\\GPBMetadata" + + "\352\0024ThreatProtection::Service::MaliciousA" + + "lertService::V1b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.getDescriptor(), + com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.getDescriptor(), + }); + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor, + new java.lang.String[] { }); + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor, + new java.lang.String[] { "MaliciousEvent", "SampleRequests", }); + descriptor.resolveAllFeaturesImmutable(); + com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.getDescriptor(); + com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} From fb9bc6919b8d043fab339951a411b1454ced2dfe Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Mon, 18 Nov 2024 13:35:22 +0530 Subject: [PATCH 16/73] refactor code --- .../malicious_event/v1/MaliciousEvent.java | 1232 +++++++++++++++++ .../v1/MaliciousEventOrBuilder.java | 62 + .../malicious_event/v1/MessageProto.java | 29 + .../message/smart_event/v1/MessageProto.java | 70 + .../message/smart_event/v1/SmartEvent.java | 840 +++++++++++ .../smart_event/v1/SmartEventOrBuilder.java | 53 + .../v1/ConsumerServiceProto.java | 107 ++ .../v1/MaliciousAlertServiceGrpc.java | 293 ++++ .../v1/RecordAlertRequest.java | 1124 +++++++++++++++ .../v1/RecordAlertRequestOrBuilder.java | 71 + .../v1/RecordAlertResponse.java | 358 +++++ .../v1/RecordAlertResponseOrBuilder.java | 11 + .../v1/SampleMaliciousEvent.java | 1042 ++++++++++++++ .../v1/SampleMaliciousEventOrBuilder.java | 71 + 14 files changed, 5363 insertions(+) create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/MessageProto.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEvent.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEventOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ConsumerServiceProto.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousAlertServiceGrpc.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponse.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponseOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEvent.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEventOrBuilder.java diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java index 362daeb9d8..bc57008c58 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java @@ -29,6 +29,7 @@ private MaliciousEvent(com.google.protobuf.GeneratedMessage.Builder builder) private MaliciousEvent() { actor_ = ""; filterId_ = ""; +<<<<<<< HEAD latestApiIp_ = ""; latestApiEndpoint_ = ""; latestApiMethod_ = ""; @@ -1477,6 +1478,1237 @@ public Builder setEventType(com.akto.proto.threat_protection.message.malicious_e public Builder clearEventType() { bitField0_ = (bitField0_ & ~0x00000100); eventType_ = 0; +||||||| parent of 36e13656a (refactor code) +======= + ip_ = ""; + url_ = ""; + method_ = ""; + payload_ = ""; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.class, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder.class); + } + + public static final int ACTOR_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + @java.lang.Override + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int IP_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object ip_ = ""; + /** + * string ip = 3 [json_name = "ip"]; + * @return The ip. + */ + @java.lang.Override + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } + } + /** + * string ip = 3 [json_name = "ip"]; + * @return The bytes for ip. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TIMESTAMP_FIELD_NUMBER = 4; + private long timestamp_ = 0L; + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + + public static final int URL_FIELD_NUMBER = 5; + @SuppressWarnings("serial") + private volatile java.lang.Object url_ = ""; + /** + * string url = 5 [json_name = "url"]; + * @return The url. + */ + @java.lang.Override + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } + } + /** + * string url = 5 [json_name = "url"]; + * @return The bytes for url. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int METHOD_FIELD_NUMBER = 6; + @SuppressWarnings("serial") + private volatile java.lang.Object method_ = ""; + /** + * string method = 6 [json_name = "method"]; + * @return The method. + */ + @java.lang.Override + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } + } + /** + * string method = 6 [json_name = "method"]; + * @return The bytes for method. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int API_COLLECTION_ID_FIELD_NUMBER = 7; + private int apiCollectionId_ = 0; + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + + public static final int PAYLOAD_FIELD_NUMBER = 8; + @SuppressWarnings("serial") + private volatile java.lang.Object payload_ = ""; + /** + * string payload = 8 [json_name = "payload"]; + * @return The payload. + */ + @java.lang.Override + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } + } + /** + * string payload = 8 [json_name = "payload"]; + * @return The bytes for payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 3, ip_); + } + if (timestamp_ != 0L) { + output.writeInt64(4, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 5, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 6, method_); + } + if (apiCollectionId_ != 0) { + output.writeInt32(7, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 8, payload_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(3, ip_); + } + if (timestamp_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(4, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(5, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(6, method_); + } + if (apiCollectionId_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(7, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(8, payload_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent other = (com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent) obj; + + if (!getActor() + .equals(other.getActor())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (!getIp() + .equals(other.getIp())) return false; + if (getTimestamp() + != other.getTimestamp()) return false; + if (!getUrl() + .equals(other.getUrl())) return false; + if (!getMethod() + .equals(other.getMethod())) return false; + if (getApiCollectionId() + != other.getApiCollectionId()) return false; + if (!getPayload() + .equals(other.getPayload())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACTOR_FIELD_NUMBER; + hash = (53 * hash) + getActor().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (37 * hash) + IP_FIELD_NUMBER; + hash = (53 * hash) + getIp().hashCode(); + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getTimestamp()); + hash = (37 * hash) + URL_FIELD_NUMBER; + hash = (53 * hash) + getUrl().hashCode(); + hash = (37 * hash) + METHOD_FIELD_NUMBER; + hash = (53 * hash) + getMethod().hashCode(); + hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; + hash = (53 * hash) + getApiCollectionId(); + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.message.malicious_event.v1.MaliciousEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.message.malicious_event.v1.MaliciousEvent) + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.class, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + actor_ = ""; + filterId_ = ""; + ip_ = ""; + timestamp_ = 0L; + url_ = ""; + method_ = ""; + apiCollectionId_ = 0; + payload_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getDefaultInstanceForType() { + return com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent build() { + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent buildPartial() { + com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result = new com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.actor_ = actor_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.filterId_ = filterId_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.ip_ = ip_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.timestamp_ = timestamp_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.url_ = url_; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.method_ = method_; + } + if (((from_bitField0_ & 0x00000040) != 0)) { + result.apiCollectionId_ = apiCollectionId_; + } + if (((from_bitField0_ & 0x00000080) != 0)) { + result.payload_ = payload_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent) { + return mergeFrom((com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent other) { + if (other == com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance()) return this; + if (!other.getActor().isEmpty()) { + actor_ = other.actor_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (!other.getIp().isEmpty()) { + ip_ = other.ip_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (other.getTimestamp() != 0L) { + setTimestamp(other.getTimestamp()); + } + if (!other.getUrl().isEmpty()) { + url_ = other.url_; + bitField0_ |= 0x00000010; + onChanged(); + } + if (!other.getMethod().isEmpty()) { + method_ = other.method_; + bitField0_ |= 0x00000020; + onChanged(); + } + if (other.getApiCollectionId() != 0) { + setApiCollectionId(other.getApiCollectionId()); + } + if (!other.getPayload().isEmpty()) { + payload_ = other.payload_; + bitField0_ |= 0x00000080; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + actor_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + ip_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 32: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000008; + break; + } // case 32 + case 42: { + url_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000010; + break; + } // case 42 + case 50: { + method_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000020; + break; + } // case 50 + case 56: { + apiCollectionId_ = input.readInt32(); + bitField0_ |= 0x00000040; + break; + } // case 56 + case 66: { + payload_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000080; + break; + } // case 66 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The actor to set. + * @return This builder for chaining. + */ + public Builder setActor( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @return This builder for chaining. + */ + public Builder clearActor() { + actor_ = getDefaultInstance().getActor(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The bytes for actor to set. + * @return This builder for chaining. + */ + public Builder setActorBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private java.lang.Object ip_ = ""; + /** + * string ip = 3 [json_name = "ip"]; + * @return The ip. + */ + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string ip = 3 [json_name = "ip"]; + * @return The bytes for ip. + */ + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string ip = 3 [json_name = "ip"]; + * @param value The ip to set. + * @return This builder for chaining. + */ + public Builder setIp( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ip_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string ip = 3 [json_name = "ip"]; + * @return This builder for chaining. + */ + public Builder clearIp() { + ip_ = getDefaultInstance().getIp(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string ip = 3 [json_name = "ip"]; + * @param value The bytes for ip to set. + * @return This builder for chaining. + */ + public Builder setIpBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ip_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private long timestamp_ ; + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + + timestamp_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000008); + timestamp_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object url_ = ""; + /** + * string url = 5 [json_name = "url"]; + * @return The url. + */ + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string url = 5 [json_name = "url"]; + * @return The bytes for url. + */ + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string url = 5 [json_name = "url"]; + * @param value The url to set. + * @return This builder for chaining. + */ + public Builder setUrl( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + url_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * string url = 5 [json_name = "url"]; + * @return This builder for chaining. + */ + public Builder clearUrl() { + url_ = getDefaultInstance().getUrl(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + return this; + } + /** + * string url = 5 [json_name = "url"]; + * @param value The bytes for url to set. + * @return This builder for chaining. + */ + public Builder setUrlBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + url_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + + private java.lang.Object method_ = ""; + /** + * string method = 6 [json_name = "method"]; + * @return The method. + */ + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string method = 6 [json_name = "method"]; + * @return The bytes for method. + */ + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string method = 6 [json_name = "method"]; + * @param value The method to set. + * @return This builder for chaining. + */ + public Builder setMethod( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + method_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * string method = 6 [json_name = "method"]; + * @return This builder for chaining. + */ + public Builder clearMethod() { + method_ = getDefaultInstance().getMethod(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + return this; + } + /** + * string method = 6 [json_name = "method"]; + * @param value The bytes for method to set. + * @return This builder for chaining. + */ + public Builder setMethodBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + method_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + + private int apiCollectionId_ ; + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @param value The apiCollectionId to set. + * @return This builder for chaining. + */ + public Builder setApiCollectionId(int value) { + + apiCollectionId_ = value; + bitField0_ |= 0x00000040; + onChanged(); + return this; + } + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @return This builder for chaining. + */ + public Builder clearApiCollectionId() { + bitField0_ = (bitField0_ & ~0x00000040); + apiCollectionId_ = 0; + onChanged(); + return this; + } + + private java.lang.Object payload_ = ""; + /** + * string payload = 8 [json_name = "payload"]; + * @return The payload. + */ + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string payload = 8 [json_name = "payload"]; + * @return The bytes for payload. + */ + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string payload = 8 [json_name = "payload"]; + * @param value The payload to set. + * @return This builder for chaining. + */ + public Builder setPayload( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + payload_ = value; + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + /** + * string payload = 8 [json_name = "payload"]; + * @return This builder for chaining. + */ + public Builder clearPayload() { + payload_ = getDefaultInstance().getPayload(); + bitField0_ = (bitField0_ & ~0x00000080); + onChanged(); + return this; + } + /** + * string payload = 8 [json_name = "payload"]; + * @param value The bytes for payload to set. + * @return This builder for chaining. + */ + public Builder setPayloadBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + payload_ = value; + bitField0_ |= 0x00000080; +>>>>>>> 36e13656a (refactor code) onChanged(); return this; } diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java index b37b92a7ec..cd9548b517 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java @@ -34,6 +34,7 @@ public interface MaliciousEventOrBuilder extends getFilterIdBytes(); /** +<<<<<<< HEAD * int64 detected_at = 3 [json_name = "detectedAt"]; * @return The detectedAt. */ @@ -103,4 +104,65 @@ public interface MaliciousEventOrBuilder extends * @return The eventType. */ com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType getEventType(); +||||||| parent of 36e13656a (refactor code) +======= + * string ip = 3 [json_name = "ip"]; + * @return The ip. + */ + java.lang.String getIp(); + /** + * string ip = 3 [json_name = "ip"]; + * @return The bytes for ip. + */ + com.google.protobuf.ByteString + getIpBytes(); + + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @return The timestamp. + */ + long getTimestamp(); + + /** + * string url = 5 [json_name = "url"]; + * @return The url. + */ + java.lang.String getUrl(); + /** + * string url = 5 [json_name = "url"]; + * @return The bytes for url. + */ + com.google.protobuf.ByteString + getUrlBytes(); + + /** + * string method = 6 [json_name = "method"]; + * @return The method. + */ + java.lang.String getMethod(); + /** + * string method = 6 [json_name = "method"]; + * @return The bytes for method. + */ + com.google.protobuf.ByteString + getMethodBytes(); + + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + int getApiCollectionId(); + + /** + * string payload = 8 [json_name = "payload"]; + * @return The payload. + */ + java.lang.String getPayload(); + /** + * string payload = 8 [json_name = "payload"]; + * @return The bytes for payload. + */ + com.google.protobuf.ByteString + getPayloadBytes(); +>>>>>>> 36e13656a (refactor code) } diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java index 5a1f6fb606..55a086f74c 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java @@ -41,6 +41,7 @@ public static void registerAllExtensions( java.lang.String[] descriptorData = { "\n:threat_protection/message/malicious_ev" + "ent/v1/message.proto\022,threat_protection." + +<<<<<<< HEAD "message.malicious_event.v1\"\215\004\n\016Malicious" + "Event\022\024\n\005actor\030\001 \001(\tR\005actor\022\033\n\tfilter_id" + "\030\002 \001(\tR\010filterId\022\037\n\013detected_at\030\003 \001(\003R\nd" + @@ -74,6 +75,34 @@ public static void registerAllExtensions( com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor, new java.lang.String[] { "Actor", "FilterId", "DetectedAt", "LatestApiIp", "LatestApiEndpoint", "LatestApiMethod", "LatestApiCollectionId", "LatestApiPayload", "EventType", }); +||||||| parent of 36e13656a (refactor code) +======= + "message.malicious_event.v1\"\341\001\n\016Malicious" + + "Event\022\024\n\005actor\030\001 \001(\tR\005actor\022\033\n\tfilter_id" + + "\030\002 \001(\tR\010filterId\022\016\n\002ip\030\003 \001(\tR\002ip\022\034\n\ttime" + + "stamp\030\004 \001(\003R\ttimestamp\022\020\n\003url\030\005 \001(\tR\003url" + + "\022\026\n\006method\030\006 \001(\tR\006method\022*\n\021api_collecti" + + "on_id\030\007 \001(\005R\017apiCollectionId\022\030\n\007payload\030" + + "\010 \001(\tR\007payloadB\226\002\n;com.akto.proto.threat" + + "_protection.message.malicious_event.v1B\014" + + "MessageProtoP\001\242\002\003TMM\252\002*ThreatProtection." + + "Message.MaliciousEvent.V1\312\002*ThreatProtec" + + "tion\\Message\\MaliciousEvent\\V1\342\0026ThreatP" + + "rotection\\Message\\MaliciousEvent\\V1\\GPBM" + + "etadata\352\002-ThreatProtection::Message::Mal" + + "iciousEvent::V1b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor, + new java.lang.String[] { "Actor", "FilterId", "Ip", "Timestamp", "Url", "Method", "ApiCollectionId", "Payload", }); +>>>>>>> 36e13656a (refactor code) descriptor.resolveAllFeaturesImmutable(); } diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/MessageProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/MessageProto.java new file mode 100644 index 0000000000..f64eb0cb39 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/MessageProto.java @@ -0,0 +1,70 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/smart_event/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.smart_event.v1; + +public final class MessageProto { + private MessageProto() {} + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + MessageProto.class.getName()); + } + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_message_smart_event_v1_SmartEvent_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n6threat_protection/message/smart_event/" + + "v1/message.proto\022(threat_protection.mess" + + "age.smart_event.v1\"y\n\nSmartEvent\022\024\n\005acto" + + "r\030\001 \001(\tR\005actor\022\033\n\tfilter_id\030\002 \001(\tR\010filte" + + "rId\022\037\n\013detected_at\030\003 \001(\003R\ndetectedAt\022\027\n\007" + + "rule_id\030\004 \001(\tR\006ruleIdB\202\002\n7com.akto.proto" + + ".threat_protection.message.smart_event.v" + + "1B\014MessageProtoP\001\242\002\003TMS\252\002&ThreatProtecti" + + "on.Message.SmartEvent.V1\312\002&ThreatProtect" + + "ion\\Message\\SmartEvent\\V1\342\0022ThreatProtec" + + "tion\\Message\\SmartEvent\\V1\\GPBMetadata\352\002" + + ")ThreatProtection::Message::SmartEvent::" + + "V1b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_threat_protection_message_smart_event_v1_SmartEvent_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor, + new java.lang.String[] { "Actor", "FilterId", "DetectedAt", "RuleId", }); + descriptor.resolveAllFeaturesImmutable(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEvent.java new file mode 100644 index 0000000000..51b3f4684b --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEvent.java @@ -0,0 +1,840 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/smart_event/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.smart_event.v1; + +/** + * Protobuf type {@code threat_protection.message.smart_event.v1.SmartEvent} + */ +public final class SmartEvent extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.message.smart_event.v1.SmartEvent) + SmartEventOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SmartEvent.class.getName()); + } + // Use SmartEvent.newBuilder() to construct. + private SmartEvent(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SmartEvent() { + actor_ = ""; + filterId_ = ""; + ruleId_ = ""; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.class, com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.Builder.class); + } + + public static final int ACTOR_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + @java.lang.Override + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DETECTED_AT_FIELD_NUMBER = 3; + private long detectedAt_ = 0L; + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + + public static final int RULE_ID_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object ruleId_ = ""; + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The ruleId. + */ + @java.lang.Override + public java.lang.String getRuleId() { + java.lang.Object ref = ruleId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ruleId_ = s; + return s; + } + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The bytes for ruleId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getRuleIdBytes() { + java.lang.Object ref = ruleId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ruleId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); + } + if (detectedAt_ != 0L) { + output.writeInt64(3, detectedAt_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ruleId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 4, ruleId_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); + } + if (detectedAt_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(3, detectedAt_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ruleId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(4, ruleId_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent other = (com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent) obj; + + if (!getActor() + .equals(other.getActor())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (getDetectedAt() + != other.getDetectedAt()) return false; + if (!getRuleId() + .equals(other.getRuleId())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACTOR_FIELD_NUMBER; + hash = (53 * hash) + getActor().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getDetectedAt()); + hash = (37 * hash) + RULE_ID_FIELD_NUMBER; + hash = (53 * hash) + getRuleId().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.message.smart_event.v1.SmartEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.message.smart_event.v1.SmartEvent) + com.akto.proto.threat_protection.message.smart_event.v1.SmartEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.class, com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + actor_ = ""; + filterId_ = ""; + detectedAt_ = 0L; + ruleId_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent getDefaultInstanceForType() { + return com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent build() { + com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent buildPartial() { + com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent result = new com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.actor_ = actor_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.filterId_ = filterId_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.detectedAt_ = detectedAt_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.ruleId_ = ruleId_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent) { + return mergeFrom((com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent other) { + if (other == com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.getDefaultInstance()) return this; + if (!other.getActor().isEmpty()) { + actor_ = other.actor_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (other.getDetectedAt() != 0L) { + setDetectedAt(other.getDetectedAt()); + } + if (!other.getRuleId().isEmpty()) { + ruleId_ = other.ruleId_; + bitField0_ |= 0x00000008; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + actor_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + detectedAt_ = input.readInt64(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 34: { + ruleId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The actor to set. + * @return This builder for chaining. + */ + public Builder setActor( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @return This builder for chaining. + */ + public Builder clearActor() { + actor_ = getDefaultInstance().getActor(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The bytes for actor to set. + * @return This builder for chaining. + */ + public Builder setActorBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private long detectedAt_ ; + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @param value The detectedAt to set. + * @return This builder for chaining. + */ + public Builder setDetectedAt(long value) { + + detectedAt_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return This builder for chaining. + */ + public Builder clearDetectedAt() { + bitField0_ = (bitField0_ & ~0x00000004); + detectedAt_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object ruleId_ = ""; + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The ruleId. + */ + public java.lang.String getRuleId() { + java.lang.Object ref = ruleId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ruleId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The bytes for ruleId. + */ + public com.google.protobuf.ByteString + getRuleIdBytes() { + java.lang.Object ref = ruleId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ruleId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @param value The ruleId to set. + * @return This builder for chaining. + */ + public Builder setRuleId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ruleId_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return This builder for chaining. + */ + public Builder clearRuleId() { + ruleId_ = getDefaultInstance().getRuleId(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @param value The bytes for ruleId to set. + * @return This builder for chaining. + */ + public Builder setRuleIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ruleId_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.message.smart_event.v1.SmartEvent) + } + + // @@protoc_insertion_point(class_scope:threat_protection.message.smart_event.v1.SmartEvent) + private static final com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent(); + } + + public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SmartEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEventOrBuilder.java new file mode 100644 index 0000000000..7790cfaa43 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEventOrBuilder.java @@ -0,0 +1,53 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/message/smart_event/v1/message.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.message.smart_event.v1; + +public interface SmartEventOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.message.smart_event.v1.SmartEvent) + com.google.protobuf.MessageOrBuilder { + + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + java.lang.String getActor(); + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + com.google.protobuf.ByteString + getActorBytes(); + + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + java.lang.String getFilterId(); + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + com.google.protobuf.ByteString + getFilterIdBytes(); + + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + long getDetectedAt(); + + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The ruleId. + */ + java.lang.String getRuleId(); + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The bytes for ruleId. + */ + com.google.protobuf.ByteString + getRuleIdBytes(); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ConsumerServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ConsumerServiceProto.java new file mode 100644 index 0000000000..f116ac17b7 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ConsumerServiceProto.java @@ -0,0 +1,107 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +public final class ConsumerServiceProto { + private ConsumerServiceProto() {} + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + ConsumerServiceProto.class.getName()); + } + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\nKthreat_protection/service/malicious_al" + + "ert_service/v1/consumer_service.proto\0224t" + + "hreat_protection.service.malicious_alert" + + "_service.v1\"\025\n\023RecordAlertResponse\"\264\001\n\024S" + + "ampleMaliciousEvent\022\016\n\002ip\030\001 \001(\tR\002ip\022\034\n\tt" + + "imestamp\030\002 \001(\003R\ttimestamp\022\020\n\003url\030\003 \001(\tR\003" + + "url\022\026\n\006method\030\004 \001(\tR\006method\022*\n\021api_colle" + + "ction_id\030\005 \001(\005R\017apiCollectionId\022\030\n\007paylo" + + "ad\030\006 \001(\tR\007payload\"\370\001\n\022RecordAlertRequest" + + "\022\024\n\005actor\030\001 \001(\tR\005actor\022\033\n\tfilter_id\030\002 \001(" + + "\tR\010filterId\022!\n\014total_events\030\003 \001(\005R\013total" + + "Events\022\037\n\013detected_at\030\004 \001(\003R\ndetectedAt\022" + + "k\n\013sample_data\030\005 \003(\0132J.threat_protection" + + ".service.malicious_alert_service.v1.Samp" + + "leMaliciousEventR\nsampleData2\276\001\n\025Malicio" + + "usAlertService\022\244\001\n\013RecordAlert\022H.threat_" + + "protection.service.malicious_alert_servi" + + "ce.v1.RecordAlertRequest\032I.threat_protec" + + "tion.service.malicious_alert_service.v1." + + "RecordAlertResponse\"\000B\302\002\nCcom.akto.proto" + + ".threat_protection.service.malicious_ale" + + "rt_service.v1B\024ConsumerServiceProtoP\001\242\002\003" + + "TSM\252\0021ThreatProtection.Service.Malicious" + + "AlertService.V1\312\0021ThreatProtection\\Servi" + + "ce\\MaliciousAlertService\\V1\342\002=ThreatProt" + + "ection\\Service\\MaliciousAlertService\\V1\\" + + "GPBMetadata\352\0024ThreatProtection::Service:" + + ":MaliciousAlertService::V1b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor, + new java.lang.String[] { }); + internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor, + new java.lang.String[] { "Ip", "Timestamp", "Url", "Method", "ApiCollectionId", "Payload", }); + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor, + new java.lang.String[] { "Actor", "FilterId", "TotalEvents", "DetectedAt", "SampleData", }); + descriptor.resolveAllFeaturesImmutable(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousAlertServiceGrpc.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousAlertServiceGrpc.java new file mode 100644 index 0000000000..d9af8ef85e --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousAlertServiceGrpc.java @@ -0,0 +1,293 @@ +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +import static io.grpc.MethodDescriptor.generateFullMethodName; + +/** + */ +@javax.annotation.Generated( + value = "by gRPC proto compiler (version 1.68.1)", + comments = "Source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto") +@io.grpc.stub.annotations.GrpcGenerated +public final class MaliciousAlertServiceGrpc { + + private MaliciousAlertServiceGrpc() {} + + public static final java.lang.String SERVICE_NAME = "threat_protection.service.malicious_alert_service.v1.MaliciousAlertService"; + + // Static method descriptors that strictly reflect the proto. + private static volatile io.grpc.MethodDescriptor getRecordAlertMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "RecordAlert", + requestType = com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.class, + responseType = com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getRecordAlertMethod() { + io.grpc.MethodDescriptor getRecordAlertMethod; + if ((getRecordAlertMethod = MaliciousAlertServiceGrpc.getRecordAlertMethod) == null) { + synchronized (MaliciousAlertServiceGrpc.class) { + if ((getRecordAlertMethod = MaliciousAlertServiceGrpc.getRecordAlertMethod) == null) { + MaliciousAlertServiceGrpc.getRecordAlertMethod = getRecordAlertMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "RecordAlert")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.getDefaultInstance())) + .setSchemaDescriptor(new MaliciousAlertServiceMethodDescriptorSupplier("RecordAlert")) + .build(); + } + } + } + return getRecordAlertMethod; + } + + /** + * Creates a new async stub that supports all call types for the service + */ + public static MaliciousAlertServiceStub newStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public MaliciousAlertServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousAlertServiceStub(channel, callOptions); + } + }; + return MaliciousAlertServiceStub.newStub(factory, channel); + } + + /** + * Creates a new blocking-style stub that supports unary and streaming output calls on the service + */ + public static MaliciousAlertServiceBlockingStub newBlockingStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public MaliciousAlertServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousAlertServiceBlockingStub(channel, callOptions); + } + }; + return MaliciousAlertServiceBlockingStub.newStub(factory, channel); + } + + /** + * Creates a new ListenableFuture-style stub that supports unary calls on the service + */ + public static MaliciousAlertServiceFutureStub newFutureStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public MaliciousAlertServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousAlertServiceFutureStub(channel, callOptions); + } + }; + return MaliciousAlertServiceFutureStub.newStub(factory, channel); + } + + /** + */ + public interface AsyncService { + + /** + */ + default void recordAlert(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getRecordAlertMethod(), responseObserver); + } + } + + /** + * Base class for the server implementation of the service MaliciousAlertService. + */ + public static abstract class MaliciousAlertServiceImplBase + implements io.grpc.BindableService, AsyncService { + + @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { + return MaliciousAlertServiceGrpc.bindService(this); + } + } + + /** + * A stub to allow clients to do asynchronous rpc calls to service MaliciousAlertService. + */ + public static final class MaliciousAlertServiceStub + extends io.grpc.stub.AbstractAsyncStub { + private MaliciousAlertServiceStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected MaliciousAlertServiceStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousAlertServiceStub(channel, callOptions); + } + + /** + */ + public void recordAlert(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getRecordAlertMethod(), getCallOptions()), request, responseObserver); + } + } + + /** + * A stub to allow clients to do synchronous rpc calls to service MaliciousAlertService. + */ + public static final class MaliciousAlertServiceBlockingStub + extends io.grpc.stub.AbstractBlockingStub { + private MaliciousAlertServiceBlockingStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected MaliciousAlertServiceBlockingStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousAlertServiceBlockingStub(channel, callOptions); + } + + /** + */ + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse recordAlert(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getRecordAlertMethod(), getCallOptions(), request); + } + } + + /** + * A stub to allow clients to do ListenableFuture-style rpc calls to service MaliciousAlertService. + */ + public static final class MaliciousAlertServiceFutureStub + extends io.grpc.stub.AbstractFutureStub { + private MaliciousAlertServiceFutureStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected MaliciousAlertServiceFutureStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new MaliciousAlertServiceFutureStub(channel, callOptions); + } + + /** + */ + public com.google.common.util.concurrent.ListenableFuture recordAlert( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getRecordAlertMethod(), getCallOptions()), request); + } + } + + private static final int METHODID_RECORD_ALERT = 0; + + private static final class MethodHandlers implements + io.grpc.stub.ServerCalls.UnaryMethod, + io.grpc.stub.ServerCalls.ServerStreamingMethod, + io.grpc.stub.ServerCalls.ClientStreamingMethod, + io.grpc.stub.ServerCalls.BidiStreamingMethod { + private final AsyncService serviceImpl; + private final int methodId; + + MethodHandlers(AsyncService serviceImpl, int methodId) { + this.serviceImpl = serviceImpl; + this.methodId = methodId; + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + case METHODID_RECORD_ALERT: + serviceImpl.recordAlert((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + default: + throw new AssertionError(); + } + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public io.grpc.stub.StreamObserver invoke( + io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + default: + throw new AssertionError(); + } + } + } + + public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { + return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) + .addMethod( + getRecordAlertMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest, + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse>( + service, METHODID_RECORD_ALERT))) + .build(); + } + + private static abstract class MaliciousAlertServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { + MaliciousAlertServiceBaseDescriptorSupplier() {} + + @java.lang.Override + public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.getDescriptor(); + } + + @java.lang.Override + public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { + return getFileDescriptor().findServiceByName("MaliciousAlertService"); + } + } + + private static final class MaliciousAlertServiceFileDescriptorSupplier + extends MaliciousAlertServiceBaseDescriptorSupplier { + MaliciousAlertServiceFileDescriptorSupplier() {} + } + + private static final class MaliciousAlertServiceMethodDescriptorSupplier + extends MaliciousAlertServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { + private final java.lang.String methodName; + + MaliciousAlertServiceMethodDescriptorSupplier(java.lang.String methodName) { + this.methodName = methodName; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { + return getServiceDescriptor().findMethodByName(methodName); + } + } + + private static volatile io.grpc.ServiceDescriptor serviceDescriptor; + + public static io.grpc.ServiceDescriptor getServiceDescriptor() { + io.grpc.ServiceDescriptor result = serviceDescriptor; + if (result == null) { + synchronized (MaliciousAlertServiceGrpc.class) { + result = serviceDescriptor; + if (result == null) { + serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) + .setSchemaDescriptor(new MaliciousAlertServiceFileDescriptorSupplier()) + .addMethod(getRecordAlertMethod()) + .build(); + } + } + } + return result; + } +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequest.java new file mode 100644 index 0000000000..25dbb34c2a --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequest.java @@ -0,0 +1,1124 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +/** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordAlertRequest} + */ +public final class RecordAlertRequest extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) + RecordAlertRequestOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + RecordAlertRequest.class.getName()); + } + // Use RecordAlertRequest.newBuilder() to construct. + private RecordAlertRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private RecordAlertRequest() { + actor_ = ""; + filterId_ = ""; + sampleData_ = java.util.Collections.emptyList(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.Builder.class); + } + + public static final int ACTOR_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + @java.lang.Override + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TOTAL_EVENTS_FIELD_NUMBER = 3; + private int totalEvents_ = 0; + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @return The totalEvents. + */ + @java.lang.Override + public int getTotalEvents() { + return totalEvents_; + } + + public static final int DETECTED_AT_FIELD_NUMBER = 4; + private long detectedAt_ = 0L; + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + + public static final int SAMPLE_DATA_FIELD_NUMBER = 5; + @SuppressWarnings("serial") + private java.util.List sampleData_; + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public java.util.List getSampleDataList() { + return sampleData_; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public java.util.List + getSampleDataOrBuilderList() { + return sampleData_; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public int getSampleDataCount() { + return sampleData_.size(); + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getSampleData(int index) { + return sampleData_.get(index); + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( + int index) { + return sampleData_.get(index); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); + } + if (totalEvents_ != 0) { + output.writeInt32(3, totalEvents_); + } + if (detectedAt_ != 0L) { + output.writeInt64(4, detectedAt_); + } + for (int i = 0; i < sampleData_.size(); i++) { + output.writeMessage(5, sampleData_.get(i)); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); + } + if (totalEvents_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(3, totalEvents_); + } + if (detectedAt_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(4, detectedAt_); + } + for (int i = 0; i < sampleData_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(5, sampleData_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) obj; + + if (!getActor() + .equals(other.getActor())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (getTotalEvents() + != other.getTotalEvents()) return false; + if (getDetectedAt() + != other.getDetectedAt()) return false; + if (!getSampleDataList() + .equals(other.getSampleDataList())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACTOR_FIELD_NUMBER; + hash = (53 * hash) + getActor().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (37 * hash) + TOTAL_EVENTS_FIELD_NUMBER; + hash = (53 * hash) + getTotalEvents(); + hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getDetectedAt()); + if (getSampleDataCount() > 0) { + hash = (37 * hash) + SAMPLE_DATA_FIELD_NUMBER; + hash = (53 * hash) + getSampleDataList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordAlertRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + actor_ = ""; + filterId_ = ""; + totalEvents_ = 0; + detectedAt_ = 0L; + if (sampleDataBuilder_ == null) { + sampleData_ = java.util.Collections.emptyList(); + } else { + sampleData_ = null; + sampleDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest build() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest buildPartial() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest(this); + buildPartialRepeatedFields(result); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartialRepeatedFields(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest result) { + if (sampleDataBuilder_ == null) { + if (((bitField0_ & 0x00000010) != 0)) { + sampleData_ = java.util.Collections.unmodifiableList(sampleData_); + bitField0_ = (bitField0_ & ~0x00000010); + } + result.sampleData_ = sampleData_; + } else { + result.sampleData_ = sampleDataBuilder_.build(); + } + } + + private void buildPartial0(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.actor_ = actor_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.filterId_ = filterId_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.totalEvents_ = totalEvents_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.detectedAt_ = detectedAt_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) { + return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest other) { + if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.getDefaultInstance()) return this; + if (!other.getActor().isEmpty()) { + actor_ = other.actor_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (other.getTotalEvents() != 0) { + setTotalEvents(other.getTotalEvents()); + } + if (other.getDetectedAt() != 0L) { + setDetectedAt(other.getDetectedAt()); + } + if (sampleDataBuilder_ == null) { + if (!other.sampleData_.isEmpty()) { + if (sampleData_.isEmpty()) { + sampleData_ = other.sampleData_; + bitField0_ = (bitField0_ & ~0x00000010); + } else { + ensureSampleDataIsMutable(); + sampleData_.addAll(other.sampleData_); + } + onChanged(); + } + } else { + if (!other.sampleData_.isEmpty()) { + if (sampleDataBuilder_.isEmpty()) { + sampleDataBuilder_.dispose(); + sampleDataBuilder_ = null; + sampleData_ = other.sampleData_; + bitField0_ = (bitField0_ & ~0x00000010); + sampleDataBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getSampleDataFieldBuilder() : null; + } else { + sampleDataBuilder_.addAllMessages(other.sampleData_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + actor_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + totalEvents_ = input.readInt32(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 32: { + detectedAt_ = input.readInt64(); + bitField0_ |= 0x00000008; + break; + } // case 32 + case 42: { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent m = + input.readMessage( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.parser(), + extensionRegistry); + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.add(m); + } else { + sampleDataBuilder_.addMessage(m); + } + break; + } // case 42 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The actor to set. + * @return This builder for chaining. + */ + public Builder setActor( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @return This builder for chaining. + */ + public Builder clearActor() { + actor_ = getDefaultInstance().getActor(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The bytes for actor to set. + * @return This builder for chaining. + */ + public Builder setActorBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private int totalEvents_ ; + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @return The totalEvents. + */ + @java.lang.Override + public int getTotalEvents() { + return totalEvents_; + } + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @param value The totalEvents to set. + * @return This builder for chaining. + */ + public Builder setTotalEvents(int value) { + + totalEvents_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @return This builder for chaining. + */ + public Builder clearTotalEvents() { + bitField0_ = (bitField0_ & ~0x00000004); + totalEvents_ = 0; + onChanged(); + return this; + } + + private long detectedAt_ ; + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @param value The detectedAt to set. + * @return This builder for chaining. + */ + public Builder setDetectedAt(long value) { + + detectedAt_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @return This builder for chaining. + */ + public Builder clearDetectedAt() { + bitField0_ = (bitField0_ & ~0x00000008); + detectedAt_ = 0L; + onChanged(); + return this; + } + + private java.util.List sampleData_ = + java.util.Collections.emptyList(); + private void ensureSampleDataIsMutable() { + if (!((bitField0_ & 0x00000010) != 0)) { + sampleData_ = new java.util.ArrayList(sampleData_); + bitField0_ |= 0x00000010; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder> sampleDataBuilder_; + + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public java.util.List getSampleDataList() { + if (sampleDataBuilder_ == null) { + return java.util.Collections.unmodifiableList(sampleData_); + } else { + return sampleDataBuilder_.getMessageList(); + } + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public int getSampleDataCount() { + if (sampleDataBuilder_ == null) { + return sampleData_.size(); + } else { + return sampleDataBuilder_.getCount(); + } + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getSampleData(int index) { + if (sampleDataBuilder_ == null) { + return sampleData_.get(index); + } else { + return sampleDataBuilder_.getMessage(index); + } + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder setSampleData( + int index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent value) { + if (sampleDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleDataIsMutable(); + sampleData_.set(index, value); + onChanged(); + } else { + sampleDataBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder setSampleData( + int index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder builderForValue) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.set(index, builderForValue.build()); + onChanged(); + } else { + sampleDataBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addSampleData(com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent value) { + if (sampleDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleDataIsMutable(); + sampleData_.add(value); + onChanged(); + } else { + sampleDataBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addSampleData( + int index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent value) { + if (sampleDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleDataIsMutable(); + sampleData_.add(index, value); + onChanged(); + } else { + sampleDataBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addSampleData( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder builderForValue) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.add(builderForValue.build()); + onChanged(); + } else { + sampleDataBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addSampleData( + int index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder builderForValue) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.add(index, builderForValue.build()); + onChanged(); + } else { + sampleDataBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addAllSampleData( + java.lang.Iterable values) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, sampleData_); + onChanged(); + } else { + sampleDataBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder clearSampleData() { + if (sampleDataBuilder_ == null) { + sampleData_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + } else { + sampleDataBuilder_.clear(); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder removeSampleData(int index) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.remove(index); + onChanged(); + } else { + sampleDataBuilder_.remove(index); + } + return this; + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder getSampleDataBuilder( + int index) { + return getSampleDataFieldBuilder().getBuilder(index); + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( + int index) { + if (sampleDataBuilder_ == null) { + return sampleData_.get(index); } else { + return sampleDataBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public java.util.List + getSampleDataOrBuilderList() { + if (sampleDataBuilder_ != null) { + return sampleDataBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(sampleData_); + } + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder addSampleDataBuilder() { + return getSampleDataFieldBuilder().addBuilder( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.getDefaultInstance()); + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder addSampleDataBuilder( + int index) { + return getSampleDataFieldBuilder().addBuilder( + index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.getDefaultInstance()); + } + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public java.util.List + getSampleDataBuilderList() { + return getSampleDataFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder> + getSampleDataFieldBuilder() { + if (sampleDataBuilder_ == null) { + sampleDataBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder>( + sampleData_, + ((bitField0_ & 0x00000010) != 0), + getParentForChildren(), + isClean()); + sampleData_ = null; + } + return sampleDataBuilder_; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) + private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest(); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RecordAlertRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequestOrBuilder.java new file mode 100644 index 0000000000..30aa5d0f0e --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequestOrBuilder.java @@ -0,0 +1,71 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +public interface RecordAlertRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + java.lang.String getActor(); + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + com.google.protobuf.ByteString + getActorBytes(); + + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + java.lang.String getFilterId(); + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + com.google.protobuf.ByteString + getFilterIdBytes(); + + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @return The totalEvents. + */ + int getTotalEvents(); + + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + long getDetectedAt(); + + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + java.util.List + getSampleDataList(); + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getSampleData(int index); + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + int getSampleDataCount(); + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + java.util.List + getSampleDataOrBuilderList(); + /** + * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( + int index); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponse.java new file mode 100644 index 0000000000..e5175020dd --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponse.java @@ -0,0 +1,358 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +/** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordAlertResponse} + */ +public final class RecordAlertResponse extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) + RecordAlertResponseOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + RecordAlertResponse.class.getName()); + } + // Use RecordAlertResponse.newBuilder() to construct. + private RecordAlertResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private RecordAlertResponse() { + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.Builder.class); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) obj; + + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordAlertResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse build() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse buildPartial() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) { + return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse other) { + if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) + private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse(); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RecordAlertResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponseOrBuilder.java new file mode 100644 index 0000000000..4ae50c1aa6 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponseOrBuilder.java @@ -0,0 +1,11 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +public interface RecordAlertResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) + com.google.protobuf.MessageOrBuilder { +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEvent.java new file mode 100644 index 0000000000..a375dc0fe3 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEvent.java @@ -0,0 +1,1042 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +/** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent} + */ +public final class SampleMaliciousEvent extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) + SampleMaliciousEventOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SampleMaliciousEvent.class.getName()); + } + // Use SampleMaliciousEvent.newBuilder() to construct. + private SampleMaliciousEvent(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SampleMaliciousEvent() { + ip_ = ""; + url_ = ""; + method_ = ""; + payload_ = ""; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder.class); + } + + public static final int IP_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object ip_ = ""; + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + @java.lang.Override + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TIMESTAMP_FIELD_NUMBER = 2; + private long timestamp_ = 0L; + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + + public static final int URL_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object url_ = ""; + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + @java.lang.Override + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } + } + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int METHOD_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object method_ = ""; + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + @java.lang.Override + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } + } + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int API_COLLECTION_ID_FIELD_NUMBER = 5; + private int apiCollectionId_ = 0; + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + + public static final int PAYLOAD_FIELD_NUMBER = 6; + @SuppressWarnings("serial") + private volatile java.lang.Object payload_ = ""; + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + @java.lang.Override + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, ip_); + } + if (timestamp_ != 0L) { + output.writeInt64(2, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 3, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 4, method_); + } + if (apiCollectionId_ != 0) { + output.writeInt32(5, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 6, payload_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, ip_); + } + if (timestamp_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(2, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(3, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(4, method_); + } + if (apiCollectionId_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(5, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(6, payload_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) obj; + + if (!getIp() + .equals(other.getIp())) return false; + if (getTimestamp() + != other.getTimestamp()) return false; + if (!getUrl() + .equals(other.getUrl())) return false; + if (!getMethod() + .equals(other.getMethod())) return false; + if (getApiCollectionId() + != other.getApiCollectionId()) return false; + if (!getPayload() + .equals(other.getPayload())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + IP_FIELD_NUMBER; + hash = (53 * hash) + getIp().hashCode(); + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getTimestamp()); + hash = (37 * hash) + URL_FIELD_NUMBER; + hash = (53 * hash) + getUrl().hashCode(); + hash = (37 * hash) + METHOD_FIELD_NUMBER; + hash = (53 * hash) + getMethod().hashCode(); + hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; + hash = (53 * hash) + getApiCollectionId(); + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ip_ = ""; + timestamp_ = 0L; + url_ = ""; + method_ = ""; + apiCollectionId_ = 0; + payload_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent build() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent buildPartial() { + com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ip_ = ip_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.timestamp_ = timestamp_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.url_ = url_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.method_ = method_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.apiCollectionId_ = apiCollectionId_; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.payload_ = payload_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) { + return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent other) { + if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.getDefaultInstance()) return this; + if (!other.getIp().isEmpty()) { + ip_ = other.ip_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.getTimestamp() != 0L) { + setTimestamp(other.getTimestamp()); + } + if (!other.getUrl().isEmpty()) { + url_ = other.url_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (!other.getMethod().isEmpty()) { + method_ = other.method_; + bitField0_ |= 0x00000008; + onChanged(); + } + if (other.getApiCollectionId() != 0) { + setApiCollectionId(other.getApiCollectionId()); + } + if (!other.getPayload().isEmpty()) { + payload_ = other.payload_; + bitField0_ |= 0x00000020; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + ip_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 26: { + url_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + method_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 40: { + apiCollectionId_ = input.readInt32(); + bitField0_ |= 0x00000010; + break; + } // case 40 + case 50: { + payload_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000020; + break; + } // case 50 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object ip_ = ""; + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @param value The ip to set. + * @return This builder for chaining. + */ + public Builder setIp( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ip_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string ip = 1 [json_name = "ip"]; + * @return This builder for chaining. + */ + public Builder clearIp() { + ip_ = getDefaultInstance().getIp(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string ip = 1 [json_name = "ip"]; + * @param value The bytes for ip to set. + * @return This builder for chaining. + */ + public Builder setIpBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ip_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private long timestamp_ ; + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + + timestamp_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000002); + timestamp_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object url_ = ""; + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string url = 3 [json_name = "url"]; + * @param value The url to set. + * @return This builder for chaining. + */ + public Builder setUrl( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + url_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string url = 3 [json_name = "url"]; + * @return This builder for chaining. + */ + public Builder clearUrl() { + url_ = getDefaultInstance().getUrl(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string url = 3 [json_name = "url"]; + * @param value The bytes for url to set. + * @return This builder for chaining. + */ + public Builder setUrlBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + url_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private java.lang.Object method_ = ""; + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string method = 4 [json_name = "method"]; + * @param value The method to set. + * @return This builder for chaining. + */ + public Builder setMethod( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + method_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * string method = 4 [json_name = "method"]; + * @return This builder for chaining. + */ + public Builder clearMethod() { + method_ = getDefaultInstance().getMethod(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * string method = 4 [json_name = "method"]; + * @param value The bytes for method to set. + * @return This builder for chaining. + */ + public Builder setMethodBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + method_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + + private int apiCollectionId_ ; + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @param value The apiCollectionId to set. + * @return This builder for chaining. + */ + public Builder setApiCollectionId(int value) { + + apiCollectionId_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return This builder for chaining. + */ + public Builder clearApiCollectionId() { + bitField0_ = (bitField0_ & ~0x00000010); + apiCollectionId_ = 0; + onChanged(); + return this; + } + + private java.lang.Object payload_ = ""; + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @param value The payload to set. + * @return This builder for chaining. + */ + public Builder setPayload( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + payload_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * string payload = 6 [json_name = "payload"]; + * @return This builder for chaining. + */ + public Builder clearPayload() { + payload_ = getDefaultInstance().getPayload(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + return this; + } + /** + * string payload = 6 [json_name = "payload"]; + * @param value The bytes for payload to set. + * @return This builder for chaining. + */ + public Builder setPayloadBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + payload_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) + private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent(); + } + + public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SampleMaliciousEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEventOrBuilder.java new file mode 100644 index 0000000000..a1628943a1 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEventOrBuilder.java @@ -0,0 +1,71 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.malicious_alert_service.v1; + +public interface SampleMaliciousEventOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) + com.google.protobuf.MessageOrBuilder { + + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + java.lang.String getIp(); + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + com.google.protobuf.ByteString + getIpBytes(); + + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + long getTimestamp(); + + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + java.lang.String getUrl(); + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + com.google.protobuf.ByteString + getUrlBytes(); + + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + java.lang.String getMethod(); + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + com.google.protobuf.ByteString + getMethodBytes(); + + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + int getApiCollectionId(); + + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + java.lang.String getPayload(); + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + com.google.protobuf.ByteString + getPayloadBytes(); +} From d4219c261abece35a096b57846f11e3cec7d6310 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Mon, 18 Nov 2024 13:35:22 +0530 Subject: [PATCH 17/73] refactor code --- .../java/com/akto/runtime/APICatalogSync.java | 1349 +++++++++++------ .../akto/cache/RedisBackedCounterCache.java | 11 +- .../java/com/akto/filters/HttpCallFilter.java | 2 +- .../WindowBasedThresholdNotifier.java | 1 + .../com/akto/utils/jobs/CleanInventory.java | 266 ++-- .../dao/monitoring/FilterYamlTemplateDao.java | 12 +- .../SampleMaliciousRequest.java | 4 +- .../v1/ConsumerServiceGrpc.java | 293 ++++ .../v1/ConsumerServiceProto.java | 106 ++ .../v1/RecordAlertRequest.java | 1124 ++++++++++++++ .../v1/RecordAlertRequestOrBuilder.java | 71 + .../v1/RecordAlertResponse.java | 358 +++++ .../v1/RecordAlertResponseOrBuilder.java | 11 + .../v1/SampleMaliciousEvent.java | 1042 +++++++++++++ .../v1/SampleMaliciousEventOrBuilder.java | 71 + 15 files changed, 4159 insertions(+), 562 deletions(-) create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceGrpc.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceProto.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponse.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponseOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEvent.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEventOrBuilder.java diff --git a/apps/api-runtime/src/main/java/com/akto/runtime/APICatalogSync.java b/apps/api-runtime/src/main/java/com/akto/runtime/APICatalogSync.java index 3f0d5ee645..ee8d46faa3 100644 --- a/apps/api-runtime/src/main/java/com/akto/runtime/APICatalogSync.java +++ b/apps/api-runtime/src/main/java/com/akto/runtime/APICatalogSync.java @@ -54,7 +54,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.util.Map.Entry; import static com.akto.dto.type.KeyTypes.patternToSubType; @@ -73,18 +72,20 @@ public class APICatalogSync { public AktoPolicyNew aktoPolicyNew; public Map sensitiveParamInfoBooleanMap; public static boolean mergeAsyncOutside = true; - public BloomFilter existingAPIsInDb = BloomFilter.create(Funnels.stringFunnel(Charsets.UTF_8), 1_000_000, 0.001 ); + public BloomFilter existingAPIsInDb = + BloomFilter.create(Funnels.stringFunnel(Charsets.UTF_8), 1_000_000, 0.001); public static Set mergedUrls; - public Map advancedFilterMap = new HashMap<>(); + public Map advancedFilterMap = new HashMap<>(); - public APICatalogSync(String userIdentifier,int thresh, boolean fetchAllSTI) { + public APICatalogSync(String userIdentifier, int thresh, boolean fetchAllSTI) { this(userIdentifier, thresh, fetchAllSTI, true); } // New overloaded constructor - public APICatalogSync(String userIdentifier, int thresh, boolean fetchAllSTI, boolean buildFromDb) { + public APICatalogSync( + String userIdentifier, int thresh, boolean fetchAllSTI, boolean buildFromDb) { this.thresh = thresh; this.userIdentifier = userIdentifier; this.dbState = new HashMap<>(); @@ -94,7 +95,8 @@ public APICatalogSync(String userIdentifier, int thresh, boolean fetchAllSTI, bo mergedUrls = new HashSet<>(); if (buildFromDb) { buildFromDB(false, fetchAllSTI); - AccountSettings accountSettings = AccountSettingsDao.instance.findOne(AccountSettingsDao.generateFilter()); + AccountSettings accountSettings = + AccountSettingsDao.instance.findOne(AccountSettingsDao.generateFilter()); if (accountSettings != null && accountSettings.getPartnerIpList() != null) { partnerIpsList = accountSettings.getPartnerIpList(); } @@ -103,9 +105,12 @@ public APICatalogSync(String userIdentifier, int thresh, boolean fetchAllSTI, bo public static final int STRING_MERGING_THRESHOLD = 10; - public void processResponse(RequestTemplate requestTemplate, Collection responses, List deletedInfo) { + public void processResponse( + RequestTemplate requestTemplate, + Collection responses, + List deletedInfo) { Iterator iter = responses.iterator(); - while(iter.hasNext()) { + while (iter.hasNext()) { try { processResponse(requestTemplate, iter.next(), deletedInfo); } catch (Exception e) { @@ -115,7 +120,10 @@ public void processResponse(RequestTemplate requestTemplate, Collection deletedInfo) { + public void processResponse( + RequestTemplate requestTemplate, + HttpResponseParams responseParams, + List deletedInfo) { int timestamp = responseParams.getTimeOrNow(); HttpRequestParams requestParams = responseParams.getRequestParams(); String urlWithParams = requestParams.getURL(); @@ -136,17 +144,43 @@ public void processResponse(RequestTemplate requestTemplate, HttpResponseParams reqPayload = "{}"; } - requestTemplate.processHeaders(requestParams.getHeaders(), baseURL.getUrl(), methodStr, -1, userId, requestParams.getApiCollectionId(), responseParams.getOrig(), sensitiveParamInfoBooleanMap, timestamp); - JSONObject jsonObject = RequestTemplate.parseRequestPayloadToJsonObject(requestParams.getPayload(), urlWithParams); + requestTemplate.processHeaders( + requestParams.getHeaders(), + baseURL.getUrl(), + methodStr, + -1, + userId, + requestParams.getApiCollectionId(), + responseParams.getOrig(), + sensitiveParamInfoBooleanMap, + timestamp); + JSONObject jsonObject = + RequestTemplate.parseRequestPayloadToJsonObject( + requestParams.getPayload(), urlWithParams); Map> flattened = JSONUtils.flattenJSONObject(jsonObject); - deletedInfo.addAll(requestTemplate.process2(flattened, baseURL.getUrl(), methodStr, -1, userId, requestParams.getApiCollectionId(), responseParams.getOrig(), sensitiveParamInfoBooleanMap, timestamp)); + deletedInfo.addAll( + requestTemplate.process2( + flattened, + baseURL.getUrl(), + methodStr, + -1, + userId, + requestParams.getApiCollectionId(), + responseParams.getOrig(), + sensitiveParamInfoBooleanMap, + timestamp)); requestTemplate.recordMessage(responseParams.getOrig()); Map responseTemplates = requestTemplate.getResponseTemplates(); - + RequestTemplate responseTemplate = responseTemplates.get(statusCode); if (responseTemplate == null) { - responseTemplate = new RequestTemplate(new HashMap<>(), null, new HashMap<>(), new TrafficRecorder(new HashMap<>())); + responseTemplate = + new RequestTemplate( + new HashMap<>(), + null, + new HashMap<>(), + new TrafficRecorder(new HashMap<>())); responseTemplates.put(statusCode, responseTemplate); } @@ -157,8 +191,8 @@ public void processResponse(RequestTemplate requestTemplate, HttpResponseParams respPayload = "{}"; } - if(respPayload.startsWith("[")) { - respPayload = "{\"json\": "+respPayload+"}"; + if (respPayload.startsWith("[")) { + respPayload = "{\"json\": " + respPayload + "}"; } JSONObject payload; @@ -169,14 +203,34 @@ public void processResponse(RequestTemplate requestTemplate, HttpResponseParams } flattened = JSONUtils.flattenJSONObject(payload); - deletedInfo.addAll(responseTemplate.process2(flattened, baseURL.getUrl(), methodStr, statusCode, userId, requestParams.getApiCollectionId(), responseParams.getOrig(), sensitiveParamInfoBooleanMap, timestamp)); - responseTemplate.processHeaders(responseParams.getHeaders(), baseURL.getUrl(), method.name(), statusCode, userId, requestParams.getApiCollectionId(), responseParams.getOrig(), sensitiveParamInfoBooleanMap, timestamp); + deletedInfo.addAll( + responseTemplate.process2( + flattened, + baseURL.getUrl(), + methodStr, + statusCode, + userId, + requestParams.getApiCollectionId(), + responseParams.getOrig(), + sensitiveParamInfoBooleanMap, + timestamp)); + responseTemplate.processHeaders( + responseParams.getHeaders(), + baseURL.getUrl(), + method.name(), + statusCode, + userId, + requestParams.getApiCollectionId(), + responseParams.getOrig(), + sensitiveParamInfoBooleanMap, + timestamp); if (!responseParams.getIsPending()) { responseTemplate.processTraffic(responseParams.getTime()); } } catch (JsonParseException e) { - loggerMaker.errorAndAddToDb("Failed to parse json payload " + e.getMessage(), LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + "Failed to parse json payload " + e.getMessage(), LogDb.RUNTIME); } } @@ -191,27 +245,31 @@ public static String extractUserId(HttpResponseParams responseParams, String use int countUsers(Set responseParamsList) { Set users = new HashSet<>(); - for(HttpResponseParams responseParams: responseParamsList) { + for (HttpResponseParams responseParams : responseParamsList) { users.add(extractUserId(responseParams, userIdentifier)); } return users.size(); } - public void computeDelta(URLAggregator origAggregator, boolean triggerTemplateGeneration, int apiCollectionId, boolean makeApisCaseInsensitive) { + public void computeDelta( + URLAggregator origAggregator, + boolean triggerTemplateGeneration, + int apiCollectionId, + boolean makeApisCaseInsensitive) { long start = System.currentTimeMillis(); APICatalog deltaCatalog = this.delta.get(apiCollectionId); if (deltaCatalog == null) { deltaCatalog = new APICatalog(apiCollectionId, new HashMap<>(), new HashMap<>()); this.delta.put(apiCollectionId, deltaCatalog); - } + } APICatalog dbCatalog = this.dbState.get(apiCollectionId); if (dbCatalog == null) { dbCatalog = new APICatalog(apiCollectionId, new HashMap<>(), new HashMap<>()); this.dbState.put(apiCollectionId, dbCatalog); - } + } URLAggregator aggregator = new URLAggregator(origAggregator.urls); logger.info("aggregator: " + (System.currentTimeMillis() - start)); @@ -220,7 +278,7 @@ public void computeDelta(URLAggregator origAggregator, boolean triggerTemplateGe Set>> entries = aggregator.urls.entrySet(); for (Map.Entry> entry : entries) { Set value = entry.getValue(); - for (HttpResponseParams responseParams: value) { + for (HttpResponseParams responseParams : value) { try { aktoPolicyNew.process(responseParams, partnerIpsList); } catch (Exception e) { @@ -232,14 +290,14 @@ public void computeDelta(URLAggregator origAggregator, boolean triggerTemplateGe start = System.currentTimeMillis(); processKnownStaticURLs(aggregator, deltaCatalog, dbCatalog, makeApisCaseInsensitive); - logger.info("processKnownStaticURLs: " + (System.currentTimeMillis() - start)); + logger.info("processKnownStaticURLs: " + (System.currentTimeMillis() - start)); start = System.currentTimeMillis(); Map pendingRequests = createRequestTemplates(aggregator); logger.info("pendingRequests: " + (System.currentTimeMillis() - start)); start = System.currentTimeMillis(); - tryWithKnownURLTemplates(pendingRequests, deltaCatalog, dbCatalog, apiCollectionId ); + tryWithKnownURLTemplates(pendingRequests, deltaCatalog, dbCatalog, apiCollectionId); logger.info("tryWithKnownURLTemplates: " + (System.currentTimeMillis() - start)); if (!mergeAsyncOutside) { @@ -247,26 +305,31 @@ public void computeDelta(URLAggregator origAggregator, boolean triggerTemplateGe tryMergingWithKnownStrictURLs(pendingRequests, dbCatalog, deltaCatalog); logger.info("tryMergingWithKnownStrictURLs: " + (System.currentTimeMillis() - start)); } else { - for (URLStatic pending: pendingRequests.keySet()) { + for (URLStatic pending : pendingRequests.keySet()) { RequestTemplate pendingTemplate = pendingRequests.get(pending); URLTemplate parameterisedTemplate = null; - if((apiCollectionId != VULNERABLE_API_COLLECTION_ID) && (apiCollectionId != LLM_API_COLLECTION_ID)){ + if ((apiCollectionId != VULNERABLE_API_COLLECTION_ID) + && (apiCollectionId != LLM_API_COLLECTION_ID)) { parameterisedTemplate = tryParamteresingUrl(pending); } - if(parameterisedTemplate != null){ - RequestTemplate rt = deltaCatalog.getTemplateURLToMethods().get(parameterisedTemplate); + if (parameterisedTemplate != null) { + RequestTemplate rt = + deltaCatalog.getTemplateURLToMethods().get(parameterisedTemplate); if (rt != null) { rt.mergeFrom(pendingTemplate); } else { - deltaCatalog.getTemplateURLToMethods().put(parameterisedTemplate, pendingTemplate); + deltaCatalog + .getTemplateURLToMethods() + .put(parameterisedTemplate, pendingTemplate); } rt = deltaCatalog.getTemplateURLToMethods().get(parameterisedTemplate); - rt.fillUrlParams(tokenize(pending.getUrl()), parameterisedTemplate, apiCollectionId); + rt.fillUrlParams( + tokenize(pending.getUrl()), parameterisedTemplate, apiCollectionId); - }else{ + } else { RequestTemplate rt = deltaCatalog.getStrictURLToMethods().get(pending); if (rt != null) { rt.mergeFrom(pendingTemplate); @@ -274,17 +337,24 @@ public void computeDelta(URLAggregator origAggregator, boolean triggerTemplateGe deltaCatalog.getStrictURLToMethods().put(pending, pendingTemplate); } } - - } } - logger.info("processTime: " + RequestTemplate.insertTime + " " + RequestTemplate.processTime + " " + RequestTemplate.deleteTime); - + logger.info( + "processTime: " + + RequestTemplate.insertTime + + " " + + RequestTemplate.processTime + + " " + + RequestTemplate.deleteTime); } - - public static ApiMergerResult tryMergeURLsInCollection(int apiCollectionId, Boolean urlRegexMatchingEnabled, boolean mergeUrlsBasic, BloomFilter existingAPIsInDb, boolean ignoreCaseInsensitiveApis) { + public static ApiMergerResult tryMergeURLsInCollection( + int apiCollectionId, + Boolean urlRegexMatchingEnabled, + boolean mergeUrlsBasic, + BloomFilter existingAPIsInDb, + boolean ignoreCaseInsensitiveApis) { ApiCollection apiCollection = ApiCollectionsDao.instance.getMeta(apiCollectionId); Bson filterQ = null; @@ -292,10 +362,10 @@ public static ApiMergerResult tryMergeURLsInCollection(int apiCollectionId, Bool filterQ = Filters.eq("apiCollectionId", apiCollectionId); } else { Bson hostFilter = SingleTypeInfoDao.filterForHostHeader(apiCollectionId, true); - Bson normalFilter = Filters.and( - Filters.eq("apiCollectionId", apiCollectionId), - Filters.or(Filters.eq("isHeader", false), Filters.eq("param", "host")) - ); + Bson normalFilter = + Filters.and( + Filters.eq("apiCollectionId", apiCollectionId), + Filters.or(Filters.eq("isHeader", false), Filters.eq("param", "host"))); filterQ = mergeUrlsBasic ? hostFilter : normalFilter; } @@ -305,20 +375,23 @@ public static ApiMergerResult tryMergeURLsInCollection(int apiCollectionId, Bool List singleTypeInfos = new ArrayList<>(); ApiMergerResult finalResult = new ApiMergerResult(new HashMap<>()); do { - singleTypeInfos = SingleTypeInfoDao.instance.findAll(filterQ, offset, limit, null, Projections.exclude("values")); + singleTypeInfos = + SingleTypeInfoDao.instance.findAll( + filterQ, offset, limit, null, Projections.exclude("values")); logger.info("Singetypeinfo size: " + singleTypeInfos.size()); Map> staticUrlToSti = new HashMap<>(); Set templateUrlSet = new HashSet<>(); List templateUrls = new ArrayList<>(); - for(SingleTypeInfo sti: singleTypeInfos) { + for (SingleTypeInfo sti : singleTypeInfos) { String key = sti.getMethod() + " " + sti.getUrl(); fillExistingAPIsInDb(sti, existingAPIsInDb); if (APICatalog.isTemplateUrl(sti.getUrl())) { templateUrlSet.add(key); continue; - }; + } + ; if (sti.getIsUrlParam()) continue; if (sti.getIsHeader()) { @@ -326,7 +399,6 @@ public static ApiMergerResult tryMergeURLsInCollection(int apiCollectionId, Bool continue; } - Set set = staticUrlToSti.get(key); if (set == null) { set = new HashSet<>(); @@ -336,7 +408,7 @@ public static ApiMergerResult tryMergeURLsInCollection(int apiCollectionId, Bool set.add(sti.getResponseCode() + " " + sti.getParam()); } - for (String s: templateUrlSet) { + for (String s : templateUrlSet) { templateUrls.add(s); } @@ -350,15 +422,15 @@ public static ApiMergerResult tryMergeURLsInCollection(int apiCollectionId, Bool String staticEndpoint = staticURL.split(" ")[1]; String tempEndpoint = staticEndpoint.toLowerCase(); - if(ignoreCaseInsensitiveApis){ - if(!seenStaticUrls.isEmpty() && seenStaticUrls.contains(tempEndpoint)){ + if (ignoreCaseInsensitiveApis) { + if (!seenStaticUrls.isEmpty() && seenStaticUrls.contains(tempEndpoint)) { finalResult.deleteStaticUrls.add(staticURL); iterator.remove(); continue; } } - for (String templateURL: templateUrls) { + for (String templateURL : templateUrls) { Method templateMethod = Method.fromString(templateURL.split(" ")[0]); String templateEndpoint = templateURL.split(" ")[1]; @@ -369,19 +441,21 @@ public static ApiMergerResult tryMergeURLsInCollection(int apiCollectionId, Bool break; } } - if(ignoreCaseInsensitiveApis){ + if (ignoreCaseInsensitiveApis) { seenStaticUrls.add(tempEndpoint); } } - Map>> sizeToUrlToSti = groupByTokenSize(staticUrlToSti); + Map>> sizeToUrlToSti = + groupByTokenSize(staticUrlToSti); sizeToUrlToSti.remove(1); sizeToUrlToSti.remove(0); - - for(int size: sizeToUrlToSti.keySet()) { - ApiMergerResult result = tryMergingWithKnownStrictURLs(sizeToUrlToSti.get(size), urlRegexMatchingEnabled, !mergeUrlsBasic); + for (int size : sizeToUrlToSti.keySet()) { + ApiMergerResult result = + tryMergingWithKnownStrictURLs( + sizeToUrlToSti.get(size), urlRegexMatchingEnabled, !mergeUrlsBasic); finalResult.templateToStaticURLs.putAll(result.templateToStaticURLs); } @@ -391,11 +465,15 @@ public static ApiMergerResult tryMergeURLsInCollection(int apiCollectionId, Bool return finalResult; } - private static Map>> groupByTokenSize(Map> catalog) { + private static Map>> groupByTokenSize( + Map> catalog) { Map>> sizeToURL = new HashMap<>(); - for(String rawURLPlusMethod: catalog.keySet()) { + for (String rawURLPlusMethod : catalog.keySet()) { String[] rawUrlPlusMethodSplit = rawURLPlusMethod.split(" "); - String rawURL = rawUrlPlusMethodSplit.length > 1 ? rawUrlPlusMethodSplit[1] : rawUrlPlusMethodSplit[0]; + String rawURL = + rawUrlPlusMethodSplit.length > 1 + ? rawUrlPlusMethodSplit[1] + : rawUrlPlusMethodSplit[0]; Set reqTemplate = catalog.get(rawURLPlusMethod); String url = APICatalogSync.trim(rawURL); String[] tokens = url.split("/"); @@ -421,10 +499,11 @@ public ApiMergerResult(Map> templateToSti) { } public String toString() { - String ret = ("templateToSti======================================================: \n"); - for(URLTemplate urlTemplate: templateToStaticURLs.keySet()) { + String ret = + ("templateToSti======================================================: \n"); + for (URLTemplate urlTemplate : templateToStaticURLs.keySet()) { ret += (urlTemplate.getTemplateString()) + "\n"; - for(String str: templateToStaticURLs.get(urlTemplate)) { + for (String str : templateToStaticURLs.get(urlTemplate)) { ret += ("\t " + str + "\n"); } } @@ -434,8 +513,9 @@ public String toString() { } private static ApiMergerResult tryMergingWithKnownStrictURLs( - Map> pendingRequests, Boolean urlRegexMatchingEnabled, boolean doBodyMatch - ) { + Map> pendingRequests, + Boolean urlRegexMatchingEnabled, + boolean doBodyMatch) { Map> templateToStaticURLs = new HashMap<>(); Iterator>> iterator = pendingRequests.entrySet().iterator(); @@ -449,7 +529,7 @@ private static ApiMergerResult tryMergingWithKnownStrictURLs( String newEndpoint = newUrl.split(" ")[1]; boolean matchedInDeltaTemplate = false; - for(URLTemplate urlTemplate: templateToStaticURLs.keySet()){ + for (URLTemplate urlTemplate : templateToStaticURLs.keySet()) { if (urlTemplate.match(newEndpoint, newMethod)) { matchedInDeltaTemplate = true; templateToStaticURLs.get(urlTemplate).add(newUrl); @@ -464,16 +544,16 @@ private static ApiMergerResult tryMergingWithKnownStrictURLs( URLStatic newStaticUrl = new URLStatic(newEndpoint, newMethod); URLTemplate tempUrlTemplate = tryParamteresingUrl(newStaticUrl); - if(tempUrlTemplate != null){ - Set matchedStaticURLs = templateToStaticURLs.getOrDefault(tempUrlTemplate, new HashSet<>()); + if (tempUrlTemplate != null) { + Set matchedStaticURLs = + templateToStaticURLs.getOrDefault(tempUrlTemplate, new HashSet<>()); matchedStaticURLs.add(newUrl); templateToStaticURLs.put(tempUrlTemplate, matchedStaticURLs); } - int countSimilarURLs = 0; Map>> potentialMerges = new HashMap<>(); - for(String aUrl: pendingRequests.keySet()) { + for (String aUrl : pendingRequests.keySet()) { Set aTemplate = pendingRequests.get(aUrl); Method aMethod = Method.fromString(aUrl.split(" ")[0]); String aEndpoint = aUrl.split(" ")[1]; @@ -484,16 +564,25 @@ private static ApiMergerResult tryMergingWithKnownStrictURLs( continue; } - boolean compareKeys = doBodyMatch && RequestTemplate.compareKeys(aTemplate, newTemplate, mergedTemplate); - if (APICatalogSync.areBothMatchingUrls(newStatic,aStatic,mergedTemplate, urlRegexMatchingEnabled) || APICatalogSync.areBothUuidUrls(newStatic,aStatic,mergedTemplate) || compareKeys) { + boolean compareKeys = + doBodyMatch + && RequestTemplate.compareKeys( + aTemplate, newTemplate, mergedTemplate); + if (APICatalogSync.areBothMatchingUrls( + newStatic, aStatic, mergedTemplate, urlRegexMatchingEnabled) + || APICatalogSync.areBothUuidUrls(newStatic, aStatic, mergedTemplate) + || compareKeys) { Map> similarTemplates = potentialMerges.get(mergedTemplate); if (similarTemplates == null) { similarTemplates = new HashMap<>(); potentialMerges.put(mergedTemplate, similarTemplates); - } + } similarTemplates.put(aUrl, aTemplate); - if (!RequestTemplate.isMergedOnStr(mergedTemplate) || APICatalogSync.areBothUuidUrls(newStatic,aStatic,mergedTemplate) || APICatalogSync.areBothMatchingUrls(newStatic,aStatic,mergedTemplate, urlRegexMatchingEnabled)) { + if (!RequestTemplate.isMergedOnStr(mergedTemplate) + || APICatalogSync.areBothUuidUrls(newStatic, aStatic, mergedTemplate) + || APICatalogSync.areBothMatchingUrls( + newStatic, aStatic, mergedTemplate, urlRegexMatchingEnabled)) { countSimilarURLs = APICatalogSync.STRING_MERGING_THRESHOLD; } @@ -512,7 +601,8 @@ private static ApiMergerResult tryMergingWithKnownStrictURLs( matchedStaticURLs.add(newUrl); - for (Map.Entry> rt: potentialMerges.getOrDefault(mergedTemplate, new HashMap<>()).entrySet()) { + for (Map.Entry> rt : + potentialMerges.getOrDefault(mergedTemplate, new HashMap<>()).entrySet()) { matchedStaticURLs.add(rt.getKey()); } } @@ -522,12 +612,13 @@ private static ApiMergerResult tryMergingWithKnownStrictURLs( } private void tryMergingWithKnownStrictURLs( - Map pendingRequests, - APICatalog dbCatalog, - APICatalog deltaCatalog - ) { - Iterator> iterator = pendingRequests.entrySet().iterator(); - Map> dbSizeToUrlToTemplate = groupByTokenSize(dbCatalog); + Map pendingRequests, + APICatalog dbCatalog, + APICatalog deltaCatalog) { + Iterator> iterator = + pendingRequests.entrySet().iterator(); + Map> dbSizeToUrlToTemplate = + groupByTokenSize(dbCatalog); Map deltaTemplates = deltaCatalog.getStrictURLToMethods(); while (iterator.hasNext()) { Map.Entry entry = iterator.next(); @@ -547,8 +638,9 @@ private void tryMergingWithKnownStrictURLs( } boolean matchedInDeltaTemplate = false; - for(URLTemplate urlTemplate: deltaCatalog.getTemplateURLToMethods().keySet()){ - RequestTemplate deltaTemplate = deltaCatalog.getTemplateURLToMethods().get(urlTemplate); + for (URLTemplate urlTemplate : deltaCatalog.getTemplateURLToMethods().keySet()) { + RequestTemplate deltaTemplate = + deltaCatalog.getTemplateURLToMethods().get(urlTemplate); if (urlTemplate.match(newUrl)) { matchedInDeltaTemplate = true; deltaTemplate.mergeFrom(newTemplate); @@ -566,45 +658,49 @@ private void tryMergingWithKnownStrictURLs( boolean newUrlMatchedInDb = false; int countSimilarURLs = 0; Map> potentialMerges = new HashMap<>(); - for(URLStatic dbUrl: dbTemplates.keySet()) { + for (URLStatic dbUrl : dbTemplates.keySet()) { RequestTemplate dbTemplate = dbTemplates.get(dbUrl); URLTemplate mergedTemplate = tryMergeUrls(dbUrl, newUrl); if (mergedTemplate == null) { continue; } - if (areBothUuidUrls(newUrl,dbUrl,mergedTemplate) || dbTemplate.compare(newTemplate, mergedTemplate)) { + if (areBothUuidUrls(newUrl, dbUrl, mergedTemplate) + || dbTemplate.compare(newTemplate, mergedTemplate)) { Set similarTemplates = potentialMerges.get(mergedTemplate); if (similarTemplates == null) { similarTemplates = new HashSet<>(); potentialMerges.put(mergedTemplate, similarTemplates); - } + } similarTemplates.add(dbTemplate); countSimilarURLs++; - } + } } - + if (countSimilarURLs >= STRING_MERGING_THRESHOLD) { URLTemplate mergedTemplate = potentialMerges.keySet().iterator().next(); - RequestTemplate alreadyInDelta = deltaCatalog.getTemplateURLToMethods().get(mergedTemplate); - RequestTemplate dbTemplate = potentialMerges.get(mergedTemplate).iterator().next(); + RequestTemplate alreadyInDelta = + deltaCatalog.getTemplateURLToMethods().get(mergedTemplate); + RequestTemplate dbTemplate = + potentialMerges.get(mergedTemplate).iterator().next(); if (alreadyInDelta != null) { alreadyInDelta.mergeFrom(newTemplate); } else { RequestTemplate dbCopy = dbTemplate.copy(); - dbCopy.mergeFrom(newTemplate); + dbCopy.mergeFrom(newTemplate); deltaCatalog.getTemplateURLToMethods().put(mergedTemplate, dbCopy); } alreadyInDelta = deltaCatalog.getTemplateURLToMethods().get(mergedTemplate); - for (RequestTemplate rt: potentialMerges.getOrDefault(mergedTemplate, new HashSet<>())) { + for (RequestTemplate rt : + potentialMerges.getOrDefault(mergedTemplate, new HashSet<>())) { alreadyInDelta.mergeFrom(rt); deltaCatalog.getDeletedInfo().addAll(rt.getAllTypeInfo()); } deltaCatalog.getDeletedInfo().addAll(dbTemplate.getAllTypeInfo()); - + newUrlMatchedInDb = true; } @@ -616,29 +712,32 @@ private void tryMergingWithKnownStrictURLs( boolean newUrlMatchedInDelta = false; - for (URLStatic deltaUrl: deltaCatalog.getStrictURLToMethods().keySet()) { + for (URLStatic deltaUrl : deltaCatalog.getStrictURLToMethods().keySet()) { RequestTemplate deltaTemplate = deltaTemplates.get(deltaUrl); URLTemplate mergedTemplate = tryMergeUrls(deltaUrl, newUrl); - if (mergedTemplate == null || (RequestTemplate.isMergedOnStr(mergedTemplate) && !areBothUuidUrls(newUrl,deltaUrl,mergedTemplate))) { + if (mergedTemplate == null + || (RequestTemplate.isMergedOnStr(mergedTemplate) + && !areBothUuidUrls(newUrl, deltaUrl, mergedTemplate))) { continue; } newUrlMatchedInDelta = true; deltaCatalog.getDeletedInfo().addAll(deltaTemplate.getAllTypeInfo()); - RequestTemplate alreadyInDelta = deltaCatalog.getTemplateURLToMethods().get(mergedTemplate); + RequestTemplate alreadyInDelta = + deltaCatalog.getTemplateURLToMethods().get(mergedTemplate); if (alreadyInDelta != null) { alreadyInDelta.mergeFrom(newTemplate); } else { RequestTemplate deltaCopy = deltaTemplate.copy(); - deltaCopy.mergeFrom(newTemplate); + deltaCopy.mergeFrom(newTemplate); deltaCatalog.getTemplateURLToMethods().put(mergedTemplate, deltaCopy); } - + deltaCatalog.getStrictURLToMethods().remove(deltaUrl); break; } - + if (newUrlMatchedInDelta) { iterator.remove(); continue; @@ -654,17 +753,18 @@ private void tryMergingWithKnownStrictURLs( } } - public static boolean areBothUuidUrls(URLStatic newUrl, URLStatic deltaUrl, URLTemplate mergedTemplate) { + public static boolean areBothUuidUrls( + URLStatic newUrl, URLStatic deltaUrl, URLTemplate mergedTemplate) { Pattern pattern = patternToSubType.get(SingleTypeInfo.UUID); String[] n = tokenize(newUrl.getUrl()); String[] o = tokenize(deltaUrl.getUrl()); SuperType[] b = mergedTemplate.getTypes(); - for (int idx =0 ; idx < b.length; idx++) { + for (int idx = 0; idx < b.length; idx++) { SuperType c = b[idx]; if (Objects.equals(c, SuperType.STRING) && o.length > idx) { String val = n[idx]; - if(!pattern.matcher(val).matches() || !pattern.matcher(o[idx]).matches()) { + if (!pattern.matcher(val).matches() || !pattern.matcher(o[idx]).matches()) { return false; } } @@ -673,18 +773,22 @@ public static boolean areBothUuidUrls(URLStatic newUrl, URLStatic deltaUrl, URLT return true; } - public static boolean areBothMatchingUrls(URLStatic newUrl, URLStatic deltaUrl, URLTemplate mergedTemplate, boolean urlRegexMatchingEnabled) { + public static boolean areBothMatchingUrls( + URLStatic newUrl, + URLStatic deltaUrl, + URLTemplate mergedTemplate, + boolean urlRegexMatchingEnabled) { String[] n = tokenize(newUrl.getUrl()); String[] o = tokenize(deltaUrl.getUrl()); SuperType[] b = mergedTemplate.getTypes(); - for (int idx =0 ; idx < b.length; idx++) { + for (int idx = 0; idx < b.length; idx++) { SuperType c = b[idx]; if (Objects.equals(c, SuperType.STRING) && o.length > idx) { String val = n[idx]; boolean isAlphaNumeric = isAlphanumericString(val) && isAlphanumericString(o[idx]); - - if(!isAlphaNumeric) { + + if (!isAlphaNumeric) { return false; } } @@ -712,9 +816,9 @@ public static boolean isAlphanumericString(String s) { return (intCount >= 3 && charCount >= 1); } - - private static boolean isValidSubtype(SubType subType){ - return !(subType.getName().equals(SingleTypeInfo.GENERIC.getName()) || subType.getName().equals(SingleTypeInfo.OTHER.getName())); + private static boolean isValidSubtype(SubType subType) { + return !(subType.getName().equals(SingleTypeInfo.GENERIC.getName()) + || subType.getName().equals(SingleTypeInfo.OTHER.getName())); } public static boolean isNumber(String val) { @@ -731,7 +835,7 @@ public static boolean isNumber(String val) { } } - public static URLTemplate tryParamteresingUrl(URLStatic newUrl){ + public static URLTemplate tryParamteresingUrl(URLStatic newUrl) { String[] tokens = tokenize(newUrl.getUrl()); boolean tokensBelowThreshold = tokens.length < 2; Pattern pattern = patternToSubType.get(SingleTypeInfo.UUID); @@ -739,33 +843,33 @@ public static URLTemplate tryParamteresingUrl(URLStatic newUrl){ SuperType[] newTypes = new SuperType[tokens.length]; int start = newUrl.getUrl().startsWith("http") ? 3 : 0; - for(int i = start; i < tokens.length; i ++) { + for (int i = start; i < tokens.length; i++) { String tempToken = tokens[i]; - if(DictionaryFilter.isEnglishWord(tempToken)) continue; + if (DictionaryFilter.isEnglishWord(tempToken)) continue; if (NumberUtils.isParsable(tempToken)) { newTypes[i] = isNumber(tempToken) ? SuperType.INTEGER : SuperType.FLOAT; tokens[i] = null; - } else if(ObjectId.isValid(tempToken)){ + } else if (ObjectId.isValid(tempToken)) { newTypes[i] = SuperType.OBJECT_ID; tokens[i] = null; - }else if(pattern.matcher(tempToken).matches()){ + } else if (pattern.matcher(tempToken).matches()) { newTypes[i] = SuperType.STRING; tokens[i] = null; } - if(tokens[i] != null){ - SubType tempSubType = KeyTypes.findSubType(tokens[i], ""+i, null,true); - if(!tokensBelowThreshold && isValidSubtype(tempSubType)){ + if (tokens[i] != null) { + SubType tempSubType = KeyTypes.findSubType(tokens[i], "" + i, null, true); + if (!tokensBelowThreshold && isValidSubtype(tempSubType)) { newTypes[i] = SuperType.STRING; tokens[i] = null; - }else if(isAlphanumericString(tempToken)){ + } else if (isAlphanumericString(tempToken)) { newTypes[i] = SuperType.STRING; tokens[i] = null; } } - - if(newTypes[i] != null){ + + if (newTypes[i] != null) { allNull = false; } } @@ -775,20 +879,20 @@ public static URLTemplate tryParamteresingUrl(URLStatic newUrl){ URLTemplate urlTemplate = new URLTemplate(tokens, newTypes, newUrl.getMethod()); try { - for(MergedUrls mergedUrl : mergedUrls) { - if(mergedUrl.getUrl().equals(urlTemplate.getTemplateString()) && - mergedUrl.getMethod().equals(urlTemplate.getMethod().name())) { + for (MergedUrls mergedUrl : mergedUrls) { + if (mergedUrl.getUrl().equals(urlTemplate.getTemplateString()) + && mergedUrl.getMethod().equals(urlTemplate.getMethod().name())) { return null; } } - } catch(Exception e) { - loggerMaker.errorAndAddToDb("Error while creating a new URL object: " + e.getMessage(), LogDb.RUNTIME); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + "Error while creating a new URL object: " + e.getMessage(), LogDb.RUNTIME); } return urlTemplate; } - public static URLTemplate tryMergeUrls(URLStatic dbUrl, URLStatic newUrl) { if (dbUrl.getMethod() != newUrl.getMethod()) { return null; @@ -804,26 +908,28 @@ public static URLTemplate tryMergeUrls(URLStatic dbUrl, URLStatic newUrl) { SuperType[] newTypes = new SuperType[newTokens.length]; int templatizedStrTokens = 0; - for(int i = 0; i < newTokens.length; i ++) { + for (int i = 0; i < newTokens.length; i++) { String tempToken = newTokens[i]; String dbToken = dbTokens[i]; - if (DictionaryFilter.isEnglishWord(tempToken) || DictionaryFilter.isEnglishWord(dbToken)) continue; + if (DictionaryFilter.isEnglishWord(tempToken) + || DictionaryFilter.isEnglishWord(dbToken)) continue; - int minCount = dbUrl.getUrl().startsWith("http") && newUrl.getUrl().startsWith("http") ? 3 : 0; + int minCount = + dbUrl.getUrl().startsWith("http") && newUrl.getUrl().startsWith("http") ? 3 : 0; if (tempToken.equalsIgnoreCase(dbToken) || i < minCount) { continue; } - + if (NumberUtils.isParsable(tempToken) && NumberUtils.isParsable(dbToken)) { newTypes[i] = SuperType.INTEGER; newTokens[i] = null; - } else if(ObjectId.isValid(tempToken) && ObjectId.isValid(dbToken)){ + } else if (ObjectId.isValid(tempToken) && ObjectId.isValid(dbToken)) { newTypes[i] = SuperType.OBJECT_ID; newTokens[i] = null; - } else if(pattern.matcher(tempToken).matches() && pattern.matcher(dbToken).matches()){ + } else if (pattern.matcher(tempToken).matches() && pattern.matcher(dbToken).matches()) { newTypes[i] = SuperType.STRING; newTokens[i] = null; - }else if(isAlphanumericString(tempToken) && isAlphanumericString(dbToken)){ + } else if (isAlphanumericString(tempToken) && isAlphanumericString(dbToken)) { newTypes[i] = SuperType.STRING; newTokens[i] = null; } else { @@ -834,7 +940,7 @@ public static URLTemplate tryMergeUrls(URLStatic dbUrl, URLStatic newUrl) { } boolean allNull = true; - for (SingleTypeInfo.SuperType superType: newTypes) { + for (SingleTypeInfo.SuperType superType : newTypes) { allNull = allNull && (superType == null); } @@ -844,32 +950,44 @@ public static URLTemplate tryMergeUrls(URLStatic dbUrl, URLStatic newUrl) { URLTemplate urlTemplate = new URLTemplate(newTokens, newTypes, newUrl.getMethod()); try { - for(MergedUrls mergedUrl : mergedUrls) { - if(mergedUrl.getUrl().equals(urlTemplate.getTemplateString()) && - mergedUrl.getMethod().equals(urlTemplate.getMethod().name())) { + for (MergedUrls mergedUrl : mergedUrls) { + if (mergedUrl.getUrl().equals(urlTemplate.getTemplateString()) + && mergedUrl.getMethod().equals(urlTemplate.getMethod().name())) { return null; } } - } catch(Exception e) { - loggerMaker.errorAndAddToDb("Error while creating a new URL object: " + e.getMessage(), LogDb.RUNTIME); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + "Error while creating a new URL object: " + e.getMessage(), LogDb.RUNTIME); } return urlTemplate; } return null; - } - public static void mergeUrlsAndSave(int apiCollectionId, Boolean urlRegexMatchingEnabled, boolean mergeUrlsBasic, BloomFilter existingAPIsInDb,boolean ignoreCaseInsensitiveApis) { - if (apiCollectionId == LLM_API_COLLECTION_ID || apiCollectionId == VULNERABLE_API_COLLECTION_ID) return; - - ApiMergerResult result = tryMergeURLsInCollection(apiCollectionId, urlRegexMatchingEnabled, mergeUrlsBasic, existingAPIsInDb, ignoreCaseInsensitiveApis); + public static void mergeUrlsAndSave( + int apiCollectionId, + Boolean urlRegexMatchingEnabled, + boolean mergeUrlsBasic, + BloomFilter existingAPIsInDb, + boolean ignoreCaseInsensitiveApis) { + if (apiCollectionId == LLM_API_COLLECTION_ID + || apiCollectionId == VULNERABLE_API_COLLECTION_ID) return; + + ApiMergerResult result = + tryMergeURLsInCollection( + apiCollectionId, + urlRegexMatchingEnabled, + mergeUrlsBasic, + existingAPIsInDb, + ignoreCaseInsensitiveApis); String deletedStaticUrlsString = ""; int counter = 0; if (result.deleteStaticUrls != null) { - for (String dUrl: result.deleteStaticUrls) { + for (String dUrl : result.deleteStaticUrls) { if (counter >= 50) break; if (dUrl == null) continue; deletedStaticUrlsString += dUrl + ", "; @@ -880,18 +998,18 @@ public static void mergeUrlsAndSave(int apiCollectionId, Boolean urlRegexMatchin loggerMaker.debugInfoAddToDb("merged URLs: ", LogDb.RUNTIME); if (result.templateToStaticURLs != null) { - for (URLTemplate urlTemplate: result.templateToStaticURLs.keySet()) { + for (URLTemplate urlTemplate : result.templateToStaticURLs.keySet()) { String tempUrl = urlTemplate.getTemplateString() + " : "; counter = 0; if (result.templateToStaticURLs == null) continue; - for (String url: result.templateToStaticURLs.get(urlTemplate)) { + for (String url : result.templateToStaticURLs.get(urlTemplate)) { if (counter >= 5) break; if (url == null) continue; tempUrl += url + ", "; counter++; } - loggerMaker.debugInfoAddToDb( tempUrl, LogDb.RUNTIME); + loggerMaker.debugInfoAddToDb(tempUrl, LogDb.RUNTIME); } } @@ -900,17 +1018,20 @@ public static void mergeUrlsAndSave(int apiCollectionId, Boolean urlRegexMatchin ArrayList> bulkUpdatesForApiInfo = new ArrayList<>(); ArrayList> bulkUpdatesForDependencyNode = new ArrayList<>(); - for (URLTemplate urlTemplate: result.templateToStaticURLs.keySet()) { + for (URLTemplate urlTemplate : result.templateToStaticURLs.keySet()) { Set matchStaticURLs = result.templateToStaticURLs.get(urlTemplate); String newTemplateUrl = urlTemplate.getTemplateString(); if (!APICatalog.isTemplateUrl(newTemplateUrl)) continue; boolean isFirst = true; - for (String matchedURL: matchStaticURLs) { + for (String matchedURL : matchStaticURLs) { Method delMethod = Method.fromString(matchedURL.split(" ")[0]); String delEndpoint = matchedURL.split(" ")[1]; - Bson filterQ = SingleTypeInfoDao.filterForSTIUsingURL(apiCollectionId, delEndpoint, delMethod); - Bson filterQSampleData = SampleDataDao.filterForSampleData(apiCollectionId, delEndpoint, delMethod); + Bson filterQ = + SingleTypeInfoDao.filterForSTIUsingURL( + apiCollectionId, delEndpoint, delMethod); + Bson filterQSampleData = + SampleDataDao.filterForSampleData(apiCollectionId, delEndpoint, delMethod); if (isFirst) { @@ -918,23 +1039,46 @@ public static void mergeUrlsAndSave(int apiCollectionId, Boolean urlRegexMatchin SuperType superType = urlTemplate.getTypes()[i]; if (superType == null) continue; - int idx = delEndpoint.startsWith("http") ? i:i+1; + int idx = delEndpoint.startsWith("http") ? i : i + 1; String word = delEndpoint.split("/")[idx]; - SingleTypeInfo.ParamId stiId = new SingleTypeInfo.ParamId(newTemplateUrl, delMethod.name(), -1, false, i+"", SingleTypeInfo.GENERIC, apiCollectionId, true); - SubType tokenSubType = KeyTypes.findSubType(word, "", null,true); + SingleTypeInfo.ParamId stiId = + new SingleTypeInfo.ParamId( + newTemplateUrl, + delMethod.name(), + -1, + false, + i + "", + SingleTypeInfo.GENERIC, + apiCollectionId, + true); + SubType tokenSubType = KeyTypes.findSubType(word, "", null, true); stiId.setSubType(tokenSubType); - SingleTypeInfo sti = new SingleTypeInfo( - stiId, new HashSet<>(), new HashSet<>(), 0, Context.now(), 0, CappedSet.create(i+""), - SingleTypeInfo.Domain.ENUM, SingleTypeInfo.ACCEPTED_MIN_VALUE, SingleTypeInfo.ACCEPTED_MAX_VALUE); - + SingleTypeInfo sti = + new SingleTypeInfo( + stiId, + new HashSet<>(), + new HashSet<>(), + 0, + Context.now(), + 0, + CappedSet.create(i + ""), + SingleTypeInfo.Domain.ENUM, + SingleTypeInfo.ACCEPTED_MIN_VALUE, + SingleTypeInfo.ACCEPTED_MAX_VALUE); // SingleTypeInfoDao.instance.insertOne(sti); bulkUpdatesForSti.add(new InsertOneModel<>(sti)); } - SingleTypeInfoDao.instance.getMCollection().updateMany(filterQ, Updates.set("url", newTemplateUrl)); + SingleTypeInfoDao.instance + .getMCollection() + .updateMany(filterQ, Updates.set("url", newTemplateUrl)); - bulkUpdatesForSti.add(new UpdateManyModel<>(filterQ, Updates.set("url", newTemplateUrl), new UpdateOptions())); + bulkUpdatesForSti.add( + new UpdateManyModel<>( + filterQ, + Updates.set("url", newTemplateUrl), + new UpdateOptions())); SampleData sd = SampleDataDao.instance.findOne(filterQSampleData); if (sd != null) { @@ -943,7 +1087,6 @@ public static void mergeUrlsAndSave(int apiCollectionId, Boolean urlRegexMatchin bulkUpdatesForSampleData.add(new InsertOneModel<>(sd)); } - ApiInfo apiInfo = ApiInfoDao.instance.findOne(filterQSampleData); if (apiInfo != null) { apiInfo.getId().url = newTemplateUrl; @@ -961,18 +1104,20 @@ public static void mergeUrlsAndSave(int apiCollectionId, Boolean urlRegexMatchin bulkUpdatesForSampleData.add(new DeleteManyModel<>(filterQSampleData)); bulkUpdatesForApiInfo.add(new DeleteManyModel<>(filterQSampleData)); - Bson filterForDependencyNode = Filters.or( - Filters.and( - Filters.eq(DependencyNode.API_COLLECTION_ID_REQ, apiCollectionId+""), - Filters.eq(DependencyNode.URL_REQ, delEndpoint), - Filters.eq(DependencyNode.METHOD_REQ, delMethod.name()) - ), - Filters.and( - Filters.eq(DependencyNode.API_COLLECTION_ID_RESP, apiCollectionId+""), - Filters.eq(DependencyNode.URL_RESP, delEndpoint), - Filters.eq(DependencyNode.METHOD_RESP,delMethod.name()) - ) - ); + Bson filterForDependencyNode = + Filters.or( + Filters.and( + Filters.eq( + DependencyNode.API_COLLECTION_ID_REQ, + apiCollectionId + ""), + Filters.eq(DependencyNode.URL_REQ, delEndpoint), + Filters.eq(DependencyNode.METHOD_REQ, delMethod.name())), + Filters.and( + Filters.eq( + DependencyNode.API_COLLECTION_ID_RESP, + apiCollectionId + ""), + Filters.eq(DependencyNode.URL_RESP, delEndpoint), + Filters.eq(DependencyNode.METHOD_RESP, delMethod.name()))); bulkUpdatesForDependencyNode.add(new DeleteManyModel<>(filterForDependencyNode)); // SampleDataDao.instance.deleteAll(filterQSampleData); @@ -980,33 +1125,35 @@ public static void mergeUrlsAndSave(int apiCollectionId, Boolean urlRegexMatchin } } - for (String deleteStaticUrl: result.deleteStaticUrls) { + for (String deleteStaticUrl : result.deleteStaticUrls) { Method delMethod = Method.fromString(deleteStaticUrl.split(" ")[0]); - String delEndpoint = deleteStaticUrl.split(" ")[1]; - Bson filterQ = Filters.and( - Filters.eq("apiCollectionId", apiCollectionId), - Filters.eq("method", delMethod.name()), - Filters.eq("url", delEndpoint) - ); - - Bson filterQSampleData = Filters.and( - Filters.eq("_id.apiCollectionId", apiCollectionId), - Filters.eq("_id.method", delMethod.name()), - Filters.eq("_id.url", delEndpoint) - ); - - Bson filterForDependencyNode = Filters.or( + String delEndpoint = deleteStaticUrl.split(" ")[1]; + Bson filterQ = Filters.and( - Filters.eq(DependencyNode.API_COLLECTION_ID_REQ, apiCollectionId+""), - Filters.eq(DependencyNode.URL_REQ, delEndpoint), - Filters.eq(DependencyNode.METHOD_REQ, delMethod.name()) - ), + Filters.eq("apiCollectionId", apiCollectionId), + Filters.eq("method", delMethod.name()), + Filters.eq("url", delEndpoint)); + + Bson filterQSampleData = Filters.and( - Filters.eq(DependencyNode.API_COLLECTION_ID_RESP, apiCollectionId+""), - Filters.eq(DependencyNode.URL_RESP, delEndpoint), - Filters.eq(DependencyNode.METHOD_RESP,delMethod.name()) - ) - ); + Filters.eq("_id.apiCollectionId", apiCollectionId), + Filters.eq("_id.method", delMethod.name()), + Filters.eq("_id.url", delEndpoint)); + + Bson filterForDependencyNode = + Filters.or( + Filters.and( + Filters.eq( + DependencyNode.API_COLLECTION_ID_REQ, + apiCollectionId + ""), + Filters.eq(DependencyNode.URL_REQ, delEndpoint), + Filters.eq(DependencyNode.METHOD_REQ, delMethod.name())), + Filters.and( + Filters.eq( + DependencyNode.API_COLLECTION_ID_RESP, + apiCollectionId + ""), + Filters.eq(DependencyNode.URL_RESP, delEndpoint), + Filters.eq(DependencyNode.METHOD_RESP, delMethod.name()))); bulkUpdatesForDependencyNode.add(new DeleteManyModel<>(filterForDependencyNode)); bulkUpdatesForSti.add(new DeleteManyModel<>(filterQ)); @@ -1017,58 +1164,76 @@ public static void mergeUrlsAndSave(int apiCollectionId, Boolean urlRegexMatchin if (bulkUpdatesForSti.size() > 0) { try { - SingleTypeInfoDao.instance.getMCollection().bulkWrite(bulkUpdatesForSti, new BulkWriteOptions().ordered(false)); + SingleTypeInfoDao.instance + .getMCollection() + .bulkWrite(bulkUpdatesForSti, new BulkWriteOptions().ordered(false)); } catch (Exception e) { - loggerMaker.errorAndAddToDb("STI bulkWrite error: " + e.getMessage(),LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + "STI bulkWrite error: " + e.getMessage(), LogDb.RUNTIME); } } if (bulkUpdatesForSampleData.size() > 0) { try { - SampleDataDao.instance.getMCollection().bulkWrite(bulkUpdatesForSampleData, new BulkWriteOptions().ordered(false)); + SampleDataDao.instance + .getMCollection() + .bulkWrite(bulkUpdatesForSampleData, new BulkWriteOptions().ordered(false)); } catch (Exception e) { - loggerMaker.errorAndAddToDb("SampleData bulkWrite error: " + e.getMessage(),LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + "SampleData bulkWrite error: " + e.getMessage(), LogDb.RUNTIME); } } if (bulkUpdatesForApiInfo.size() > 0) { try { - ApiInfoDao.instance.getMCollection().bulkWrite(bulkUpdatesForApiInfo, new BulkWriteOptions().ordered(false)); + ApiInfoDao.instance + .getMCollection() + .bulkWrite(bulkUpdatesForApiInfo, new BulkWriteOptions().ordered(false)); } catch (Exception e) { - loggerMaker.errorAndAddToDb("ApiInfo bulkWrite error: " + e.getMessage(),LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + "ApiInfo bulkWrite error: " + e.getMessage(), LogDb.RUNTIME); } } if (bulkUpdatesForDependencyNode.size() > 0) { - BulkWriteResult bulkWriteResult = DependencyNodeDao.instance.getMCollection().bulkWrite(bulkUpdatesForDependencyNode, new BulkWriteOptions().ordered(false)); + BulkWriteResult bulkWriteResult = + DependencyNodeDao.instance + .getMCollection() + .bulkWrite( + bulkUpdatesForDependencyNode, + new BulkWriteOptions().ordered(false)); } } private void tryWithKnownURLTemplates( - Map pendingRequests, - APICatalog deltaCatalog, - APICatalog dbCatalog, - int apiCollectionId - ) { - Iterator> iterator = pendingRequests.entrySet().iterator(); + Map pendingRequests, + APICatalog deltaCatalog, + APICatalog dbCatalog, + int apiCollectionId) { + Iterator> iterator = + pendingRequests.entrySet().iterator(); try { while (iterator.hasNext()) { Map.Entry entry = iterator.next(); URLStatic newUrl = entry.getKey(); RequestTemplate newRequestTemplate = entry.getValue(); - for (URLTemplate urlTemplate: dbCatalog.getTemplateURLToMethods().keySet()) { + for (URLTemplate urlTemplate : dbCatalog.getTemplateURLToMethods().keySet()) { if (urlTemplate.match(newUrl)) { - RequestTemplate alreadyInDelta = deltaCatalog.getTemplateURLToMethods().get(urlTemplate); + RequestTemplate alreadyInDelta = + deltaCatalog.getTemplateURLToMethods().get(urlTemplate); if (alreadyInDelta != null) { - alreadyInDelta.fillUrlParams(tokenize(newUrl.getUrl()), urlTemplate, apiCollectionId); + alreadyInDelta.fillUrlParams( + tokenize(newUrl.getUrl()), urlTemplate, apiCollectionId); alreadyInDelta.mergeFrom(newRequestTemplate); } else { - RequestTemplate dbTemplate = dbCatalog.getTemplateURLToMethods().get(urlTemplate); + RequestTemplate dbTemplate = + dbCatalog.getTemplateURLToMethods().get(urlTemplate); RequestTemplate dbCopy = dbTemplate.copy(); dbCopy.mergeFrom(newRequestTemplate); - dbCopy.fillUrlParams(tokenize(newUrl.getUrl()), urlTemplate, apiCollectionId); + dbCopy.fillUrlParams( + tokenize(newUrl.getUrl()), urlTemplate, apiCollectionId); deltaCatalog.getTemplateURLToMethods().put(urlTemplate, dbCopy); } iterator.remove(); @@ -1081,11 +1246,11 @@ private void tryWithKnownURLTemplates( } } - private Map createRequestTemplates(URLAggregator aggregator) { Map ret = new HashMap<>(); List deletedInfo = new ArrayList<>(); - Iterator>> iterator = aggregator.urls.entrySet().iterator(); + Iterator>> iterator = + aggregator.urls.entrySet().iterator(); try { while (iterator.hasNext()) { Map.Entry> entry = iterator.next(); @@ -1093,7 +1258,12 @@ private Map createRequestTemplates(URLAggregator agg Set responseParamsList = entry.getValue(); RequestTemplate requestTemplate = ret.get(url); if (requestTemplate == null) { - requestTemplate = new RequestTemplate(new HashMap<>(), new HashMap<>(), new HashMap<>(), new TrafficRecorder(new HashMap<>())); + requestTemplate = + new RequestTemplate( + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + new TrafficRecorder(new HashMap<>())); ret.put(url, requestTemplate); } processResponse(requestTemplate, responseParamsList, deletedInfo); @@ -1106,8 +1276,13 @@ private Map createRequestTemplates(URLAggregator agg return ret; } - private void processKnownStaticURLs(URLAggregator aggregator, APICatalog deltaCatalog, APICatalog dbCatalog, boolean makeApisCaseInsensitive) { - Iterator>> iterator = aggregator.urls.entrySet().iterator(); + private void processKnownStaticURLs( + URLAggregator aggregator, + APICatalog deltaCatalog, + APICatalog dbCatalog, + boolean makeApisCaseInsensitive) { + Iterator>> iterator = + aggregator.urls.entrySet().iterator(); List deletedInfo = deltaCatalog.getDeletedInfo(); // handle case insensitive apis here in the same aggregator @@ -1120,8 +1295,8 @@ private void processKnownStaticURLs(URLAggregator aggregator, APICatalog deltaCa Set responseParamsList = entry.getValue(); String endpoint = url.getUrl(); - if(makeApisCaseInsensitive){ - if(lowerCaseApisSet.contains(endpoint.toLowerCase())){ + if (makeApisCaseInsensitive) { + if (lowerCaseApisSet.contains(endpoint.toLowerCase())) { iterator.remove(); continue; } @@ -1130,40 +1305,44 @@ private void processKnownStaticURLs(URLAggregator aggregator, APICatalog deltaCa RequestTemplate strictMatch = dbCatalog.getStrictURLToMethods().get(url); if (strictMatch != null) { - Map deltaCatalogStrictURLToMethods = deltaCatalog.getStrictURLToMethods(); + Map deltaCatalogStrictURLToMethods = + deltaCatalog.getStrictURLToMethods(); RequestTemplate requestTemplate = deltaCatalogStrictURLToMethods.get(url); if (requestTemplate == null) { - requestTemplate = strictMatch.copy(); // to further process the requestTemplate - deltaCatalogStrictURLToMethods.put(url, requestTemplate) ; - strictMatch.mergeFrom(requestTemplate); // to update the existing requestTemplate in db with new data + requestTemplate = + strictMatch.copy(); // to further process the requestTemplate + deltaCatalogStrictURLToMethods.put(url, requestTemplate); + strictMatch.mergeFrom( + requestTemplate); // to update the existing requestTemplate in db + // with new data } processResponse(requestTemplate, responseParamsList, deletedInfo); iterator.remove(); } - } } catch (Exception e) { - loggerMaker.errorAndAddToDb(e.toString(),LogDb.RUNTIME); + loggerMaker.errorAndAddToDb(e.toString(), LogDb.RUNTIME); } } public static String trim(String url) { // if (mergeAsyncOutside) { - // if ( !(url.startsWith("/") ) && !( url.startsWith("http") || url.startsWith("ftp")) ){ + // if ( !(url.startsWith("/") ) && !( url.startsWith("http") || url.startsWith("ftp")) + // ){ // url = "/" + url; // } // } else { - if (url.startsWith("/")) url = url.substring(1, url.length()); + if (url.startsWith("/")) url = url.substring(1, url.length()); // } - - if (url.endsWith("/")) url = url.substring(0, url.length()-1); + + if (url.endsWith("/")) url = url.substring(0, url.length() - 1); return url; } private Map> groupByTokenSize(APICatalog catalog) { Map> sizeToURL = new HashMap<>(); - for(URLStatic rawURL: catalog.getStrictURLToMethods().keySet()) { + for (URLStatic rawURL : catalog.getStrictURLToMethods().keySet()) { RequestTemplate reqTemplate = catalog.getStrictURLToMethods().get(rawURL); if (reqTemplate.getUserIds().size() < 5) { String url = trim(rawURL.getUrl()); @@ -1198,105 +1377,160 @@ Map convertToMap(List l) { return ret; } - public ArrayList> getDBUpdatesForSampleData(int apiCollectionId, APICatalog currentDelta, APICatalog dbCatalog, boolean forceUpdate, boolean accountLevelRedact, boolean apiCollectionLevelRedact) { + public ArrayList> getDBUpdatesForSampleData( + int apiCollectionId, + APICatalog currentDelta, + APICatalog dbCatalog, + boolean forceUpdate, + boolean accountLevelRedact, + boolean apiCollectionLevelRedact) { List sampleData = new ArrayList<>(); - Map deltaStrictURLToMethods = currentDelta.getStrictURLToMethods(); + Map deltaStrictURLToMethods = + currentDelta.getStrictURLToMethods(); Map dbStrictURLToMethods = dbCatalog.getStrictURLToMethods(); - for(Map.Entry entry: deltaStrictURLToMethods.entrySet()) { + for (Map.Entry entry : deltaStrictURLToMethods.entrySet()) { if (forceUpdate || !dbStrictURLToMethods.containsKey(entry.getKey())) { - Key key = new Key(apiCollectionId, entry.getKey().getUrl(), entry.getKey().getMethod(), -1, 0, 0); + Key key = + new Key( + apiCollectionId, + entry.getKey().getUrl(), + entry.getKey().getMethod(), + -1, + 0, + 0); sampleData.add(new SampleData(key, entry.getValue().removeAllSampleMessage())); } } - Map deltaTemplateURLToMethods = currentDelta.getTemplateURLToMethods(); - Map dbTemplateURLToMethods = dbCatalog.getTemplateURLToMethods(); + Map deltaTemplateURLToMethods = + currentDelta.getTemplateURLToMethods(); + Map dbTemplateURLToMethods = + dbCatalog.getTemplateURLToMethods(); - for(Map.Entry entry: deltaTemplateURLToMethods.entrySet()) { + for (Map.Entry entry : deltaTemplateURLToMethods.entrySet()) { if (forceUpdate || !dbTemplateURLToMethods.containsKey(entry.getKey())) { - Key key = new Key(apiCollectionId, entry.getKey().getTemplateString(), entry.getKey().getMethod(), -1, 0, 0); + Key key = + new Key( + apiCollectionId, + entry.getKey().getTemplateString(), + entry.getKey().getMethod(), + -1, + 0, + 0); sampleData.add(new SampleData(key, entry.getValue().removeAllSampleMessage())); } } ArrayList> bulkUpdates = new ArrayList<>(); - for (SampleData sample: sampleData) { + for (SampleData sample : sampleData) { if (sample.getSamples().size() == 0) { continue; } List finalSamples = new ArrayList<>(); - for (String s: sample.getSamples()) { + for (String s : sample.getSamples()) { try { - String redactedSample = RedactSampleData.redactIfRequired(s, accountLevelRedact, apiCollectionLevelRedact); + String redactedSample = + RedactSampleData.redactIfRequired( + s, accountLevelRedact, apiCollectionLevelRedact); finalSamples.add(redactedSample); } catch (Exception e) { - loggerMaker.errorAndAddToDb(e,"Error while redacting data" , LogDb.RUNTIME) - ; + loggerMaker.errorAndAddToDb(e, "Error while redacting data", LogDb.RUNTIME); } } - Bson bson = Updates.combine( - Updates.pushEach("samples", finalSamples, new PushOptions().slice(-10)), - Updates.setOnInsert(SingleTypeInfo._COLLECTION_IDS, Arrays.asList(sample.getId().getApiCollectionId())) - ); + Bson bson = + Updates.combine( + Updates.pushEach("samples", finalSamples, new PushOptions().slice(-10)), + Updates.setOnInsert( + SingleTypeInfo._COLLECTION_IDS, + Arrays.asList(sample.getId().getApiCollectionId()))); bulkUpdates.add( - new UpdateOneModel<>(Filters.eq("_id", sample.getId()), bson, new UpdateOptions().upsert(true)) - ); + new UpdateOneModel<>( + Filters.eq("_id", sample.getId()), + bson, + new UpdateOptions().upsert(true))); } - return bulkUpdates; + return bulkUpdates; } - - - - public ArrayList> getDBUpdatesForTraffic(int apiCollectionId, APICatalog currentDelta) { + public ArrayList> getDBUpdatesForTraffic( + int apiCollectionId, APICatalog currentDelta) { List trafficInfos = new ArrayList<>(); - for(Map.Entry entry: currentDelta.getStrictURLToMethods().entrySet()) { - trafficInfos.addAll(entry.getValue().removeAllTrafficInfo(apiCollectionId, entry.getKey().getUrl(), entry.getKey().getMethod(), -1)); - } - - for(Map.Entry entry: currentDelta.getTemplateURLToMethods().entrySet()) { - trafficInfos.addAll(entry.getValue().removeAllTrafficInfo(apiCollectionId, entry.getKey().getTemplateString(), entry.getKey().getMethod(), -1)); + for (Map.Entry entry : + currentDelta.getStrictURLToMethods().entrySet()) { + trafficInfos.addAll( + entry.getValue() + .removeAllTrafficInfo( + apiCollectionId, + entry.getKey().getUrl(), + entry.getKey().getMethod(), + -1)); + } + + for (Map.Entry entry : + currentDelta.getTemplateURLToMethods().entrySet()) { + trafficInfos.addAll( + entry.getValue() + .removeAllTrafficInfo( + apiCollectionId, + entry.getKey().getTemplateString(), + entry.getKey().getMethod(), + -1)); } ArrayList> bulkUpdates = new ArrayList<>(); - for (TrafficInfo trafficInfo: trafficInfos) { + for (TrafficInfo trafficInfo : trafficInfos) { List updates = new ArrayList<>(); - for (Map.Entry entry: trafficInfo.mapHoursToCount.entrySet()) { - updates.add(Updates.inc("mapHoursToCount."+entry.getKey(), entry.getValue())); + for (Map.Entry entry : trafficInfo.mapHoursToCount.entrySet()) { + updates.add(Updates.inc("mapHoursToCount." + entry.getKey(), entry.getValue())); } - updates.add(Updates.setOnInsert(SingleTypeInfo._COLLECTION_IDS, Arrays.asList(trafficInfo.getId().getApiCollectionId()))); + updates.add( + Updates.setOnInsert( + SingleTypeInfo._COLLECTION_IDS, + Arrays.asList(trafficInfo.getId().getApiCollectionId()))); bulkUpdates.add( - new UpdateOneModel<>(Filters.eq("_id", trafficInfo.getId()), Updates.combine(updates), new UpdateOptions().upsert(true)) - ); + new UpdateOneModel<>( + Filters.eq("_id", trafficInfo.getId()), + Updates.combine(updates), + new UpdateOptions().upsert(true))); } return bulkUpdates; } - public DbUpdateReturn getDBUpdatesForParams(APICatalog currentDelta, APICatalog currentState, boolean redactSampleData, boolean collectionLevelRedact) { + public DbUpdateReturn getDBUpdatesForParams( + APICatalog currentDelta, + APICatalog currentState, + boolean redactSampleData, + boolean collectionLevelRedact) { Map dbInfoMap = convertToMap(currentState.getAllTypeInfo()); Map deltaInfoMap = convertToMap(currentDelta.getAllTypeInfo()); ArrayList> bulkUpdates = new ArrayList<>(); ArrayList> bulkUpdatesForSampleData = new ArrayList<>(); int now = Context.now(); - for(String key: deltaInfoMap.keySet()) { + for (String key : deltaInfoMap.keySet()) { SingleTypeInfo dbInfo = dbInfoMap.get(key); SingleTypeInfo deltaInfo = deltaInfoMap.get(key); Bson update; int inc = deltaInfo.getCount() - (dbInfo == null ? 0 : dbInfo.getCount()); - long lastSeenDiff = deltaInfo.getLastSeen() - (dbInfo == null ? 0 : dbInfo.getLastSeen()); - boolean minMaxChanged = (dbInfo == null) || (dbInfo.getMinValue() != deltaInfo.getMinValue()) || (dbInfo.getMaxValue() != deltaInfo.getMaxValue()); - boolean valuesChanged = (dbInfo == null) || (dbInfo.getValues().count() != deltaInfo.getValues().count()); - - if (inc == 0 && lastSeenDiff < (60*30) && !minMaxChanged && !valuesChanged) { + long lastSeenDiff = + deltaInfo.getLastSeen() - (dbInfo == null ? 0 : dbInfo.getLastSeen()); + boolean minMaxChanged = + (dbInfo == null) + || (dbInfo.getMinValue() != deltaInfo.getMinValue()) + || (dbInfo.getMaxValue() != deltaInfo.getMaxValue()); + boolean valuesChanged = + (dbInfo == null) + || (dbInfo.getValues().count() != deltaInfo.getValues().count()); + + if (inc == 0 && lastSeenDiff < (60 * 30) && !minMaxChanged && !valuesChanged) { continue; } else { inc = 1; @@ -1309,14 +1543,20 @@ public DbUpdateReturn getDBUpdatesForParams(APICatalog currentDelta, APICatalog int timestamp = deltaInfo.getTimestamp() > 0 ? deltaInfo.getTimestamp() : now; update = Updates.combine(update, Updates.setOnInsert("timestamp", timestamp)); - update = Updates.combine(update, Updates.max(SingleTypeInfo.LAST_SEEN, deltaInfo.getLastSeen())); - update = Updates.combine(update, Updates.max(SingleTypeInfo.MAX_VALUE, deltaInfo.getMaxValue())); - update = Updates.combine(update, Updates.min(SingleTypeInfo.MIN_VALUE, deltaInfo.getMinValue())); + update = + Updates.combine( + update, Updates.max(SingleTypeInfo.LAST_SEEN, deltaInfo.getLastSeen())); + update = + Updates.combine( + update, Updates.max(SingleTypeInfo.MAX_VALUE, deltaInfo.getMaxValue())); + update = + Updates.combine( + update, Updates.min(SingleTypeInfo.MIN_VALUE, deltaInfo.getMinValue())); if (!Main.isOnprem) { if (dbInfo != null) { SingleTypeInfo.Domain domain = dbInfo.getDomain(); - if (domain == SingleTypeInfo.Domain.ENUM) { + if (domain == SingleTypeInfo.Domain.ENUM) { CappedSet values = dbInfo.getValues(); Set elements = new HashSet<>(); if (values != null) { @@ -1325,18 +1565,25 @@ public DbUpdateReturn getDBUpdatesForParams(APICatalog currentDelta, APICatalog int valuesSize = elements.size(); if (valuesSize >= SingleTypeInfo.VALUES_LIMIT) { SingleTypeInfo.Domain newDomain; - if (dbInfo.getSubType().equals(SingleTypeInfo.INTEGER_32) || dbInfo.getSubType().equals(SingleTypeInfo.INTEGER_64) || dbInfo.getSubType().equals(SingleTypeInfo.FLOAT)) { + if (dbInfo.getSubType().equals(SingleTypeInfo.INTEGER_32) + || dbInfo.getSubType().equals(SingleTypeInfo.INTEGER_64) + || dbInfo.getSubType().equals(SingleTypeInfo.FLOAT)) { newDomain = SingleTypeInfo.Domain.RANGE; } else { newDomain = SingleTypeInfo.Domain.ANY; } - update = Updates.combine(update, Updates.set(SingleTypeInfo._DOMAIN, newDomain)); + update = + Updates.combine( + update, Updates.set(SingleTypeInfo._DOMAIN, newDomain)); } } else { deltaInfo.setDomain(dbInfo.getDomain()); deltaInfo.setValues(new CappedSet<>()); if (!dbInfo.getValues().getElements().isEmpty()) { - Bson bson = Updates.set(SingleTypeInfo._VALUES +".elements",new ArrayList<>()); + Bson bson = + Updates.set( + SingleTypeInfo._VALUES + ".elements", + new ArrayList<>()); update = Updates.combine(update, bson); } } @@ -1345,24 +1592,29 @@ public DbUpdateReturn getDBUpdatesForParams(APICatalog currentDelta, APICatalog CappedSet values = deltaInfo.getValues(); if (values != null) { Set elements = new HashSet<>(); - for (String el: values.getElements()) { + for (String el : values.getElements()) { if (redactSampleData) { - elements.add(el.hashCode()+""); + elements.add(el.hashCode() + ""); } else { elements.add(el); } } - Bson bson = Updates.addEachToSet(SingleTypeInfo._VALUES +".elements",new ArrayList<>(elements)); + Bson bson = + Updates.addEachToSet( + SingleTypeInfo._VALUES + ".elements", + new ArrayList<>(elements)); update = Updates.combine(update, bson); deltaInfo.setValues(new CappedSet<>()); } } } - if (!(redactSampleData || collectionLevelRedact) && deltaInfo.getExamples() != null && !deltaInfo.getExamples().isEmpty()) { + if (!(redactSampleData || collectionLevelRedact) + && deltaInfo.getExamples() != null + && !deltaInfo.getExamples().isEmpty()) { Set updatedSampleData = new HashSet<>(); for (Object example : deltaInfo.getExamples()) { - try{ + try { String exampleStr = (String) example; String s = RedactSampleData.redactDataTypes(exampleStr); updatedSampleData.add(s); @@ -1371,64 +1623,79 @@ public DbUpdateReturn getDBUpdatesForParams(APICatalog currentDelta, APICatalog } } deltaInfo.setExamples(updatedSampleData); - Bson bson = Updates.combine( - Updates.pushEach(SensitiveSampleData.SAMPLE_DATA, Arrays.asList(deltaInfo.getExamples().toArray()), new PushOptions().slice(-1 *SensitiveSampleData.cap)), - Updates.setOnInsert(SingleTypeInfo._COLLECTION_IDS, Arrays.asList(deltaInfo.getApiCollectionId())) - ); + Bson bson = + Updates.combine( + Updates.pushEach( + SensitiveSampleData.SAMPLE_DATA, + Arrays.asList(deltaInfo.getExamples().toArray()), + new PushOptions().slice(-1 * SensitiveSampleData.cap)), + Updates.setOnInsert( + SingleTypeInfo._COLLECTION_IDS, + Arrays.asList(deltaInfo.getApiCollectionId()))); bulkUpdatesForSampleData.add( new UpdateOneModel<>( SensitiveSampleDataDao.getFilters(deltaInfo), bson, - new UpdateOptions().upsert(true) - ) - ); + new UpdateOptions().upsert(true))); } Bson updateKey = SingleTypeInfoDao.createFilters(deltaInfo); - update = Updates.combine(update, - Updates.setOnInsert(SingleTypeInfo._COLLECTION_IDS, Arrays.asList(deltaInfo.getApiCollectionId()))); + update = + Updates.combine( + update, + Updates.setOnInsert( + SingleTypeInfo._COLLECTION_IDS, + Arrays.asList(deltaInfo.getApiCollectionId()))); - bulkUpdates.add(new UpdateOneModel<>(updateKey, update, new UpdateOptions().upsert(true))); + bulkUpdates.add( + new UpdateOneModel<>(updateKey, update, new UpdateOptions().upsert(true))); } - for(SingleTypeInfo deleted: currentDelta.getDeletedInfo()) { - currentDelta.getStrictURLToMethods().remove(new URLStatic(deleted.getUrl(), Method.fromString(deleted.getMethod()))); - bulkUpdates.add(new DeleteOneModel<>(SingleTypeInfoDao.createFilters(deleted), new DeleteOptions())); - bulkUpdatesForSampleData.add(new DeleteOneModel<>(SensitiveSampleDataDao.getFilters(deleted), new DeleteOptions())); + for (SingleTypeInfo deleted : currentDelta.getDeletedInfo()) { + currentDelta + .getStrictURLToMethods() + .remove( + new URLStatic( + deleted.getUrl(), Method.fromString(deleted.getMethod()))); + bulkUpdates.add( + new DeleteOneModel<>( + SingleTypeInfoDao.createFilters(deleted), new DeleteOptions())); + bulkUpdatesForSampleData.add( + new DeleteOneModel<>( + SensitiveSampleDataDao.getFilters(deleted), new DeleteOptions())); } - - ArrayList> bulkUpdatesForSensitiveParamInfo = new ArrayList<>(); - for (SensitiveParamInfo sensitiveParamInfo: sensitiveParamInfoBooleanMap.keySet()) { + ArrayList> bulkUpdatesForSensitiveParamInfo = + new ArrayList<>(); + for (SensitiveParamInfo sensitiveParamInfo : sensitiveParamInfoBooleanMap.keySet()) { if (!sensitiveParamInfoBooleanMap.get(sensitiveParamInfo)) continue; bulkUpdatesForSensitiveParamInfo.add( new UpdateOneModel( SensitiveParamInfoDao.getFilters(sensitiveParamInfo), Updates.set(SensitiveParamInfo.SAMPLE_DATA_SAVED, true), - new UpdateOptions().upsert(false) - ) - ); + new UpdateOptions().upsert(false))); } - return new DbUpdateReturn(bulkUpdates, bulkUpdatesForSampleData, bulkUpdatesForSensitiveParamInfo); + return new DbUpdateReturn( + bulkUpdates, bulkUpdatesForSampleData, bulkUpdatesForSensitiveParamInfo); } public static class DbUpdateReturn { public ArrayList> bulkUpdatesForSingleTypeInfo; public ArrayList> bulkUpdatesForSampleData; - public ArrayList> bulkUpdatesForSensitiveParamInfo = new ArrayList<>(); + public ArrayList> bulkUpdatesForSensitiveParamInfo = + new ArrayList<>(); - public DbUpdateReturn(ArrayList> bulkUpdatesForSingleTypeInfo, - ArrayList> bulkUpdatesForSampleData, - ArrayList> bulkUpdatesForSensitiveParamInfo - ) { + public DbUpdateReturn( + ArrayList> bulkUpdatesForSingleTypeInfo, + ArrayList> bulkUpdatesForSampleData, + ArrayList> bulkUpdatesForSensitiveParamInfo) { this.bulkUpdatesForSingleTypeInfo = bulkUpdatesForSingleTypeInfo; this.bulkUpdatesForSampleData = bulkUpdatesForSampleData; this.bulkUpdatesForSensitiveParamInfo = bulkUpdatesForSensitiveParamInfo; } } - public static String[] trimAndSplit(String url) { return trim(url).split("/"); } @@ -1436,7 +1703,7 @@ public static String[] trimAndSplit(String url) { public static URLTemplate createUrlTemplate(String url, Method method) { String[] tokens = trimAndSplit(url); SuperType[] types = new SuperType[tokens.length]; - for(int i = 0; i < tokens.length; i ++ ) { + for (int i = 0; i < tokens.length; i++) { String token = tokens[i]; if (token.equals(SuperType.STRING.name())) { @@ -1454,7 +1721,6 @@ public static URLTemplate createUrlTemplate(String url, Method method) { } else { types[i] = null; } - } URLTemplate urlTemplate = new URLTemplate(tokens, types, method); @@ -1472,43 +1738,59 @@ public static void clearValuesInDB() { int sliceLimit = -1 * SingleTypeInfo.VALUES_LIMIT; // range update - UpdateResult rangeUpdateResult = SingleTypeInfoDao.instance.updateMany( - Filters.and( - Filters.exists(fieldName, true), - Filters.in(SingleTypeInfo.SUB_TYPE, rangeSubTypes) - ), - Updates.combine( - Updates.pushEach("values.elements", new ArrayList<>(), new PushOptions().slice(sliceLimit)), - Updates.set(SingleTypeInfo._DOMAIN, Domain.RANGE.name()) - ) - ); - loggerMaker.infoAndAddToDb("RangeUpdateResult for clearValuesInDb function = " + "match count: " + rangeUpdateResult.getMatchedCount() + ", modify count: " + rangeUpdateResult.getModifiedCount(), LogDb.RUNTIME); + UpdateResult rangeUpdateResult = + SingleTypeInfoDao.instance.updateMany( + Filters.and( + Filters.exists(fieldName, true), + Filters.in(SingleTypeInfo.SUB_TYPE, rangeSubTypes)), + Updates.combine( + Updates.pushEach( + "values.elements", + new ArrayList<>(), + new PushOptions().slice(sliceLimit)), + Updates.set(SingleTypeInfo._DOMAIN, Domain.RANGE.name()))); + loggerMaker.infoAndAddToDb( + "RangeUpdateResult for clearValuesInDb function = " + + "match count: " + + rangeUpdateResult.getMatchedCount() + + ", modify count: " + + rangeUpdateResult.getModifiedCount(), + LogDb.RUNTIME); // any update - UpdateResult anyUpdateResult = SingleTypeInfoDao.instance.updateMany( - Filters.and( - Filters.exists(fieldName, true), - Filters.nin(SingleTypeInfo.SUB_TYPE, rangeSubTypes) - ), - Updates.combine( - Updates.pushEach("values.elements", new ArrayList<>(), new PushOptions().slice(sliceLimit)), - Updates.set(SingleTypeInfo._DOMAIN, Domain.ANY.name()) - ) - ); - - loggerMaker.infoAndAddToDb("AnyUpdateResult for clearValuesInDb function = " + "match count: " + anyUpdateResult.getMatchedCount() + ", modify count: " + anyUpdateResult.getModifiedCount(), LogDb.RUNTIME); + UpdateResult anyUpdateResult = + SingleTypeInfoDao.instance.updateMany( + Filters.and( + Filters.exists(fieldName, true), + Filters.nin(SingleTypeInfo.SUB_TYPE, rangeSubTypes)), + Updates.combine( + Updates.pushEach( + "values.elements", + new ArrayList<>(), + new PushOptions().slice(sliceLimit)), + Updates.set(SingleTypeInfo._DOMAIN, Domain.ANY.name()))); + + loggerMaker.infoAndAddToDb( + "AnyUpdateResult for clearValuesInDb function = " + + "match count: " + + anyUpdateResult.getMatchedCount() + + ", modify count: " + + anyUpdateResult.getModifiedCount(), + LogDb.RUNTIME); } private int lastMergeAsyncOutsideTs = 0; + public void buildFromDB(boolean calcDiff, boolean fetchAllSTI) { loggerMaker.infoAndAddToDb("Started building from dB", LogDb.RUNTIME); boolean mergingCalled = false; demosAndDeactivatedCollections = UsageMetricCalculator.getDemosAndDeactivated(); - existingAPIsInDb = BloomFilter.create(Funnels.stringFunnel(Charsets.UTF_8), 1_000_000, 0.001 ); + existingAPIsInDb = + BloomFilter.create(Funnels.stringFunnel(Charsets.UTF_8), 1_000_000, 0.001); - if (mergeAsyncOutside && fetchAllSTI ) { + if (mergeAsyncOutside && fetchAllSTI) { if (Context.now() - lastMergeAsyncOutsideTs > 600) { loggerMaker.infoAndAddToDb("Started mergeAsyncOutside", LogDb.RUNTIME); this.lastMergeAsyncOutsideTs = Context.now(); @@ -1517,55 +1799,99 @@ public void buildFromDB(boolean calcDiff, boolean fetchAllSTI) { if (gotDibs) { loggerMaker.infoAndAddToDb("Got dibs", LogDb.RUNTIME); mergingCalled = true; - BackwardCompatibility backwardCompatibility = BackwardCompatibilityDao.instance.findOne(new BasicDBObject()); - if (backwardCompatibility == null || backwardCompatibility.getMergeOnHostInit() == 0) { + BackwardCompatibility backwardCompatibility = + BackwardCompatibilityDao.instance.findOne(new BasicDBObject()); + if (backwardCompatibility == null + || backwardCompatibility.getMergeOnHostInit() == 0) { loggerMaker.infoAndAddToDb("Merging hosts...", LogDb.RUNTIME); new MergeOnHostOnly().mergeHosts(); - Bson update = Updates.set(BackwardCompatibility.MERGE_ON_HOST_INIT, Context.now()); - BackwardCompatibilityDao.instance.getMCollection().updateMany(new BasicDBObject(), update); + Bson update = + Updates.set( + BackwardCompatibility.MERGE_ON_HOST_INIT, Context.now()); + BackwardCompatibilityDao.instance + .getMCollection() + .updateMany(new BasicDBObject(), update); loggerMaker.infoAndAddToDb("Merging hosts completed", LogDb.RUNTIME); } try { - List allCollections = ApiCollectionsDao.instance.getMetaAll(); - AccountSettings accountSettings = AccountSettingsDao.instance.findOne(AccountSettingsDao.generateFilter()); + List allCollections = + ApiCollectionsDao.instance.getMetaAll(); + AccountSettings accountSettings = + AccountSettingsDao.instance.findOne( + AccountSettingsDao.generateFilter()); boolean makeApisCaseInsensitive = false; - if(accountSettings != null){ - makeApisCaseInsensitive = accountSettings.getHandleApisCaseInsensitive(); + if (accountSettings != null) { + makeApisCaseInsensitive = + accountSettings.getHandleApisCaseInsensitive(); } - - Boolean urlRegexMatchingEnabled = accountSettings == null || accountSettings.getUrlRegexMatchingEnabled(); - loggerMaker.infoAndAddToDb("url regex matching enabled status is " + urlRegexMatchingEnabled, LogDb.RUNTIME); - for(ApiCollection apiCollection: allCollections) { + + Boolean urlRegexMatchingEnabled = + accountSettings == null + || accountSettings.getUrlRegexMatchingEnabled(); + loggerMaker.infoAndAddToDb( + "url regex matching enabled status is " + urlRegexMatchingEnabled, + LogDb.RUNTIME); + for (ApiCollection apiCollection : allCollections) { int start = Context.now(); - loggerMaker.infoAndAddToDb("Started merging API collection " + apiCollection.getId(), LogDb.RUNTIME); + loggerMaker.infoAndAddToDb( + "Started merging API collection " + apiCollection.getId(), + LogDb.RUNTIME); try { - mergeUrlsAndSave(apiCollection.getId(), true, true, existingAPIsInDb, makeApisCaseInsensitive); - loggerMaker.infoAndAddToDb("Finished merging API collection basic " + apiCollection.getId() + " in " + (Context.now() - start) + " seconds", LogDb.RUNTIME); + mergeUrlsAndSave( + apiCollection.getId(), + true, + true, + existingAPIsInDb, + makeApisCaseInsensitive); + loggerMaker.infoAndAddToDb( + "Finished merging API collection basic " + + apiCollection.getId() + + " in " + + (Context.now() - start) + + " seconds", + LogDb.RUNTIME); } catch (Exception e) { - loggerMaker.errorAndAddToDb(e.getMessage(),LogDb.RUNTIME); + loggerMaker.errorAndAddToDb(e.getMessage(), LogDb.RUNTIME); } try { - mergeUrlsAndSave(apiCollection.getId(), true, false, existingAPIsInDb, makeApisCaseInsensitive); - loggerMaker.infoAndAddToDb("Finished merging API collection all" + apiCollection.getId() + " in " + (Context.now() - start) + " seconds", LogDb.RUNTIME); + mergeUrlsAndSave( + apiCollection.getId(), + true, + false, + existingAPIsInDb, + makeApisCaseInsensitive); + loggerMaker.infoAndAddToDb( + "Finished merging API collection all" + + apiCollection.getId() + + " in " + + (Context.now() - start) + + " seconds", + LogDb.RUNTIME); } catch (Exception e) { - loggerMaker.errorAndAddToDb(e.getMessage(),LogDb.RUNTIME); + loggerMaker.errorAndAddToDb(e.getMessage(), LogDb.RUNTIME); } - } } catch (Exception e) { - String err = e.getStackTrace().length > 0 ? e.getStackTrace()[0].toString() : e.getMessage() ; - loggerMaker.errorAndAddToDb("error in mergeUrlsAndSave: " + err, LogDb.RUNTIME); + String err = + e.getStackTrace().length > 0 + ? e.getStackTrace()[0].toString() + : e.getMessage(); + loggerMaker.errorAndAddToDb( + "error in mergeUrlsAndSave: " + err, LogDb.RUNTIME); e.printStackTrace(); } try { loggerMaker.infoAndAddToDb("Started clearing values in db ", LogDb.RUNTIME); clearValuesInDB(); - loggerMaker.infoAndAddToDb("Finished clearing values in db ", LogDb.RUNTIME); + loggerMaker.infoAndAddToDb( + "Finished clearing values in db ", LogDb.RUNTIME); } catch (Exception e) { - loggerMaker.infoAndAddToDb("Error while clearing values in db: " + e.getMessage(), LogDb.RUNTIME); + loggerMaker.infoAndAddToDb( + "Error while clearing values in db: " + e.getMessage(), + LogDb.RUNTIME); } } loggerMaker.infoAndAddToDb("Finished mergeAsyncOutside", LogDb.RUNTIME); @@ -1575,26 +1901,39 @@ public void buildFromDB(boolean calcDiff, boolean fetchAllSTI) { loggerMaker.infoAndAddToDb("Fetching STIs: " + fetchAllSTI, LogDb.RUNTIME); List allParams; if (fetchAllSTI) { - Bson filterForHostHeader = SingleTypeInfoDao.filterForHostHeader(-1,false); - Bson filterQ = Filters.and(filterForHostHeader, Filters.regex(SingleTypeInfo._URL, "STRING|INTEGER")); - allParams = SingleTypeInfoDao.instance.findAll(filterQ, Projections.exclude(SingleTypeInfo._VALUES)); - allParams.addAll(SingleTypeInfoDao.instance.findAll(new BasicDBObject(), Projections.exclude(SingleTypeInfo._VALUES))); + Bson filterForHostHeader = SingleTypeInfoDao.filterForHostHeader(-1, false); + Bson filterQ = + Filters.and( + filterForHostHeader, + Filters.regex(SingleTypeInfo._URL, "STRING|INTEGER")); + allParams = + SingleTypeInfoDao.instance.findAll( + filterQ, Projections.exclude(SingleTypeInfo._VALUES)); + allParams.addAll( + SingleTypeInfoDao.instance.findAll( + new BasicDBObject(), Projections.exclude(SingleTypeInfo._VALUES))); int dependencyFlowLimit = 1_000; if (mergingCalled && allParams.size() < dependencyFlowLimit) { - loggerMaker.infoAndAddToDb("ALl params less than " + dependencyFlowLimit +", running dependency flow", LogDb.RUNTIME); + loggerMaker.infoAndAddToDb( + "ALl params less than " + dependencyFlowLimit + ", running dependency flow", + LogDb.RUNTIME); try { DependencyFlow dependencyFlow = new DependencyFlow(); dependencyFlow.run(null); dependencyFlow.syncWithDb(); loggerMaker.infoAndAddToDb("Finished running dependency flow", LogDb.RUNTIME); } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, "Error while running dependency flow in runtime: " + e.getMessage(), LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + e, + "Error while running dependency flow in runtime: " + e.getMessage(), + LogDb.RUNTIME); } } } else { - List apiCollectionIds = ApiCollectionsDao.instance.fetchNonTrafficApiCollectionsIds(); + List apiCollectionIds = + ApiCollectionsDao.instance.fetchNonTrafficApiCollectionsIds(); allParams = SingleTypeInfoDao.instance.fetchStiOfCollections(apiCollectionIds); } loggerMaker.infoAndAddToDb("Fetched STIs count: " + allParams.size(), LogDb.RUNTIME); @@ -1603,9 +1942,10 @@ public void buildFromDB(boolean calcDiff, boolean fetchAllSTI) { this.dbState = build(allParams, existingAPIsInDb); loggerMaker.infoAndAddToDb("Done building dbState", LogDb.RUNTIME); this.sensitiveParamInfoBooleanMap = new HashMap<>(); - List sensitiveParamInfos = SensitiveParamInfoDao.instance.getUnsavedSensitiveParamInfos(); + List sensitiveParamInfos = + SensitiveParamInfoDao.instance.getUnsavedSensitiveParamInfos(); loggerMaker.infoAndAddToDb("Done fetching sensitiveParamInfos", LogDb.RUNTIME); - for (SensitiveParamInfo sensitiveParamInfo: sensitiveParamInfos) { + for (SensitiveParamInfo sensitiveParamInfo : sensitiveParamInfos) { this.sensitiveParamInfoBooleanMap.put(sensitiveParamInfo, false); } @@ -1613,20 +1953,25 @@ public void buildFromDB(boolean calcDiff, boolean fetchAllSTI) { this.delta = new HashMap<>(); } - List advancedFilterTemplates = AdvancedTrafficFiltersDao.instance.findAll(Filters.ne(YamlTemplate.INACTIVE, true)); - advancedFilterMap = FilterYamlTemplateDao.instance.fetchFilterConfig(false, advancedFilterTemplates, true); + List advancedFilterTemplates = + AdvancedTrafficFiltersDao.instance.findAll(Filters.ne(YamlTemplate.INACTIVE, true)); + advancedFilterMap = + FilterYamlTemplateDao.fetchFilterConfig(false, advancedFilterTemplates, true); try { // fetchAllSTI check added to make sure only runs in dashboard if (!fetchAllSTI) { - loggerMaker.infoAndAddToDb("Started running update API collection count function", LogDb.RUNTIME); - for(int collectionId: this.dbState.keySet()) { + loggerMaker.infoAndAddToDb( + "Started running update API collection count function", LogDb.RUNTIME); + for (int collectionId : this.dbState.keySet()) { APICatalog newCatalog = this.dbState.get(collectionId); updateApiCollectionCount(newCatalog, collectionId); } - loggerMaker.infoAndAddToDb("Finished running update API collection count function", LogDb.RUNTIME); + loggerMaker.infoAndAddToDb( + "Finished running update API collection count function", LogDb.RUNTIME); } } catch (Exception e) { - loggerMaker.errorAndAddToDb("Error while filling urls in apiCollection: " + e.getMessage(), LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + "Error while filling urls in apiCollection: " + e.getMessage(), LogDb.RUNTIME); } mergedUrls = MergedUrlsDao.instance.getMergedUrls(); @@ -1637,16 +1982,16 @@ public void buildFromDB(boolean calcDiff, boolean fetchAllSTI) { public static void updateApiCollectionCount(APICatalog apiCatalog, int apiCollectionId) { Set newURLs = new HashSet<>(); - + if (apiCatalog == null) { return; } - for(URLTemplate url: apiCatalog.getTemplateURLToMethods().keySet()) { - newURLs.add(url.getTemplateString()+ " "+ url.getMethod().name()); + for (URLTemplate url : apiCatalog.getTemplateURLToMethods().keySet()) { + newURLs.add(url.getTemplateString() + " " + url.getMethod().name()); } - for(URLStatic url: apiCatalog.getStrictURLToMethods().keySet()) { - newURLs.add(url.getUrl()+ " "+ url.getMethod().name()); + for (URLStatic url : apiCatalog.getStrictURLToMethods().keySet()) { + newURLs.add(url.getUrl() + " " + url.getMethod().name()); } Bson findQ = Filters.eq("_id", apiCollectionId); @@ -1654,27 +1999,43 @@ public static void updateApiCollectionCount(APICatalog apiCatalog, int apiCollec int count = 0; Set batchedUrls = new HashSet<>(); - ApiCollectionsDao.instance.getMCollection().updateOne(findQ, Updates.unset(ApiCollection.URLS_STRING)); + ApiCollectionsDao.instance + .getMCollection() + .updateOne(findQ, Updates.unset(ApiCollection.URLS_STRING)); for (String url : newURLs) { batchedUrls.add(url); count++; if (count == batchSize) { - ApiCollectionsDao.instance.getMCollection().bulkWrite( - Collections.singletonList(new UpdateManyModel<>(findQ, - Updates.addEachToSet(ApiCollection.URLS_STRING, new ArrayList<>(batchedUrls)), new UpdateOptions())), - new BulkWriteOptions().ordered(false)); + ApiCollectionsDao.instance + .getMCollection() + .bulkWrite( + Collections.singletonList( + new UpdateManyModel<>( + findQ, + Updates.addEachToSet( + ApiCollection.URLS_STRING, + new ArrayList<>(batchedUrls)), + new UpdateOptions())), + new BulkWriteOptions().ordered(false)); count = 0; batchedUrls.clear(); } } if (!batchedUrls.isEmpty()) { - ApiCollectionsDao.instance.getMCollection().bulkWrite( - Collections.singletonList(new UpdateManyModel<>(findQ, - Updates.addEachToSet(ApiCollection.URLS_STRING, new ArrayList<>(batchedUrls)), new UpdateOptions())), - new BulkWriteOptions().ordered(false)); + ApiCollectionsDao.instance + .getMCollection() + .bulkWrite( + Collections.singletonList( + new UpdateManyModel<>( + findQ, + Updates.addEachToSet( + ApiCollection.URLS_STRING, + new ArrayList<>(batchedUrls)), + new UpdateOptions())), + new BulkWriteOptions().ordered(false)); } } @@ -1693,7 +2054,12 @@ private static void buildHelper(SingleTypeInfo param, Map r reqTemplate = catalog.getTemplateURLToMethods().get(urlTemplate); if (reqTemplate == null) { - reqTemplate = new RequestTemplate(new HashMap<>(), new HashMap<>(), new HashMap<>(), new TrafficRecorder(new HashMap<>())); + reqTemplate = + new RequestTemplate( + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + new TrafficRecorder(new HashMap<>())); catalog.getTemplateURLToMethods().put(urlTemplate, reqTemplate); } @@ -1701,7 +2067,12 @@ private static void buildHelper(SingleTypeInfo param, Map r URLStatic urlStatic = new URLStatic(url, Method.fromString(param.getMethod())); reqTemplate = catalog.getStrictURLToMethods().get(urlStatic); if (reqTemplate == null) { - reqTemplate = new RequestTemplate(new HashMap<>(), new HashMap<>(), new HashMap<>(), new TrafficRecorder(new HashMap<>())); + reqTemplate = + new RequestTemplate( + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + new TrafficRecorder(new HashMap<>())); catalog.getStrictURLToMethods().put(urlStatic, reqTemplate); } } @@ -1723,22 +2094,30 @@ private static void buildHelper(SingleTypeInfo param, Map r } keyTypes.getOccurrences().put(param.getSubType(), param); } catch (Exception e) { - loggerMaker.errorAndAddToDb("ERROR while parsing url param position: " + p, LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + "ERROR while parsing url param position: " + p, LogDb.RUNTIME); } return; } if (param.getResponseCode() > 0) { - RequestTemplate respTemplate = reqTemplate.getResponseTemplates().get(param.getResponseCode()); + RequestTemplate respTemplate = + reqTemplate.getResponseTemplates().get(param.getResponseCode()); if (respTemplate == null) { - respTemplate = new RequestTemplate(new HashMap<>(), new HashMap<>(), new HashMap<>(), new TrafficRecorder(new HashMap<>())); + respTemplate = + new RequestTemplate( + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + new TrafficRecorder(new HashMap<>())); reqTemplate.getResponseTemplates().put(param.getResponseCode(), respTemplate); } reqTemplate = respTemplate; } - Map keyTypesMap = param.getIsHeader() ? reqTemplate.getHeaders() : reqTemplate.getParameters(); + Map keyTypesMap = + param.getIsHeader() ? reqTemplate.getHeaders() : reqTemplate.getParameters(); KeyTypes keyTypes = keyTypesMap.get(param.getParam()); if (keyTypes == null) { @@ -1759,11 +2138,13 @@ private static void buildHelper(SingleTypeInfo param, Map r keyTypes.getOccurrences().put(param.getSubType(), param); } - private static Set demosAndDeactivatedCollections = UsageMetricCalculator.getDemosAndDeactivated(); + private static Set demosAndDeactivatedCollections = + UsageMetricCalculator.getDemosAndDeactivated(); - private static void fillExistingAPIsInDb(SingleTypeInfo sti, BloomFilter existingAPIsInDb) { + private static void fillExistingAPIsInDb( + SingleTypeInfo sti, BloomFilter existingAPIsInDb) { - if(existingAPIsInDb==null){ + if (existingAPIsInDb == null) { return; } @@ -1776,16 +2157,18 @@ private static void fillExistingAPIsInDb(SingleTypeInfo sti, BloomFilter build(List allParams, BloomFilter existingAPIsInDb) { + public static Map build( + List allParams, BloomFilter existingAPIsInDb) { Map ret = new HashMap<>(); - - for (SingleTypeInfo param: allParams) { + + for (SingleTypeInfo param : allParams) { try { buildHelper(param, ret); fillExistingAPIsInDb(param, existingAPIsInDb); } catch (Exception e) { e.printStackTrace(); - loggerMaker.errorAndAddToDb("Error while building from db: " + e.getMessage(), LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + "Error while building from db: " + e.getMessage(), LogDb.RUNTIME); } } @@ -1794,9 +2177,14 @@ public static Map build(List allParams, Blo int counter = 0; List partnerIpsList = new ArrayList<>(); - + public void syncWithDB(boolean syncImmediately, boolean fetchAllSTI, SyncLimit syncLimit) { - loggerMaker.infoAndAddToDb("Started sync with db! syncImmediately="+syncImmediately + " fetchAllSTI="+fetchAllSTI, LogDb.RUNTIME); + loggerMaker.infoAndAddToDb( + "Started sync with db! syncImmediately=" + + syncImmediately + + " fetchAllSTI=" + + fetchAllSTI, + LogDb.RUNTIME); List> writesForParams = new ArrayList<>(); List> writesForSensitiveSampleData = new ArrayList<>(); List> writesForTraffic = new ArrayList<>(); @@ -1804,30 +2192,33 @@ public void syncWithDB(boolean syncImmediately, boolean fetchAllSTI, SyncLimit s List> writesForSensitiveParamInfo = new ArrayList<>(); Map apiCollectionToRedactPayload = new HashMap<>(); List all = ApiCollectionsDao.instance.findAll(new BasicDBObject()); - for(ApiCollection apiCollection: all) { + for (ApiCollection apiCollection : all) { apiCollectionToRedactPayload.put(apiCollection.getId(), apiCollection.getRedact()); } - AccountSettings accountSettings = AccountSettingsDao.instance.findOne(AccountSettingsDao.generateFilter()); + AccountSettings accountSettings = + AccountSettingsDao.instance.findOne(AccountSettingsDao.generateFilter()); boolean redact = false; if (accountSettings != null) { - redact = accountSettings.isRedactPayload(); + redact = accountSettings.isRedactPayload(); if (accountSettings.getPartnerIpList() != null) { partnerIpsList = accountSettings.getPartnerIpList(); } } counter++; - for(int apiCollectionId: this.delta.keySet()) { + for (int apiCollectionId : this.delta.keySet()) { APICatalog deltaCatalog = this.delta.get(apiCollectionId); - Set demosAndDeactivatedCollections = UsageMetricCalculator.getDemosAndDeactivated(); + Set demosAndDeactivatedCollections = + UsageMetricCalculator.getDemosAndDeactivated(); if (syncLimit.checkLimit && !demosAndDeactivatedCollections.contains(apiCollectionId)) { int deltaUsage = 0; - Iterator> staticUrlIterator = deltaCatalog.getStrictURLToMethods().entrySet().iterator(); + Iterator> staticUrlIterator = + deltaCatalog.getStrictURLToMethods().entrySet().iterator(); while (staticUrlIterator.hasNext()) { Entry entry = staticUrlIterator.next(); URLStatic urlStatic = entry.getKey(); @@ -1842,12 +2233,17 @@ public void syncWithDB(boolean syncImmediately, boolean fetchAllSTI, SyncLimit s } } - Iterator> templateUrlIterator = deltaCatalog.getTemplateURLToMethods().entrySet().iterator(); - while(templateUrlIterator.hasNext()) { + Iterator> templateUrlIterator = + deltaCatalog.getTemplateURLToMethods().entrySet().iterator(); + while (templateUrlIterator.hasNext()) { Entry entry = templateUrlIterator.next(); URLTemplate urlTemplate = entry.getKey(); - String checkString = apiCollectionId + " " + urlTemplate.getTemplateString() + " " - + urlTemplate.getMethod().name(); + String checkString = + apiCollectionId + + " " + + urlTemplate.getTemplateString() + + " " + + urlTemplate.getMethod().name(); if (!existingAPIsInDb.mightContain(checkString)) { if (syncLimit.updateUsageLeftAndCheckSkip()) { templateUrlIterator.remove(); @@ -1858,7 +2254,8 @@ public void syncWithDB(boolean syncImmediately, boolean fetchAllSTI, SyncLimit s } } - UsageMetricHandler.calcAndFetchFeatureAccessUsingDeltaUsage(MetricTypes.ACTIVE_ENDPOINTS, Context.accountId.get(), deltaUsage); + UsageMetricHandler.calcAndFetchFeatureAccessUsingDeltaUsage( + MetricTypes.ACTIVE_ENDPOINTS, Context.accountId.get(), deltaUsage); } /* @@ -1871,9 +2268,14 @@ public void syncWithDB(boolean syncImmediately, boolean fetchAllSTI, SyncLimit s * slow the runtime instance. */ - APICatalog dbCatalog = this.dbState.getOrDefault(apiCollectionId, new APICatalog(apiCollectionId, new HashMap<>(), new HashMap<>())); - boolean redactCollectionLevel = apiCollectionToRedactPayload.getOrDefault(apiCollectionId, false); - DbUpdateReturn dbUpdateReturn = getDBUpdatesForParams(deltaCatalog, dbCatalog, redact, redactCollectionLevel); + APICatalog dbCatalog = + this.dbState.getOrDefault( + apiCollectionId, + new APICatalog(apiCollectionId, new HashMap<>(), new HashMap<>())); + boolean redactCollectionLevel = + apiCollectionToRedactPayload.getOrDefault(apiCollectionId, false); + DbUpdateReturn dbUpdateReturn = + getDBUpdatesForParams(deltaCatalog, dbCatalog, redact, redactCollectionLevel); writesForParams.addAll(dbUpdateReturn.bulkUpdatesForSingleTypeInfo); writesForSensitiveSampleData.addAll(dbUpdateReturn.bulkUpdatesForSampleData); writesForSensitiveParamInfo.addAll(dbUpdateReturn.bulkUpdatesForSensitiveParamInfo); @@ -1881,63 +2283,100 @@ public void syncWithDB(boolean syncImmediately, boolean fetchAllSTI, SyncLimit s deltaCatalog.setDeletedInfo(new ArrayList<>()); boolean forceUpdate = syncImmediately || counter % 10 == 0; - writesForSampleData.addAll(getDBUpdatesForSampleData(apiCollectionId, deltaCatalog,dbCatalog, forceUpdate, redact, redactCollectionLevel)); - } - - loggerMaker.infoAndAddToDb("adding " + writesForParams.size() + " updates for params", LogDb.RUNTIME); + writesForSampleData.addAll( + getDBUpdatesForSampleData( + apiCollectionId, + deltaCatalog, + dbCatalog, + forceUpdate, + redact, + redactCollectionLevel)); + } + + loggerMaker.infoAndAddToDb( + "adding " + writesForParams.size() + " updates for params", LogDb.RUNTIME); int from = 0; int batch = 10000; long start = System.currentTimeMillis(); - if (writesForParams.size() >0) { + if (writesForParams.size() > 0) { do { - List> slicedWrites = writesForParams.subList(from, Math.min(from + batch, writesForParams.size())); + List> slicedWrites = + writesForParams.subList( + from, Math.min(from + batch, writesForParams.size())); from += batch; BulkWriteResult res = - SingleTypeInfoDao.instance.getMCollection().bulkWrite( - slicedWrites, - new BulkWriteOptions().ordered(true).bypassDocumentValidation(false) - ); - - loggerMaker.infoAndAddToDb((System.currentTimeMillis() - start) + ": " + res.getInserts().size() + " " + res.getUpserts().size(), LogDb.RUNTIME); + SingleTypeInfoDao.instance + .getMCollection() + .bulkWrite( + slicedWrites, + new BulkWriteOptions() + .ordered(true) + .bypassDocumentValidation(false)); + + loggerMaker.infoAndAddToDb( + (System.currentTimeMillis() - start) + + ": " + + res.getInserts().size() + + " " + + res.getUpserts().size(), + LogDb.RUNTIME); } while (from < writesForParams.size()); } aktoPolicyNew.syncWithDb(); - loggerMaker.infoAndAddToDb("adding " + writesForTraffic.size() + " updates for traffic", LogDb.RUNTIME); - if(writesForTraffic.size() > 0) { - BulkWriteResult res = TrafficInfoDao.instance.getMCollection().bulkWrite(writesForTraffic); - - loggerMaker.infoAndAddToDb(res.getInserts().size() + " " +res.getUpserts().size(), LogDb.RUNTIME); + loggerMaker.infoAndAddToDb( + "adding " + writesForTraffic.size() + " updates for traffic", LogDb.RUNTIME); + if (writesForTraffic.size() > 0) { + BulkWriteResult res = + TrafficInfoDao.instance.getMCollection().bulkWrite(writesForTraffic); + loggerMaker.infoAndAddToDb( + res.getInserts().size() + " " + res.getUpserts().size(), LogDb.RUNTIME); } - loggerMaker.infoAndAddToDb("adding " + writesForSampleData.size() + " updates for samples", LogDb.RUNTIME); - if(writesForSampleData.size() > 0) { - BulkWriteResult res = SampleDataDao.instance.getMCollection().bulkWrite(writesForSampleData); - - loggerMaker.infoAndAddToDb(res.getInserts().size() + " " +res.getUpserts().size(), LogDb.RUNTIME); + loggerMaker.infoAndAddToDb( + "adding " + writesForSampleData.size() + " updates for samples", LogDb.RUNTIME); + if (writesForSampleData.size() > 0) { + BulkWriteResult res = + SampleDataDao.instance.getMCollection().bulkWrite(writesForSampleData); + loggerMaker.infoAndAddToDb( + res.getInserts().size() + " " + res.getUpserts().size(), LogDb.RUNTIME); } if (writesForSensitiveSampleData.size() > 0) { try { - SensitiveSampleDataDao.instance.getMCollection().bulkWrite(writesForSensitiveSampleData); - loggerMaker.infoAndAddToDb("successfully added " + writesForSensitiveSampleData.size() + " updates for sensitive sample data" , LogDb.RUNTIME); + SensitiveSampleDataDao.instance + .getMCollection() + .bulkWrite(writesForSensitiveSampleData); + loggerMaker.infoAndAddToDb( + "successfully added " + + writesForSensitiveSampleData.size() + + " updates for sensitive sample data", + LogDb.RUNTIME); } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, "error while adding SensitiveSampleData",LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + e, "error while adding SensitiveSampleData", LogDb.RUNTIME); } } if (writesForSensitiveParamInfo.size() > 0) { try { - SensitiveParamInfoDao.instance.getMCollection().bulkWrite(writesForSensitiveParamInfo); - loggerMaker.infoAndAddToDb("successfully added " + writesForSensitiveParamInfo.size() + " updates for sensitive sample param info" , LogDb.RUNTIME); + SensitiveParamInfoDao.instance + .getMCollection() + .bulkWrite(writesForSensitiveParamInfo); + loggerMaker.infoAndAddToDb( + "successfully added " + + writesForSensitiveParamInfo.size() + + " updates for sensitive sample param info", + LogDb.RUNTIME); } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, "error while adding SensitiveParamInfo",LogDb.RUNTIME); + loggerMaker.errorAndAddToDb( + e, "error while adding SensitiveParamInfo", LogDb.RUNTIME); } } @@ -1947,21 +2386,19 @@ public void syncWithDB(boolean syncImmediately, boolean fetchAllSTI, SyncLimit s } public void printNewURLsInDelta(APICatalog deltaCatalog) { - for(URLStatic s: deltaCatalog.getStrictURLToMethods().keySet()) { + for (URLStatic s : deltaCatalog.getStrictURLToMethods().keySet()) { logger.info(s.getUrl()); } - for(URLTemplate s: deltaCatalog.getTemplateURLToMethods().keySet()) { + for (URLTemplate s : deltaCatalog.getTemplateURLToMethods().keySet()) { logger.info(s.getTemplateString()); } } - public APICatalog getDelta(int apiCollectionId) { return this.delta.get(apiCollectionId); } - public APICatalog getDbState(int apiCollectionId) { return this.dbState.get(apiCollectionId); } diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java index f481d256ac..1f9a961f1b 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java +++ b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java @@ -39,13 +39,14 @@ public long getValue() { public RedisBackedCounterCache(RedisClient redisClient, String prefix) { this.prefix = prefix; this.redis = redisClient.connect(new LongValueCodec()); - this.localCache = Caffeine.newBuilder() - .maximumSize(100000) - .expireAfterWrite(1, TimeUnit.HOURS) - .build(); + this.localCache = + Caffeine.newBuilder() + .maximumSize(100000) + .expireAfterWrite(3, TimeUnit.HOURS) + .build(); ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); - executor.scheduleAtFixedRate(this::syncToRedis, 60, 1, TimeUnit.SECONDS); + executor.scheduleAtFixedRate(this::syncToRedis, 60, 5, TimeUnit.SECONDS); this.pendingOps = new ConcurrentLinkedQueue<>(); } diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 58c94bcdaf..f4e4efa8ea 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -68,7 +68,7 @@ public void filterFunction(List responseParams) { if ((lastFilterFetch + FILTER_REFRESH_INTERVAL) < now) { // TODO: add support for only active templates. List templates = dataActor.fetchFilterYamlTemplates(); - apiFilters = FilterYamlTemplateDao.instance.fetchFilterConfig(false, templates, false); + apiFilters = FilterYamlTemplateDao.fetchFilterConfig(false, templates, false); lastFilterFetch = now; } diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index a16d1e5ec9..0fe5c2ec17 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -17,6 +17,7 @@ public class WindowBasedThresholdNotifier { // more than once by multiple instances of the service. // But on 1 instance, we should not notify more than once in the cooldown // period. + // TODO: Move this to redis private final ConcurrentMap notifiedMap; public static class Config { diff --git a/apps/dashboard/src/main/java/com/akto/utils/jobs/CleanInventory.java b/apps/dashboard/src/main/java/com/akto/utils/jobs/CleanInventory.java index 231939d22c..18ec2476e0 100644 --- a/apps/dashboard/src/main/java/com/akto/utils/jobs/CleanInventory.java +++ b/apps/dashboard/src/main/java/com/akto/utils/jobs/CleanInventory.java @@ -59,36 +59,46 @@ public class CleanInventory { - private static final LoggerMaker loggerMaker = new LoggerMaker(CleanInventory.class, LogDb.DASHBOARD); + private static final LoggerMaker loggerMaker = + new LoggerMaker(CleanInventory.class, LogDb.DASHBOARD); private static final Logger logger = LoggerFactory.getLogger(CleanInventory.class); - final static ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + static final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); public static void cleanInventoryJobRunner() { - scheduler.scheduleAtFixedRate(new Runnable() { - public void run() { - - int now = Context.now(); - logger.info("Starting cleanInventoryJob for all accounts at " + now); - - AccountTask.instance.executeTask(new Consumer() { - @Override - public void accept(Account t) { - try { - cleanInventoryJob(); - } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, "Error in cleanInventoryJob"); - } + scheduler.scheduleAtFixedRate( + new Runnable() { + public void run() { + + int now = Context.now(); + logger.info("Starting cleanInventoryJob for all accounts at " + now); + + AccountTask.instance.executeTask( + new Consumer() { + @Override + public void accept(Account t) { + try { + cleanInventoryJob(); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + e, "Error in cleanInventoryJob"); + } + } + }, + "clean-inventory-job"); + + int now2 = Context.now(); + int diffNow = now2 - now; + logger.info( + String.format( + "Completed cleanInventoryJob for all accounts at %d , time taken : %d", + now2, diffNow)); } - }, "clean-inventory-job"); - - int now2 = Context.now(); - int diffNow = now2-now; - logger.info(String.format("Completed cleanInventoryJob for all accounts at %d , time taken : %d", now2, diffNow)); - } - }, 0, 5, TimeUnit.HOURS); - + }, + 0, + 5, + TimeUnit.HOURS); } private static Set methodSet = new HashSet<>(); @@ -99,10 +109,14 @@ private static Set getMethodSet() { return methodSet; } - List lowerCaseMethods = Arrays.asList(URLMethods.Method.getValuesArray()).stream() - .map(s -> s.name().toLowerCase()).collect(Collectors.toList()); - List upperCaseMethods = Arrays.asList(URLMethods.Method.getValuesArray()).stream() - .map(s -> s.name().toUpperCase()).collect(Collectors.toList()); + List lowerCaseMethods = + Arrays.asList(URLMethods.Method.getValuesArray()).stream() + .map(s -> s.name().toLowerCase()) + .collect(Collectors.toList()); + List upperCaseMethods = + Arrays.asList(URLMethods.Method.getValuesArray()).stream() + .map(s -> s.name().toUpperCase()) + .collect(Collectors.toList()); methodSet.addAll(upperCaseMethods); methodSet.addAll(lowerCaseMethods); return methodSet; @@ -124,23 +138,33 @@ private static void cleanInventoryJob() { int diff = now2 - now; if (diff >= 2) { - loggerMaker.infoAndAddToDb(String.format("cleanInventoryJob finished, time taken: %d ", diff)); + loggerMaker.infoAndAddToDb( + String.format("cleanInventoryJob finished, time taken: %d ", diff)); } - } - - public static void cleanFilteredSampleDataFromAdvancedFilters(List apiCollections, List yamlTemplates, List redundantUrlList, String filePath, boolean shouldDeleteRequest, boolean saveLogsToDB) throws IOException{ - Map apiCollectionMap = apiCollections.stream().collect(Collectors.toMap(ApiCollection::getId, Function.identity())); + public static void cleanFilteredSampleDataFromAdvancedFilters( + List apiCollections, + List yamlTemplates, + List redundantUrlList, + String filePath, + boolean shouldDeleteRequest, + boolean saveLogsToDB) + throws IOException { + + Map apiCollectionMap = + apiCollections.stream() + .collect(Collectors.toMap(ApiCollection::getId, Function.identity())); // BufferedWriter writer = new BufferedWriter(new FileWriter(new File(filePath))); List sampleDataList = new ArrayList<>(); Bson filters = Filters.empty(); int skip = 0; int limit = 100; Bson sort = Sorts.ascending("_id.apiCollectionId", "_id.url", "_id.method"); - Map collectionWiseDeletionCountMap = new HashMap<>(); + Map collectionWiseDeletionCountMap = new HashMap<>(); - Map filterMap = FilterYamlTemplateDao.instance.fetchFilterConfig(false, yamlTemplates, true); + Map filterMap = + FilterYamlTemplateDao.fetchFilterConfig(false, yamlTemplates, true); Pattern pattern = createRegexPatternFromList(redundantUrlList); do { sampleDataList = SampleDataDao.instance.findAll(filters, skip, limit, sort); @@ -153,41 +177,52 @@ public static void cleanFilteredSampleDataFromAdvancedFilters(List samples = sampleData.getSamples(); remainingSamples.clear(); if (samples == null || samples.isEmpty()) { - logger.info("[BadApisRemover] No samples found for : " + sampleData.getId()); + logger.info( + "[BadApisRemover] No samples found for : " + sampleData.getId()); continue; } - ApiCollection apiCollection = apiCollectionMap.get(sampleData.getId().getApiCollectionId()); + ApiCollection apiCollection = + apiCollectionMap.get(sampleData.getId().getApiCollectionId()); if (apiCollection == null) { - if(!DashboardMode.isMetered()){ - logger.info("[BadApisRemover] No apiCollection found for : " + sampleData.getId()); + if (!DashboardMode.isMetered()) { + logger.info( + "[BadApisRemover] No apiCollection found for : " + + sampleData.getId()); } continue; } - boolean isRedundant = false; boolean isNetsparkerPresent = false; boolean movingApi = false; for (String sample : samples) { - HttpResponseParams httpResponseParams = HttpCallParser.parseKafkaMessage(sample); + HttpResponseParams httpResponseParams = + HttpCallParser.parseKafkaMessage(sample); isNetsparkerPresent |= sample.toLowerCase().contains("netsparker"); - if(httpResponseParams != null){ - isRedundant = HttpCallParser.isRedundantEndpoint(httpResponseParams.getRequestParams().getURL(), pattern); - if(!isRedundant){ - Map> executorNodesMap = ParseAndExecute.createExecutorNodeMap(filterMap); - Pair temp = HttpCallParser.applyAdvancedFilters(httpResponseParams, executorNodesMap, filterMap); + if (httpResponseParams != null) { + isRedundant = + HttpCallParser.isRedundantEndpoint( + httpResponseParams.getRequestParams().getURL(), + pattern); + if (!isRedundant) { + Map> executorNodesMap = + ParseAndExecute.createExecutorNodeMap(filterMap); + Pair temp = + HttpCallParser.applyAdvancedFilters( + httpResponseParams, executorNodesMap, filterMap); HttpResponseParams param = temp.getFirst(); FILTER_TYPE filterType = temp.getSecond(); - if(param != null){ - // comes when Filter_Block is not valid {Remaining => Unchanged, Modified, Allowed} - if(filterType.equals(FILTER_TYPE.MODIFIED)){ + if (param != null) { + // comes when Filter_Block is not valid {Remaining => Unchanged, + // Modified, Allowed} + if (filterType.equals(FILTER_TYPE.MODIFIED)) { // filter passed and modified movingApi = true; remainingSamples.add(sample); break; - }else if(filterType.equals(FILTER_TYPE.ALLOWED)){ + } else if (filterType.equals(FILTER_TYPE.ALLOWED)) { // filter passed and not modified remainingSamples.add(sample); }else if(filterMap.size() == 1){ @@ -202,7 +237,7 @@ public static void cleanFilteredSampleDataFromAdvancedFilters(List filter from advanced filter is passed || filter from block filter fails - if(saveLogsToDB){ + // other cases like: => filter from advanced filter is passed || filter from + // block filter fails + if (saveLogsToDB) { loggerMaker.infoAndAddToDb( - "Filter did not pass, keeping api found from filter: " + sampleData.getId(), LogDb.DASHBOARD - ); - }else{ - logger.info("[BadApisRemover] " + isNetsparkerPresent + " Keeping API from template: " + sampleData.getId(), LogDb.DASHBOARD); - } - + "Filter did not pass, keeping api found from filter: " + + sampleData.getId(), + LogDb.DASHBOARD); + } else { + logger.info( + "[BadApisRemover] " + + isNetsparkerPresent + + " Keeping API from template: " + + sampleData.getId(), + LogDb.DASHBOARD); + } } } catch (Exception e) { - loggerMaker.errorAndAddToDb("[BadApisRemover] Couldn't delete an api for default payload: " + sampleData.getId() + e.getMessage(), LogDb.DASHBOARD); + loggerMaker.errorAndAddToDb( + "[BadApisRemover] Couldn't delete an api for default payload: " + + sampleData.getId() + + e.getMessage(), + LogDb.DASHBOARD); } } if (shouldDeleteRequest) { @@ -263,13 +308,15 @@ public static void cleanFilteredSampleDataFromAdvancedFilters(List iterator: collectionWiseDeletionCountMap.entrySet()){ + for (Map.Entry iterator : collectionWiseDeletionCountMap.entrySet()) { int collId = iterator.getKey(); int deletionCount = iterator.getValue(); String name = apiCollectionMap.get(collId).getDisplayName(); - if(saveLogsToDB){ - loggerMaker.infoAndAddToDb("Total apis deleted from collection: " + name + " are: " + deletionCount, LogDb.DASHBOARD); + if (saveLogsToDB) { + loggerMaker.infoAndAddToDb( + "Total apis deleted from collection: " + name + " are: " + deletionCount, + LogDb.DASHBOARD); } } @@ -277,22 +324,29 @@ public static void cleanFilteredSampleDataFromAdvancedFilters(List apiCollections, boolean shouldDeleteRequest){ + public static void removeUnnecessaryEndpoints( + List apiCollections, boolean shouldDeleteRequest) { try { - for (ApiCollection apiCollection: apiCollections) { + for (ApiCollection apiCollection : apiCollections) { List toBeDeleted = new ArrayList<>(); if (apiCollection.getHostName() == null) { continue; } - List endpoints = com.akto.action.observe.Utils.fetchEndpointsInCollectionUsingHost(apiCollection.getId(), 0); + List endpoints = + com.akto.action.observe.Utils.fetchEndpointsInCollectionUsingHost( + apiCollection.getId(), 0); if (endpoints == null || endpoints.isEmpty()) { continue; } - logger.info("[BadApisRemover] Starting for APICollection: " + apiCollection.getId(), LogDb.DASHBOARD); - for (BasicDBObject singleTypeInfo: endpoints) { - singleTypeInfo = (BasicDBObject) (singleTypeInfo.getOrDefault("_id", new BasicDBObject())); + logger.info( + "[BadApisRemover] Starting for APICollection: " + apiCollection.getId(), + LogDb.DASHBOARD); + for (BasicDBObject singleTypeInfo : endpoints) { + singleTypeInfo = + (BasicDBObject) + (singleTypeInfo.getOrDefault("_id", new BasicDBObject())); int apiCollectionId = singleTypeInfo.getInt("apiCollectionId"); String url = singleTypeInfo.getString("url"); String method = singleTypeInfo.getString("method"); @@ -300,41 +354,60 @@ public static void removeUnnecessaryEndpoints(List apiCollections Key key = new Key(apiCollectionId, url, Method.fromString(method), -1, 0, 0); if (method.equalsIgnoreCase("options")) { - logger.info("[BadApisRemover] OPTIONS Deleting bad API: " + key, LogDb.DASHBOARD); + logger.info( + "[BadApisRemover] OPTIONS Deleting bad API: " + key, + LogDb.DASHBOARD); toBeDeleted.add(key); continue; } if (!method.equalsIgnoreCase("get")) { - logger.info("[BadApisRemover] Non-get Deleting bad API: " + key, LogDb.DASHBOARD); + logger.info( + "[BadApisRemover] Non-get Deleting bad API: " + key, + LogDb.DASHBOARD); continue; } Bson filter = ApiInfoDao.getFilter(url, method, apiCollectionId); - + SampleData sampleData = SampleDataDao.instance.findOne(filter); - if (sampleData == null || sampleData.getSamples() == null || sampleData.getSamples().isEmpty()) { - Bson stiFilterReq = Filters.and( - Filters.eq("url", url), - Filters.eq("method", method), - Filters.in("responseCode", new Integer[]{-1, 200, 201, 204, 302}), - Filters.eq("isHeader", false), - Filters.or(Filters.eq("isUrlParam", false), Filters.exists("isUrlParam", false)), - Filters.eq("apiCollectionId", apiCollectionId) - ); - SingleTypeInfo singleTypeInfoForApi = SingleTypeInfoDao.instance.findOne(stiFilterReq); + if (sampleData == null + || sampleData.getSamples() == null + || sampleData.getSamples().isEmpty()) { + Bson stiFilterReq = + Filters.and( + Filters.eq("url", url), + Filters.eq("method", method), + Filters.in( + "responseCode", + new Integer[] {-1, 200, 201, 204, 302}), + Filters.eq("isHeader", false), + Filters.or( + Filters.eq("isUrlParam", false), + Filters.exists("isUrlParam", false)), + Filters.eq("apiCollectionId", apiCollectionId)); + SingleTypeInfo singleTypeInfoForApi = + SingleTypeInfoDao.instance.findOne(stiFilterReq); if (singleTypeInfoForApi == null) { - logger.info("[BadApisRemover] no-sample Deleting bad API: " + key, LogDb.DASHBOARD); - toBeDeleted.add(key); + logger.info( + "[BadApisRemover] no-sample Deleting bad API: " + key, + LogDb.DASHBOARD); + toBeDeleted.add(key); } else { - logger.info("[BadApisRemover] yes-sti Deleting bad API: " + key + " " + singleTypeInfoForApi.composeKey(), LogDb.DASHBOARD); + logger.info( + "[BadApisRemover] yes-sti Deleting bad API: " + + key + + " " + + singleTypeInfoForApi.composeKey(), + LogDb.DASHBOARD); } } else { - logger.info("[BadApisRemover] yes-sample Deleting bad API: " + key, LogDb.DASHBOARD); + logger.info( + "[BadApisRemover] yes-sample Deleting bad API: " + key, + LogDb.DASHBOARD); } } - if (shouldDeleteRequest) { logger.info("starting deletion of apis"); deleteApis(toBeDeleted); @@ -342,24 +415,28 @@ public static void removeUnnecessaryEndpoints(List apiCollections } } catch (Exception e) { - loggerMaker.errorAndAddToDb("Couldn't complete scan for APIs remover: " + e.getMessage(), LogDb.DASHBOARD); + loggerMaker.errorAndAddToDb( + "Couldn't complete scan for APIs remover: " + e.getMessage(), LogDb.DASHBOARD); e.printStackTrace(); } } - public static void deleteOptionsAPIs(List apiCollections){ - for (ApiCollection apiCollection: apiCollections) { + public static void deleteOptionsAPIs(List apiCollections) { + for (ApiCollection apiCollection : apiCollections) { List toBeDeleted = new ArrayList<>(); if (apiCollection.getHostName() == null) { continue; } - List endpoints = com.akto.action.observe.Utils.fetchEndpointsInCollectionUsingHost(apiCollection.getId(), 0); + List endpoints = + com.akto.action.observe.Utils.fetchEndpointsInCollectionUsingHost( + apiCollection.getId(), 0); if (endpoints == null || endpoints.isEmpty()) { continue; } - for (BasicDBObject singleTypeInfo: endpoints) { - singleTypeInfo = (BasicDBObject) (singleTypeInfo.getOrDefault("_id", new BasicDBObject())); + for (BasicDBObject singleTypeInfo : endpoints) { + singleTypeInfo = + (BasicDBObject) (singleTypeInfo.getOrDefault("_id", new BasicDBObject())); int apiCollectionId = singleTypeInfo.getInt("apiCollectionId"); String url = singleTypeInfo.getString("url"); String method = singleTypeInfo.getString("method"); @@ -374,5 +451,4 @@ public static void deleteOptionsAPIs(List apiCollections){ deleteApis(toBeDeleted); } } - -} \ No newline at end of file +} diff --git a/libs/dao/src/main/java/com/akto/dao/monitoring/FilterYamlTemplateDao.java b/libs/dao/src/main/java/com/akto/dao/monitoring/FilterYamlTemplateDao.java index 8a898c387c..1e6ad20238 100644 --- a/libs/dao/src/main/java/com/akto/dao/monitoring/FilterYamlTemplateDao.java +++ b/libs/dao/src/main/java/com/akto/dao/monitoring/FilterYamlTemplateDao.java @@ -13,17 +13,23 @@ public class FilterYamlTemplateDao extends AccountsContextDao { public static final FilterYamlTemplateDao instance = new FilterYamlTemplateDao(); - public Map fetchFilterConfig(boolean includeYamlContent, boolean shouldParseExecutor) { + public Map fetchFilterConfig( + boolean includeYamlContent, boolean shouldParseExecutor) { List yamlTemplates = FilterYamlTemplateDao.instance.findAll(Filters.empty()); return fetchFilterConfig(includeYamlContent, yamlTemplates, shouldParseExecutor); } - public Map fetchFilterConfig(boolean includeYamlContent, List yamlTemplates, boolean shouldParseExecutor) { + public static Map fetchFilterConfig( + boolean includeYamlContent, + List yamlTemplates, + boolean shouldParseExecutor) { Map filterConfigMap = new HashMap<>(); for (YamlTemplate yamlTemplate : yamlTemplates) { try { if (yamlTemplate != null) { - FilterConfig filterConfig = FilterConfigYamlParser.parseTemplate(yamlTemplate.getContent(), shouldParseExecutor); + FilterConfig filterConfig = + FilterConfigYamlParser.parseTemplate( + yamlTemplate.getContent(), shouldParseExecutor); filterConfig.setAuthor(yamlTemplate.getAuthor()); filterConfig.setCreatedAt(yamlTemplate.getCreatedAt()); filterConfig.setUpdatedAt(yamlTemplate.getUpdatedAt()); diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java index a0c502b6f0..a88e6e891d 100644 --- a/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java +++ b/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java @@ -31,9 +31,9 @@ public SampleMaliciousRequest( this.url = responseParams.getRequestParams().getURL(); this.method = Method.fromString(responseParams.getRequestParams().getMethod()); - // For now, we are hardcoding it to 1 hr. + // For now, we are hardcoding it to 3 hrs. // But later we will read it through FilterConfig - this.expiry = now + (60 * 60); + this.expiry = now + (3 * 60 * 60); } public static int generateBinId(HttpResponseParams responseParam) { diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceGrpc.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceGrpc.java new file mode 100644 index 0000000000..d9e12f2b9d --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceGrpc.java @@ -0,0 +1,293 @@ +package com.akto.proto.threat_protection.service.consumer_service.v1; + +import static io.grpc.MethodDescriptor.generateFullMethodName; + +/** + */ +@javax.annotation.Generated( + value = "by gRPC proto compiler (version 1.68.1)", + comments = "Source: threat_protection/service/consumer_service/v1/consumer_service.proto") +@io.grpc.stub.annotations.GrpcGenerated +public final class ConsumerServiceGrpc { + + private ConsumerServiceGrpc() {} + + public static final java.lang.String SERVICE_NAME = "threat_protection.service.consumer_service.v1.ConsumerService"; + + // Static method descriptors that strictly reflect the proto. + private static volatile io.grpc.MethodDescriptor getRecordAlertMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "RecordAlert", + requestType = com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.class, + responseType = com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getRecordAlertMethod() { + io.grpc.MethodDescriptor getRecordAlertMethod; + if ((getRecordAlertMethod = ConsumerServiceGrpc.getRecordAlertMethod) == null) { + synchronized (ConsumerServiceGrpc.class) { + if ((getRecordAlertMethod = ConsumerServiceGrpc.getRecordAlertMethod) == null) { + ConsumerServiceGrpc.getRecordAlertMethod = getRecordAlertMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "RecordAlert")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.getDefaultInstance())) + .setSchemaDescriptor(new ConsumerServiceMethodDescriptorSupplier("RecordAlert")) + .build(); + } + } + } + return getRecordAlertMethod; + } + + /** + * Creates a new async stub that supports all call types for the service + */ + public static ConsumerServiceStub newStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public ConsumerServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceStub(channel, callOptions); + } + }; + return ConsumerServiceStub.newStub(factory, channel); + } + + /** + * Creates a new blocking-style stub that supports unary and streaming output calls on the service + */ + public static ConsumerServiceBlockingStub newBlockingStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public ConsumerServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceBlockingStub(channel, callOptions); + } + }; + return ConsumerServiceBlockingStub.newStub(factory, channel); + } + + /** + * Creates a new ListenableFuture-style stub that supports unary calls on the service + */ + public static ConsumerServiceFutureStub newFutureStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public ConsumerServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceFutureStub(channel, callOptions); + } + }; + return ConsumerServiceFutureStub.newStub(factory, channel); + } + + /** + */ + public interface AsyncService { + + /** + */ + default void recordAlert(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getRecordAlertMethod(), responseObserver); + } + } + + /** + * Base class for the server implementation of the service ConsumerService. + */ + public static abstract class ConsumerServiceImplBase + implements io.grpc.BindableService, AsyncService { + + @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { + return ConsumerServiceGrpc.bindService(this); + } + } + + /** + * A stub to allow clients to do asynchronous rpc calls to service ConsumerService. + */ + public static final class ConsumerServiceStub + extends io.grpc.stub.AbstractAsyncStub { + private ConsumerServiceStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected ConsumerServiceStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceStub(channel, callOptions); + } + + /** + */ + public void recordAlert(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getRecordAlertMethod(), getCallOptions()), request, responseObserver); + } + } + + /** + * A stub to allow clients to do synchronous rpc calls to service ConsumerService. + */ + public static final class ConsumerServiceBlockingStub + extends io.grpc.stub.AbstractBlockingStub { + private ConsumerServiceBlockingStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected ConsumerServiceBlockingStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceBlockingStub(channel, callOptions); + } + + /** + */ + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse recordAlert(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getRecordAlertMethod(), getCallOptions(), request); + } + } + + /** + * A stub to allow clients to do ListenableFuture-style rpc calls to service ConsumerService. + */ + public static final class ConsumerServiceFutureStub + extends io.grpc.stub.AbstractFutureStub { + private ConsumerServiceFutureStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected ConsumerServiceFutureStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceFutureStub(channel, callOptions); + } + + /** + */ + public com.google.common.util.concurrent.ListenableFuture recordAlert( + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getRecordAlertMethod(), getCallOptions()), request); + } + } + + private static final int METHODID_RECORD_ALERT = 0; + + private static final class MethodHandlers implements + io.grpc.stub.ServerCalls.UnaryMethod, + io.grpc.stub.ServerCalls.ServerStreamingMethod, + io.grpc.stub.ServerCalls.ClientStreamingMethod, + io.grpc.stub.ServerCalls.BidiStreamingMethod { + private final AsyncService serviceImpl; + private final int methodId; + + MethodHandlers(AsyncService serviceImpl, int methodId) { + this.serviceImpl = serviceImpl; + this.methodId = methodId; + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + case METHODID_RECORD_ALERT: + serviceImpl.recordAlert((com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + default: + throw new AssertionError(); + } + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public io.grpc.stub.StreamObserver invoke( + io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + default: + throw new AssertionError(); + } + } + } + + public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { + return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) + .addMethod( + getRecordAlertMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest, + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse>( + service, METHODID_RECORD_ALERT))) + .build(); + } + + private static abstract class ConsumerServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { + ConsumerServiceBaseDescriptorSupplier() {} + + @java.lang.Override + public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.getDescriptor(); + } + + @java.lang.Override + public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { + return getFileDescriptor().findServiceByName("ConsumerService"); + } + } + + private static final class ConsumerServiceFileDescriptorSupplier + extends ConsumerServiceBaseDescriptorSupplier { + ConsumerServiceFileDescriptorSupplier() {} + } + + private static final class ConsumerServiceMethodDescriptorSupplier + extends ConsumerServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { + private final java.lang.String methodName; + + ConsumerServiceMethodDescriptorSupplier(java.lang.String methodName) { + this.methodName = methodName; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { + return getServiceDescriptor().findMethodByName(methodName); + } + } + + private static volatile io.grpc.ServiceDescriptor serviceDescriptor; + + public static io.grpc.ServiceDescriptor getServiceDescriptor() { + io.grpc.ServiceDescriptor result = serviceDescriptor; + if (result == null) { + synchronized (ConsumerServiceGrpc.class) { + result = serviceDescriptor; + if (result == null) { + serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) + .setSchemaDescriptor(new ConsumerServiceFileDescriptorSupplier()) + .addMethod(getRecordAlertMethod()) + .build(); + } + } + } + return result; + } +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceProto.java new file mode 100644 index 0000000000..f1960e1aab --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceProto.java @@ -0,0 +1,106 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.consumer_service.v1; + +public final class ConsumerServiceProto { + private ConsumerServiceProto() {} + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + ConsumerServiceProto.class.getName()); + } + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\nDthreat_protection/service/consumer_ser" + + "vice/v1/consumer_service.proto\022-threat_p" + + "rotection.service.consumer_service.v1\"\025\n" + + "\023RecordAlertResponse\"\264\001\n\024SampleMalicious" + + "Event\022\016\n\002ip\030\001 \001(\tR\002ip\022\034\n\ttimestamp\030\002 \001(\003" + + "R\ttimestamp\022\020\n\003url\030\003 \001(\tR\003url\022\026\n\006method\030" + + "\004 \001(\tR\006method\022*\n\021api_collection_id\030\005 \001(\005" + + "R\017apiCollectionId\022\030\n\007payload\030\006 \001(\tR\007payl" + + "oad\"\361\001\n\022RecordAlertRequest\022\024\n\005actor\030\001 \001(" + + "\tR\005actor\022\033\n\tfilter_id\030\002 \001(\tR\010filterId\022!\n" + + "\014total_events\030\003 \001(\005R\013totalEvents\022\037\n\013dete" + + "cted_at\030\004 \001(\003R\ndetectedAt\022d\n\013sample_data" + + "\030\005 \003(\0132C.threat_protection.service.consu" + + "mer_service.v1.SampleMaliciousEventR\nsam" + + "pleData2\252\001\n\017ConsumerService\022\226\001\n\013RecordAl" + + "ert\022A.threat_protection.service.consumer" + + "_service.v1.RecordAlertRequest\032B.threat_" + + "protection.service.consumer_service.v1.R" + + "ecordAlertResponse\"\000B\243\002\n builder) { + super(builder); + } + private RecordAlertRequest() { + actor_ = ""; + filterId_ = ""; + sampleData_ = java.util.Collections.emptyList(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.class, com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.Builder.class); + } + + public static final int ACTOR_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + @java.lang.Override + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TOTAL_EVENTS_FIELD_NUMBER = 3; + private int totalEvents_ = 0; + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @return The totalEvents. + */ + @java.lang.Override + public int getTotalEvents() { + return totalEvents_; + } + + public static final int DETECTED_AT_FIELD_NUMBER = 4; + private long detectedAt_ = 0L; + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + + public static final int SAMPLE_DATA_FIELD_NUMBER = 5; + @SuppressWarnings("serial") + private java.util.List sampleData_; + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public java.util.List getSampleDataList() { + return sampleData_; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public java.util.List + getSampleDataOrBuilderList() { + return sampleData_; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public int getSampleDataCount() { + return sampleData_.size(); + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getSampleData(int index) { + return sampleData_.get(index); + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( + int index) { + return sampleData_.get(index); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); + } + if (totalEvents_ != 0) { + output.writeInt32(3, totalEvents_); + } + if (detectedAt_ != 0L) { + output.writeInt64(4, detectedAt_); + } + for (int i = 0; i < sampleData_.size(); i++) { + output.writeMessage(5, sampleData_.get(i)); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); + } + if (totalEvents_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(3, totalEvents_); + } + if (detectedAt_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(4, detectedAt_); + } + for (int i = 0; i < sampleData_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(5, sampleData_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest other = (com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest) obj; + + if (!getActor() + .equals(other.getActor())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (getTotalEvents() + != other.getTotalEvents()) return false; + if (getDetectedAt() + != other.getDetectedAt()) return false; + if (!getSampleDataList() + .equals(other.getSampleDataList())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACTOR_FIELD_NUMBER; + hash = (53 * hash) + getActor().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (37 * hash) + TOTAL_EVENTS_FIELD_NUMBER; + hash = (53 * hash) + getTotalEvents(); + hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getDetectedAt()); + if (getSampleDataCount() > 0) { + hash = (37 * hash) + SAMPLE_DATA_FIELD_NUMBER; + hash = (53 * hash) + getSampleDataList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.consumer_service.v1.RecordAlertRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.consumer_service.v1.RecordAlertRequest) + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.class, com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + actor_ = ""; + filterId_ = ""; + totalEvents_ = 0; + detectedAt_ = 0L; + if (sampleDataBuilder_ == null) { + sampleData_ = java.util.Collections.emptyList(); + } else { + sampleData_ = null; + sampleDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest build() { + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest buildPartial() { + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest result = new com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest(this); + buildPartialRepeatedFields(result); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartialRepeatedFields(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest result) { + if (sampleDataBuilder_ == null) { + if (((bitField0_ & 0x00000010) != 0)) { + sampleData_ = java.util.Collections.unmodifiableList(sampleData_); + bitField0_ = (bitField0_ & ~0x00000010); + } + result.sampleData_ = sampleData_; + } else { + result.sampleData_ = sampleDataBuilder_.build(); + } + } + + private void buildPartial0(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.actor_ = actor_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.filterId_ = filterId_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.totalEvents_ = totalEvents_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.detectedAt_ = detectedAt_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest) { + return mergeFrom((com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest other) { + if (other == com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.getDefaultInstance()) return this; + if (!other.getActor().isEmpty()) { + actor_ = other.actor_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (other.getTotalEvents() != 0) { + setTotalEvents(other.getTotalEvents()); + } + if (other.getDetectedAt() != 0L) { + setDetectedAt(other.getDetectedAt()); + } + if (sampleDataBuilder_ == null) { + if (!other.sampleData_.isEmpty()) { + if (sampleData_.isEmpty()) { + sampleData_ = other.sampleData_; + bitField0_ = (bitField0_ & ~0x00000010); + } else { + ensureSampleDataIsMutable(); + sampleData_.addAll(other.sampleData_); + } + onChanged(); + } + } else { + if (!other.sampleData_.isEmpty()) { + if (sampleDataBuilder_.isEmpty()) { + sampleDataBuilder_.dispose(); + sampleDataBuilder_ = null; + sampleData_ = other.sampleData_; + bitField0_ = (bitField0_ & ~0x00000010); + sampleDataBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getSampleDataFieldBuilder() : null; + } else { + sampleDataBuilder_.addAllMessages(other.sampleData_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + actor_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + totalEvents_ = input.readInt32(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 32: { + detectedAt_ = input.readInt64(); + bitField0_ |= 0x00000008; + break; + } // case 32 + case 42: { + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent m = + input.readMessage( + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.parser(), + extensionRegistry); + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.add(m); + } else { + sampleDataBuilder_.addMessage(m); + } + break; + } // case 42 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object actor_ = ""; + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + public java.lang.String getActor() { + java.lang.Object ref = actor_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actor_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + public com.google.protobuf.ByteString + getActorBytes() { + java.lang.Object ref = actor_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The actor to set. + * @return This builder for chaining. + */ + public Builder setActor( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @return This builder for chaining. + */ + public Builder clearActor() { + actor_ = getDefaultInstance().getActor(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string actor = 1 [json_name = "actor"]; + * @param value The bytes for actor to set. + * @return This builder for chaining. + */ + public Builder setActorBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + actor_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private int totalEvents_ ; + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @return The totalEvents. + */ + @java.lang.Override + public int getTotalEvents() { + return totalEvents_; + } + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @param value The totalEvents to set. + * @return This builder for chaining. + */ + public Builder setTotalEvents(int value) { + + totalEvents_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @return This builder for chaining. + */ + public Builder clearTotalEvents() { + bitField0_ = (bitField0_ & ~0x00000004); + totalEvents_ = 0; + onChanged(); + return this; + } + + private long detectedAt_ ; + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @param value The detectedAt to set. + * @return This builder for chaining. + */ + public Builder setDetectedAt(long value) { + + detectedAt_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @return This builder for chaining. + */ + public Builder clearDetectedAt() { + bitField0_ = (bitField0_ & ~0x00000008); + detectedAt_ = 0L; + onChanged(); + return this; + } + + private java.util.List sampleData_ = + java.util.Collections.emptyList(); + private void ensureSampleDataIsMutable() { + if (!((bitField0_ & 0x00000010) != 0)) { + sampleData_ = new java.util.ArrayList(sampleData_); + bitField0_ |= 0x00000010; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder> sampleDataBuilder_; + + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public java.util.List getSampleDataList() { + if (sampleDataBuilder_ == null) { + return java.util.Collections.unmodifiableList(sampleData_); + } else { + return sampleDataBuilder_.getMessageList(); + } + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public int getSampleDataCount() { + if (sampleDataBuilder_ == null) { + return sampleData_.size(); + } else { + return sampleDataBuilder_.getCount(); + } + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getSampleData(int index) { + if (sampleDataBuilder_ == null) { + return sampleData_.get(index); + } else { + return sampleDataBuilder_.getMessage(index); + } + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder setSampleData( + int index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent value) { + if (sampleDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleDataIsMutable(); + sampleData_.set(index, value); + onChanged(); + } else { + sampleDataBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder setSampleData( + int index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder builderForValue) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.set(index, builderForValue.build()); + onChanged(); + } else { + sampleDataBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addSampleData(com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent value) { + if (sampleDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleDataIsMutable(); + sampleData_.add(value); + onChanged(); + } else { + sampleDataBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addSampleData( + int index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent value) { + if (sampleDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSampleDataIsMutable(); + sampleData_.add(index, value); + onChanged(); + } else { + sampleDataBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addSampleData( + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder builderForValue) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.add(builderForValue.build()); + onChanged(); + } else { + sampleDataBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addSampleData( + int index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder builderForValue) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.add(index, builderForValue.build()); + onChanged(); + } else { + sampleDataBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder addAllSampleData( + java.lang.Iterable values) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, sampleData_); + onChanged(); + } else { + sampleDataBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder clearSampleData() { + if (sampleDataBuilder_ == null) { + sampleData_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + } else { + sampleDataBuilder_.clear(); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public Builder removeSampleData(int index) { + if (sampleDataBuilder_ == null) { + ensureSampleDataIsMutable(); + sampleData_.remove(index); + onChanged(); + } else { + sampleDataBuilder_.remove(index); + } + return this; + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder getSampleDataBuilder( + int index) { + return getSampleDataFieldBuilder().getBuilder(index); + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( + int index) { + if (sampleDataBuilder_ == null) { + return sampleData_.get(index); } else { + return sampleDataBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public java.util.List + getSampleDataOrBuilderList() { + if (sampleDataBuilder_ != null) { + return sampleDataBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(sampleData_); + } + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder addSampleDataBuilder() { + return getSampleDataFieldBuilder().addBuilder( + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.getDefaultInstance()); + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder addSampleDataBuilder( + int index) { + return getSampleDataFieldBuilder().addBuilder( + index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.getDefaultInstance()); + } + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + public java.util.List + getSampleDataBuilderList() { + return getSampleDataFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder> + getSampleDataFieldBuilder() { + if (sampleDataBuilder_ == null) { + sampleDataBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder>( + sampleData_, + ((bitField0_ & 0x00000010) != 0), + getParentForChildren(), + isClean()); + sampleData_ = null; + } + return sampleDataBuilder_; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.consumer_service.v1.RecordAlertRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.consumer_service.v1.RecordAlertRequest) + private static final com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest(); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RecordAlertRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequestOrBuilder.java new file mode 100644 index 0000000000..108e4ea234 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequestOrBuilder.java @@ -0,0 +1,71 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.consumer_service.v1; + +public interface RecordAlertRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.consumer_service.v1.RecordAlertRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * string actor = 1 [json_name = "actor"]; + * @return The actor. + */ + java.lang.String getActor(); + /** + * string actor = 1 [json_name = "actor"]; + * @return The bytes for actor. + */ + com.google.protobuf.ByteString + getActorBytes(); + + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + java.lang.String getFilterId(); + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + com.google.protobuf.ByteString + getFilterIdBytes(); + + /** + * int32 total_events = 3 [json_name = "totalEvents"]; + * @return The totalEvents. + */ + int getTotalEvents(); + + /** + * int64 detected_at = 4 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + long getDetectedAt(); + + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + java.util.List + getSampleDataList(); + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getSampleData(int index); + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + int getSampleDataCount(); + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + java.util.List + getSampleDataOrBuilderList(); + /** + * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; + */ + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( + int index); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponse.java new file mode 100644 index 0000000000..dcc5c5325c --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponse.java @@ -0,0 +1,358 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.consumer_service.v1; + +/** + * Protobuf type {@code threat_protection.service.consumer_service.v1.RecordAlertResponse} + */ +public final class RecordAlertResponse extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.consumer_service.v1.RecordAlertResponse) + RecordAlertResponseOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + RecordAlertResponse.class.getName()); + } + // Use RecordAlertResponse.newBuilder() to construct. + private RecordAlertResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private RecordAlertResponse() { + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.class, com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.Builder.class); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse other = (com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse) obj; + + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.consumer_service.v1.RecordAlertResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.consumer_service.v1.RecordAlertResponse) + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.class, com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse build() { + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse buildPartial() { + com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse result = new com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse) { + return mergeFrom((com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse other) { + if (other == com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.consumer_service.v1.RecordAlertResponse) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.consumer_service.v1.RecordAlertResponse) + private static final com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse(); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RecordAlertResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponseOrBuilder.java new file mode 100644 index 0000000000..2c1bd6c68c --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponseOrBuilder.java @@ -0,0 +1,11 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.consumer_service.v1; + +public interface RecordAlertResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.consumer_service.v1.RecordAlertResponse) + com.google.protobuf.MessageOrBuilder { +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEvent.java new file mode 100644 index 0000000000..78a761f47d --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEvent.java @@ -0,0 +1,1042 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.consumer_service.v1; + +/** + * Protobuf type {@code threat_protection.service.consumer_service.v1.SampleMaliciousEvent} + */ +public final class SampleMaliciousEvent extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) + SampleMaliciousEventOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SampleMaliciousEvent.class.getName()); + } + // Use SampleMaliciousEvent.newBuilder() to construct. + private SampleMaliciousEvent(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SampleMaliciousEvent() { + ip_ = ""; + url_ = ""; + method_ = ""; + payload_ = ""; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.class, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder.class); + } + + public static final int IP_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object ip_ = ""; + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + @java.lang.Override + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TIMESTAMP_FIELD_NUMBER = 2; + private long timestamp_ = 0L; + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + + public static final int URL_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object url_ = ""; + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + @java.lang.Override + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } + } + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int METHOD_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object method_ = ""; + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + @java.lang.Override + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } + } + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int API_COLLECTION_ID_FIELD_NUMBER = 5; + private int apiCollectionId_ = 0; + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + + public static final int PAYLOAD_FIELD_NUMBER = 6; + @SuppressWarnings("serial") + private volatile java.lang.Object payload_ = ""; + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + @java.lang.Override + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, ip_); + } + if (timestamp_ != 0L) { + output.writeInt64(2, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 3, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 4, method_); + } + if (apiCollectionId_ != 0) { + output.writeInt32(5, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 6, payload_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, ip_); + } + if (timestamp_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(2, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(3, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(4, method_); + } + if (apiCollectionId_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(5, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(6, payload_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent other = (com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent) obj; + + if (!getIp() + .equals(other.getIp())) return false; + if (getTimestamp() + != other.getTimestamp()) return false; + if (!getUrl() + .equals(other.getUrl())) return false; + if (!getMethod() + .equals(other.getMethod())) return false; + if (getApiCollectionId() + != other.getApiCollectionId()) return false; + if (!getPayload() + .equals(other.getPayload())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + IP_FIELD_NUMBER; + hash = (53 * hash) + getIp().hashCode(); + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getTimestamp()); + hash = (37 * hash) + URL_FIELD_NUMBER; + hash = (53 * hash) + getUrl().hashCode(); + hash = (37 * hash) + METHOD_FIELD_NUMBER; + hash = (53 * hash) + getMethod().hashCode(); + hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; + hash = (53 * hash) + getApiCollectionId(); + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.service.consumer_service.v1.SampleMaliciousEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.class, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ip_ = ""; + timestamp_ = 0L; + url_ = ""; + method_ = ""; + apiCollectionId_ = 0; + payload_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getDefaultInstanceForType() { + return com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent build() { + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent buildPartial() { + com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent result = new com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ip_ = ip_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.timestamp_ = timestamp_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.url_ = url_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.method_ = method_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.apiCollectionId_ = apiCollectionId_; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.payload_ = payload_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent) { + return mergeFrom((com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent other) { + if (other == com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.getDefaultInstance()) return this; + if (!other.getIp().isEmpty()) { + ip_ = other.ip_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.getTimestamp() != 0L) { + setTimestamp(other.getTimestamp()); + } + if (!other.getUrl().isEmpty()) { + url_ = other.url_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (!other.getMethod().isEmpty()) { + method_ = other.method_; + bitField0_ |= 0x00000008; + onChanged(); + } + if (other.getApiCollectionId() != 0) { + setApiCollectionId(other.getApiCollectionId()); + } + if (!other.getPayload().isEmpty()) { + payload_ = other.payload_; + bitField0_ |= 0x00000020; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + ip_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 26: { + url_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + method_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 40: { + apiCollectionId_ = input.readInt32(); + bitField0_ |= 0x00000010; + break; + } // case 40 + case 50: { + payload_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000020; + break; + } // case 50 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object ip_ = ""; + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string ip = 1 [json_name = "ip"]; + * @param value The ip to set. + * @return This builder for chaining. + */ + public Builder setIp( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ip_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string ip = 1 [json_name = "ip"]; + * @return This builder for chaining. + */ + public Builder clearIp() { + ip_ = getDefaultInstance().getIp(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string ip = 1 [json_name = "ip"]; + * @param value The bytes for ip to set. + * @return This builder for chaining. + */ + public Builder setIpBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ip_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private long timestamp_ ; + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + + timestamp_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000002); + timestamp_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object url_ = ""; + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string url = 3 [json_name = "url"]; + * @param value The url to set. + * @return This builder for chaining. + */ + public Builder setUrl( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + url_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string url = 3 [json_name = "url"]; + * @return This builder for chaining. + */ + public Builder clearUrl() { + url_ = getDefaultInstance().getUrl(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string url = 3 [json_name = "url"]; + * @param value The bytes for url to set. + * @return This builder for chaining. + */ + public Builder setUrlBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + url_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private java.lang.Object method_ = ""; + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string method = 4 [json_name = "method"]; + * @param value The method to set. + * @return This builder for chaining. + */ + public Builder setMethod( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + method_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * string method = 4 [json_name = "method"]; + * @return This builder for chaining. + */ + public Builder clearMethod() { + method_ = getDefaultInstance().getMethod(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * string method = 4 [json_name = "method"]; + * @param value The bytes for method to set. + * @return This builder for chaining. + */ + public Builder setMethodBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + method_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + + private int apiCollectionId_ ; + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @param value The apiCollectionId to set. + * @return This builder for chaining. + */ + public Builder setApiCollectionId(int value) { + + apiCollectionId_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return This builder for chaining. + */ + public Builder clearApiCollectionId() { + bitField0_ = (bitField0_ & ~0x00000010); + apiCollectionId_ = 0; + onChanged(); + return this; + } + + private java.lang.Object payload_ = ""; + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string payload = 6 [json_name = "payload"]; + * @param value The payload to set. + * @return This builder for chaining. + */ + public Builder setPayload( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + payload_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * string payload = 6 [json_name = "payload"]; + * @return This builder for chaining. + */ + public Builder clearPayload() { + payload_ = getDefaultInstance().getPayload(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + return this; + } + /** + * string payload = 6 [json_name = "payload"]; + * @param value The bytes for payload to set. + * @return This builder for chaining. + */ + public Builder setPayloadBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + payload_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) + } + + // @@protoc_insertion_point(class_scope:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) + private static final com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent(); + } + + public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SampleMaliciousEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEventOrBuilder.java new file mode 100644 index 0000000000..52b1b9da96 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEventOrBuilder.java @@ -0,0 +1,71 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/service/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.service.consumer_service.v1; + +public interface SampleMaliciousEventOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) + com.google.protobuf.MessageOrBuilder { + + /** + * string ip = 1 [json_name = "ip"]; + * @return The ip. + */ + java.lang.String getIp(); + /** + * string ip = 1 [json_name = "ip"]; + * @return The bytes for ip. + */ + com.google.protobuf.ByteString + getIpBytes(); + + /** + * int64 timestamp = 2 [json_name = "timestamp"]; + * @return The timestamp. + */ + long getTimestamp(); + + /** + * string url = 3 [json_name = "url"]; + * @return The url. + */ + java.lang.String getUrl(); + /** + * string url = 3 [json_name = "url"]; + * @return The bytes for url. + */ + com.google.protobuf.ByteString + getUrlBytes(); + + /** + * string method = 4 [json_name = "method"]; + * @return The method. + */ + java.lang.String getMethod(); + /** + * string method = 4 [json_name = "method"]; + * @return The bytes for method. + */ + com.google.protobuf.ByteString + getMethodBytes(); + + /** + * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + int getApiCollectionId(); + + /** + * string payload = 6 [json_name = "payload"]; + * @return The payload. + */ + java.lang.String getPayload(); + /** + * string payload = 6 [json_name = "payload"]; + * @return The bytes for payload. + */ + com.google.protobuf.ByteString + getPayloadBytes(); +} From 491cddb4c364001baa33533b8f172e1494ad47da Mon Sep 17 00:00:00 2001 From: ayushaga14 Date: Thu, 21 Nov 2024 11:21:51 +0530 Subject: [PATCH 18/73] add aggregation parse layer --- .../AggregationLayerParser.java | 37 +++++++++++++++++++ .../AggregationRules.java | 24 ++++++++++++ .../api_protection_parse_layer/Condition.java | 31 ++++++++++++++++ .../dto/api_protection_parse_layer/Rule.java | 31 ++++++++++++++++ 4 files changed, 123 insertions(+) create mode 100644 libs/dao/src/main/java/com/akto/dao/api_protection_parse_layer/AggregationLayerParser.java create mode 100644 libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/AggregationRules.java create mode 100644 libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/Condition.java create mode 100644 libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/Rule.java diff --git a/libs/dao/src/main/java/com/akto/dao/api_protection_parse_layer/AggregationLayerParser.java b/libs/dao/src/main/java/com/akto/dao/api_protection_parse_layer/AggregationLayerParser.java new file mode 100644 index 0000000000..90af26affe --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dao/api_protection_parse_layer/AggregationLayerParser.java @@ -0,0 +1,37 @@ +package com.akto.dao.api_protection_parse_layer; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import com.akto.dto.api_protection_parse_layer.AggregationRules; +import com.akto.dto.api_protection_parse_layer.Rule; +import com.fasterxml.jackson.databind.ObjectMapper; + +public class AggregationLayerParser { + + ObjectMapper objectMapper = new ObjectMapper(); + + public AggregationLayerParser() { + } + + public AggregationRules parse(Map aggregationRules) throws Exception { + + List rules = new ArrayList<>(); + AggregationRules aggRules = new AggregationRules(rules); + + try { + for (Object aggObj: (List) aggregationRules.get("aggregation_rules")) { + Map aggObjMap = (Map) aggObj; + Rule rule = objectMapper.convertValue(aggObjMap.get("rule"), Rule.class); + rules.add(rule); + } + aggRules.setRule(rules); + } catch (Exception e) { + throw e; + } + + return aggRules; + } + +} diff --git a/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/AggregationRules.java b/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/AggregationRules.java new file mode 100644 index 0000000000..18a8602009 --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/AggregationRules.java @@ -0,0 +1,24 @@ +package com.akto.dto.api_protection_parse_layer; + +import java.util.List; + +public class AggregationRules { + + private List rule; + + public AggregationRules() { + } + + public AggregationRules(List rule) { + this.rule = rule; + } + + public List getRule() { + return rule; + } + + public void setRule(List rule) { + this.rule = rule; + } + +} diff --git a/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/Condition.java b/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/Condition.java new file mode 100644 index 0000000000..bcb7a906e4 --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/Condition.java @@ -0,0 +1,31 @@ +package com.akto.dto.api_protection_parse_layer; + +public class Condition { + + private int matchCount; + private int windowThreshold; + // private String operation; + // private float value; + + public Condition() { + } + + public Condition(int matchCount, int windowThreshold) { + this.matchCount = matchCount; + this.windowThreshold = windowThreshold; + } + + public int getMatchCount() { + return matchCount; + } + public void setMatchCount(int matchCount) { + this.matchCount = matchCount; + } + public int getWindowThreshold() { + return windowThreshold; + } + public void setWindowThreshold(int windowThreshold) { + this.windowThreshold = windowThreshold; + } + // private List actions; +} diff --git a/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/Rule.java b/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/Rule.java new file mode 100644 index 0000000000..50fc2422a8 --- /dev/null +++ b/libs/dao/src/main/java/com/akto/dto/api_protection_parse_layer/Rule.java @@ -0,0 +1,31 @@ +package com.akto.dto.api_protection_parse_layer; + +public class Rule { + + String name; + Condition condition; + + public Rule() { + } + + public Rule(String name, Condition condition) { + this.name = name; + this.condition = condition; + } + + public String getName() { + return name; + } + public void setName(String name) { + this.name = name; + } + public Condition getCondition() { + return condition; + } + public void setCondition(Condition condition) { + this.condition = condition; + } + + + +} From 28bcd9a8160867c17c38702bf99dc501c28e97f6 Mon Sep 17 00:00:00 2001 From: Ajinkya <109141486+ag060@users.noreply.github.com> Date: Fri, 22 Nov 2024 11:18:14 +0530 Subject: [PATCH 19/73] grpc service for consuming malicious and smart events (#1748) * added protobuf lib * setup threat detection backend service * added grpc based service for consuming malicious and smart events --- Makefile | 4 + apps/api-threat-detection/pom.xml | 143 +- .../java/com/akto/filters/HttpCallFilter.java | 96 +- .../aggregators/window_based}/Bin.java | 2 +- .../WindowBasedThresholdNotifier.java | 22 +- .../com/akto/suspect_data/CleanUpTask.java | 109 -- .../akto/suspect_data/FlushMessagesTask.java | 85 +- .../java/com/akto/suspect_data/Message.java | 38 +- .../java/com/akto/threat/detection/Main.java | 26 +- .../WindowBasedThresholdNotifierTest.java | 25 +- apps/pom.xml | 11 + apps/threat-protection-backend/pom.xml | 182 +++ .../akto/threat/protection/BackendServer.java | 60 + .../ConsumerMaliciousEventService.java | 37 + .../java/com/akto/threat/protection/Main.java | 16 + .../akto/threat/protection/db/DBService.java | 54 + .../protection/db/MaliciousEventModel.java | 60 +- .../threat/protection/db/SmartEventModel.java | 32 +- buf.gen.yaml | 20 + libs/dao/src/main/java/com/akto/DaoInit.java | 762 ++++++---- .../DetectedThreatAlertDao.java | 20 - .../SampleMaliciousRequestDao.java | 19 - libs/pom.xml | 1 + libs/protobuf/pom.xml | 60 + .../v1/ConsumerServiceGrpc.java | 367 +++++ .../v1/ConsumerServiceProto.java | 146 ++ .../consumer_service/v1/MaliciousEvent.java | 1314 +++++++++++++++++ .../v1/MaliciousEventOrBuilder.java | 95 ++ .../v1/SaveMaliciousEventRequest.java | 785 ++++++++++ .../SaveMaliciousEventRequestOrBuilder.java | 41 + .../v1/SaveMaliciousEventResponse.java | 358 +++++ .../SaveMaliciousEventResponseOrBuilder.java | 11 + .../v1/SaveSmartEventRequest.java | 624 ++++++++ .../v1/SaveSmartEventRequestOrBuilder.java | 32 + .../v1/SaveSmartEventResponse.java | 358 +++++ .../v1/SaveSmartEventResponseOrBuilder.java | 11 + .../consumer_service/v1/SmartEvent.java | 704 +++++++++ .../v1/SmartEventOrBuilder.java | 41 + protobuf/buf.yaml | 7 + .../v1/consumer_service.proto | 47 + 40 files changed, 6114 insertions(+), 711 deletions(-) create mode 100644 Makefile rename {libs/dao/src/main/java/com/akto/dto/threat_detection => apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based}/Bin.java (84%) delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/suspect_data/CleanUpTask.java create mode 100644 apps/threat-protection-backend/pom.xml create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/DBService.java rename libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java => apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java (59%) rename libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java => apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java (51%) create mode 100644 buf.gen.yaml delete mode 100644 libs/dao/src/main/java/com/akto/dao/threat_detection/DetectedThreatAlertDao.java delete mode 100644 libs/dao/src/main/java/com/akto/dao/threat_detection/SampleMaliciousRequestDao.java create mode 100644 libs/protobuf/pom.xml create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceGrpc.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEvent.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEventOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponse.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponseOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponse.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponseOrBuilder.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEvent.java create mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java create mode 100644 protobuf/buf.yaml create mode 100644 protobuf/threat_protection/consumer_service/v1/consumer_service.proto diff --git a/Makefile b/Makefile new file mode 100644 index 0000000000..c5d4de70d7 --- /dev/null +++ b/Makefile @@ -0,0 +1,4 @@ +proto-gen: + buf lint protobuf && \ + rm -rf ./libs/protobuf/src/main/java/com/akto && \ + buf generate protobuf \ No newline at end of file diff --git a/apps/api-threat-detection/pom.xml b/apps/api-threat-detection/pom.xml index 4f5b3b1fc7..5c1a9d2fbb 100644 --- a/apps/api-threat-detection/pom.xml +++ b/apps/api-threat-detection/pom.xml @@ -1,7 +1,7 @@ + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 @@ -70,6 +70,13 @@ test + + com.akto.libs.protobuf + protobuf + 1.0-SNAPSHOT + compile + + io.lettuce lettuce-core @@ -124,79 +131,79 @@ - + - normal + normal true - - - org.apache.maven.plugins - maven-assembly-plugin - - - package - - single - - - - - - com.akto.threat.detection.Main - - - - - jar-with-dependencies - - - - - - - - + + + org.apache.maven.plugins + maven-assembly-plugin + + + package + + single + + + + + + com.akto.threat.detection.Main + + + + + jar-with-dependencies + + + + + + + + - + devcontainer - - - org.apache.maven.plugins - maven-jar-plugin - - api-threat-detection-1.0-SNAPSHOT-jar-with-dependencies - - - true - com.akto.threat.detection.Main - dependency-jars/ - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - copy-dependencies - package - - copy-dependencies - - - ${project.build.directory}/dependency-jars/ - - - - - + + + org.apache.maven.plugins + maven-jar-plugin + + api-threat-detection-1.0-SNAPSHOT-jar-with-dependencies + + + true + com.akto.threat.detection.Main + dependency-jars/ + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + package + + copy-dependencies + + + ${project.build.directory}/dependency-jars/ + + + + + - - + + - + \ No newline at end of file diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index f4e4efa8ea..9b11828a97 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -5,7 +5,6 @@ import com.akto.cache.RedisBackedCounterCache; import com.akto.dao.context.Context; import com.akto.dao.monitoring.FilterYamlTemplateDao; -import com.akto.dao.threat_detection.DetectedThreatAlertDao; import com.akto.data_actor.DataActor; import com.akto.data_actor.DataActorFactory; import com.akto.dto.ApiInfo.ApiInfoKey; @@ -13,8 +12,6 @@ import com.akto.dto.RawApi; import com.akto.dto.monitoring.FilterConfig; import com.akto.dto.test_editor.YamlTemplate; -import com.akto.dto.threat_detection.DetectedThreatAlert; -import com.akto.dto.threat_detection.SampleMaliciousRequest; import com.akto.dto.type.URLMethods.Method; import com.akto.filters.aggregators.key_generator.SourceIPKeyGenerator; import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier; @@ -22,16 +19,25 @@ import com.akto.kafka.Kafka; import com.akto.log.LoggerMaker; import com.akto.log.LoggerMaker.LogDb; +import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc.ConsumerServiceBlockingStub; +import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; +import com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest; +import com.akto.proto.threat_protection.consumer_service.v1.SmartEvent; import com.akto.rules.TestPlugin; import com.akto.suspect_data.Message; import com.akto.test_editor.execution.VariableResolver; import com.akto.test_editor.filter.data_operands_impl.ValidationResult; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.util.JsonFormat; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; import io.lettuce.core.RedisClient; public class HttpCallFilter { - private static final LoggerMaker loggerMaker = - new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); + private static final LoggerMaker loggerMaker = new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); private Map apiFilters; private final HttpCallParser httpCallParser; @@ -48,6 +54,8 @@ public class HttpCallFilter { private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; + private final ConsumerServiceBlockingStub consumerServiceBlockingStub; + public HttpCallFilter( RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { this.apiFilters = new HashMap<>(); @@ -56,10 +64,14 @@ public HttpCallFilter( String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); this.kafka = new Kafka(kafkaBrokerUrl, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); - this.windowBasedThresholdNotifier = - new WindowBasedThresholdNotifier( - new RedisBackedCounterCache(redisClient, "wbt"), - new WindowBasedThresholdNotifier.Config(100, 10 * 60)); + this.windowBasedThresholdNotifier = new WindowBasedThresholdNotifier( + new RedisBackedCounterCache(redisClient, "wbt"), + new WindowBasedThresholdNotifier.Config(100, 10 * 60)); + + String target = "localhost:8980"; + ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) + .build(); + this.consumerServiceBlockingStub = ConsumerServiceGrpc.newBlockingStub(channel); } public void filterFunction(List responseParams) { @@ -95,28 +107,40 @@ public void filterFunction(List responseParams) { actor -> { String groupKey = apiFilter.getId(); String aggKey = actor + "|" + groupKey; - SampleMaliciousRequest sampleMaliciousRequest = - new SampleMaliciousRequest( - apiFilter, actor, responseParam); + + MaliciousEvent maliciousEvent = MaliciousEvent.newBuilder().setActorId(actor) + .setFilterId(apiFilter.getId()) + .setUrl(responseParam.getRequestParams().getURL()) + .setMethod(responseParam.getRequestParams().getMethod()) + .setPayload(responseParam.getOrig()) + .setIp(actor) // For now using actor as IP + .setApiCollectionId( + responseParam.getRequestParams().getApiCollectionId()) + .setTimestamp(responseParam.getTime()) + .build(); maliciousSamples.add( new Message( responseParam.getAccountId(), - sampleMaliciousRequest)); + maliciousEvent)); - WindowBasedThresholdNotifier.Result result = - this.windowBasedThresholdNotifier.shouldNotify( - aggKey, sampleMaliciousRequest); + WindowBasedThresholdNotifier.Result result = this.windowBasedThresholdNotifier + .shouldNotify( + aggKey, maliciousEvent); if (result.shouldNotify()) { - DetectedThreatAlert alert = - new DetectedThreatAlert( - groupKey, - actor, - System.currentTimeMillis() / 1000L, - result.getBins()); - - DetectedThreatAlertDao.instance.insertOne(alert); + this.consumerServiceBlockingStub.saveSmartEvent( + SaveSmartEventRequest + .newBuilder() + .setAccountId( + Integer.parseInt(responseParam.getAccountId())) + .setEvent( + SmartEvent.newBuilder() + .setFilterId(apiFilter.getId()) + .setActorId(actor) + .setDetectedAt(responseParam.getTime()) + .build()) + .build()); } }); } @@ -128,11 +152,12 @@ public void filterFunction(List responseParams) { try { maliciousSamples.forEach( sample -> { - sample.marshall() - .ifPresent( - s -> { - kafka.send(s, KAFKA_MALICIOUS_TOPIC); - }); + try { + String data = JsonFormat.printer().print(null); + kafka.send(data, KAFKA_MALICIOUS_TOPIC); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } }); } catch (Exception e) { e.printStackTrace(); @@ -159,13 +184,12 @@ private boolean validateFilterForRequest( }, apiInfoKey); String filterExecutionLogId = UUID.randomUUID().toString(); - ValidationResult res = - TestPlugin.validateFilter( - apiFilter.getFilter().getNode(), - rawApi, - apiInfoKey, - varMap, - filterExecutionLogId); + ValidationResult res = TestPlugin.validateFilter( + apiFilter.getFilter().getNode(), + rawApi, + apiInfoKey, + varMap, + filterExecutionLogId); return res.getIsValid(); } catch (Exception e) { diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/Bin.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Bin.java similarity index 84% rename from libs/dao/src/main/java/com/akto/dto/threat_detection/Bin.java rename to apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Bin.java index df181406f2..ac7af5055f 100644 --- a/libs/dao/src/main/java/com/akto/dto/threat_detection/Bin.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Bin.java @@ -1,4 +1,4 @@ -package com.akto.dto.threat_detection; +package com.akto.filters.aggregators.window_based; public class Bin { int binId; diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index 0fe5c2ec17..af7e7d6572 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -6,8 +6,7 @@ import java.util.concurrent.ConcurrentHashMap; import com.akto.cache.CounterCache; -import com.akto.dto.threat_detection.Bin; -import com.akto.dto.threat_detection.SampleMaliciousRequest; +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; public class WindowBasedThresholdNotifier { @@ -45,20 +44,14 @@ public int getNotificationCooldownInSeconds() { public static class Result { private final boolean shouldNotify; - private final List bins; - public Result(boolean shouldNotify, List bins) { + public Result(boolean shouldNotify) { this.shouldNotify = shouldNotify; - this.bins = bins; } public boolean shouldNotify() { return shouldNotify; } - - public List getBins() { - return bins; - } } public Config getConfig() { @@ -73,8 +66,8 @@ public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.notifiedMap = new ConcurrentHashMap<>(); } - public Result shouldNotify(String aggKey, SampleMaliciousRequest sampleMaliciousRequest) { - int binId = sampleMaliciousRequest.getBinId(); + public Result shouldNotify(String aggKey, MaliciousEvent maliciousEvent) { + int binId = (int) maliciousEvent.getTimestamp() / 60; String cacheKey = aggKey + "|" + binId; this.cache.increment(cacheKey); @@ -89,15 +82,14 @@ public Result shouldNotify(String aggKey, SampleMaliciousRequest sampleMalicious long now = System.currentTimeMillis() / 1000L; long lastNotified = this.notifiedMap.getOrDefault(aggKey, 0L); - boolean cooldownBreached = - (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); + boolean cooldownBreached = (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); if (thresholdBreached && cooldownBreached) { this.notifiedMap.put(aggKey, now); - return new Result(true, bins); + return new Result(true); } - return new Result(false, bins); + return new Result(false); } public List getBins(String aggKey, int binStart, int binEnd) { diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/CleanUpTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/CleanUpTask.java deleted file mode 100644 index dd40bc0f20..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/CleanUpTask.java +++ /dev/null @@ -1,109 +0,0 @@ -package com.akto.suspect_data; - -import java.util.*; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; - -import com.akto.dao.threat_detection.CleanupAuditDao; -import com.akto.dao.threat_detection.DetectedThreatAlertDao; -import com.akto.dao.threat_detection.SampleMaliciousRequestDao; -import com.akto.dto.Account; -import com.akto.dto.threat_detection.CleanupAudit; -import com.akto.util.AccountTask; -import com.mongodb.BasicDBList; -import com.mongodb.BasicDBObject; -import com.mongodb.client.MongoCursor; -import com.mongodb.client.model.Accumulators; -import com.mongodb.client.model.Aggregates; -import com.mongodb.client.model.Filters; -import com.mongodb.client.model.Projections; -import org.bson.conversions.Bson; - -public class CleanUpTask { - private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(2); - - private CleanUpTask() {} - - public static CleanUpTask instance = new CleanUpTask(); - - public void init() { - this.executor.scheduleAtFixedRate(this::cleanUp, 0, 5, TimeUnit.HOURS); - } - - public void cleanUp() { - AccountTask.instance.executeTask( - new Consumer() { - @Override - public void accept(Account account) { - cleanUpForAccount(); - } - }, - "cleanup-malicious-requests"); - } - - public void cleanUpForAccount() { - // Remove all the requests that have passed their expiry. - // AND those requests whose actor and filter that don't have any alerts associated with - // them. - long now = System.currentTimeMillis() / 1000L; - - // Get the latest cleanup audit if exists - Optional audit = CleanupAuditDao.instance.getLatestEntry(); - long start = audit.map(CleanupAudit::getAlertWindowEnd).orElse(0L); - - List pipeline = - Arrays.asList( - Aggregates.match( - Filters.and( - Filters.gte("detectedAt", start), - Filters.lt("detectedAt", now))), - Aggregates.group( - 0, - Accumulators.addToSet( - "validFilters", - new BasicDBObject("filterId", "$filterId") - .append("actor", "$actor"))), - Aggregates.project( - Projections.fields( - Projections.include("validFilters"), - Projections.excludeId()))); - - try (MongoCursor result = - DetectedThreatAlertDao.instance - .getMCollection() - .aggregate(pipeline, BasicDBObject.class) - .cursor()) { - - BasicDBObject validFilters = result.tryNext(); - if (validFilters == null) { - return; - } - - BasicDBList filters = (BasicDBList) validFilters.get("validFilters"); - List filterList = new ArrayList<>(); - for (Object filter : filters) { - BasicDBObject filterObj = (BasicDBObject) filter; - filterList.add( - new BasicDBObject("filterId", filterObj.getString("filterId")) - .append("actor", filterObj.getString("actor"))); - } - - // Remove all the requests that have passed their expiry. - SampleMaliciousRequestDao.instance - .getMCollection() - .deleteMany( - Filters.and( - Filters.lt("expiry", now), - Filters.nor( - filterList.stream() - .map(Filters::and) - .toArray(Bson[]::new)))); - - // TODO: For any given filter, only keep last 1000 requests - - CleanupAuditDao.instance.insertOne(new CleanupAudit(start, now)); - } - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java index 53d021157a..2ff4551c3c 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java @@ -10,9 +10,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import com.akto.dao.threat_detection.SampleMaliciousRequestDao; -import com.akto.dto.threat_detection.SampleMaliciousRequest; -import com.mongodb.client.model.BulkWriteOptions; +import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; +import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc.ConsumerServiceStub; +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; +import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest; +import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse; + import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -20,13 +23,19 @@ import com.akto.dao.context.Context; import com.akto.runtime.utils.Utils; -import com.mongodb.client.model.InsertOneModel; -import com.mongodb.client.model.WriteModel; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.util.JsonFormat; + +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import io.grpc.stub.StreamObserver; public class FlushMessagesTask { private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); private final Consumer consumer; + private final ConsumerServiceStub asyncStub; private FlushMessagesTask() { String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); @@ -34,6 +43,14 @@ private FlushMessagesTask() { Properties properties = Utils.configProperties(kafkaBrokerUrl, groupId, 100); this.consumer = new KafkaConsumer<>(properties); + + // String target = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); + // TODO: Secure this connection + String target = "localhost:8980"; + ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) + .build(); + + this.asyncStub = ConsumerServiceGrpc.newStub(channel); } public static FlushMessagesTask instance = new FlushMessagesTask(); @@ -46,8 +63,7 @@ public void init() { public void run() { while (true) { try { - ConsumerRecords records = - consumer.poll(Duration.ofMillis(100)); + ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); processRecords(records); } catch (Exception e) { e.printStackTrace(); @@ -59,33 +75,44 @@ public void run() { } public void processRecords(ConsumerRecords records) { - Map> accWiseMessages = new HashMap<>(); + Map> accWiseMessages = new HashMap<>(); for (ConsumerRecord record : records) { - String msgStr = record.value(); - Message.unmarshall(msgStr) - .ifPresent( - msg -> { - accWiseMessages - .computeIfAbsent(msg.getAccountId(), k -> new ArrayList<>()) - .add(msg.getData()); - }); + try { + MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); + JsonFormat.parser().merge(record.value(), builder); + MaliciousEvent event = builder.build(); + accWiseMessages.computeIfAbsent(record.key(), k -> new ArrayList<>()).add(event); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } } - for (Map.Entry> entry : accWiseMessages.entrySet()) { - String accountId = entry.getKey(); - List sampleDatas = entry.getValue(); - Context.accountId.set(Integer.parseInt(accountId)); + for (Map.Entry> entry : accWiseMessages.entrySet()) { + int accountId = Integer.parseInt(entry.getKey()); + List events = entry.getValue(); + Context.accountId.set(accountId); - try { - List> bulkUpdates = new ArrayList<>(); - sampleDatas.forEach( - sampleData -> bulkUpdates.add(new InsertOneModel<>(sampleData))); + this.asyncStub.saveMaliciousEvent( + SaveMaliciousEventRequest.newBuilder().setAccountId(accountId).addAllEvents(events).build(), + new StreamObserver() { + @Override + public void onNext(SaveMaliciousEventResponse value) { + // Do nothing + } - SampleMaliciousRequestDao.instance.bulkWrite( - bulkUpdates, new BulkWriteOptions().ordered(false)); - } catch (Exception e) { - e.printStackTrace(); - } + @Override + public void onError(Throwable t) { + t.printStackTrace(); + } + + @Override + public void onCompleted() { + // Do nothing + System.out.println(String.format( + "Saved malicious events for account: %d. Saved event counts: %d", accountId, + events.size())); + } + }); } } } diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java index 508d4a35e5..334590a39f 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java @@ -1,20 +1,16 @@ package com.akto.suspect_data; -import com.akto.dto.threat_detection.SampleMaliciousRequest; -import com.fasterxml.jackson.databind.ObjectMapper; - -import java.util.Optional; +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; // Kafka Message Wrapper for suspect data public class Message { private String accountId; - private SampleMaliciousRequest data; - - private static final ObjectMapper objectMapper = new ObjectMapper(); + private MaliciousEvent data; - public Message() {} + public Message() { + } - public Message(String accountId, SampleMaliciousRequest data) { + public Message(String accountId, MaliciousEvent data) { this.accountId = accountId; this.data = data; } @@ -27,31 +23,11 @@ public void setAccountId(String accountId) { this.accountId = accountId; } - public SampleMaliciousRequest getData() { + public MaliciousEvent getData() { return data; } - public void setData(SampleMaliciousRequest data) { + public void setData(MaliciousEvent data) { this.data = data; } - - public static Optional marshall(Message m) { - try { - return Optional.of(objectMapper.writeValueAsString(m)); - } catch (Exception e) { - return Optional.empty(); - } - } - - public Optional marshall() { - return marshall(this); - } - - public static Optional unmarshall(String s) { - try { - return Optional.of(objectMapper.readValue(s, Message.class)); - } catch (Exception e) { - return Optional.empty(); - } - } } diff --git a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java index 016428da3e..84dda8e8ed 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -2,14 +2,12 @@ import java.util.*; -import com.akto.suspect_data.CleanUpTask; import com.akto.suspect_data.FlushMessagesTask; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.akto.DaoInit; import com.akto.dao.context.Context; import com.akto.dto.HttpResponseParams; import com.akto.log.LoggerMaker; @@ -17,8 +15,6 @@ import com.akto.metrics.AllMetrics; import com.akto.runtime.utils.Utils; import com.akto.traffic.KafkaRunner; -import com.mongodb.ConnectionString; - import io.lettuce.core.RedisClient; import com.akto.filters.HttpCallFilter; @@ -37,18 +33,9 @@ public class Main { private static final RedisClient redisClient = createRedisClient(); public static void main(String[] args) { - // We have a separate Mongo for storing threat detection data - // Metadata is stored in the main Mongo, which we call using API - // So we always need to enable hybrid mode for this module - String mongoURI = System.getenv("AKTO_THREAT_DETECTION_MONGO_CONN"); - DaoInit.init(new ConnectionString(mongoURI)); - // Flush Messages task FlushMessagesTask.instance.init(); - // Clean up sample requests for which no alert is generated - CleanUpTask.instance.init(); - String topicName = System.getenv("AKTO_KAFKA_TOPIC_NAME"); if (topicName == null) { String defaultTopic = "akto.api.protection"; @@ -107,8 +94,7 @@ public static void processRecords(ConsumerRecords records) { Context.accountId.set(accountIdInt); if (!httpCallFilterMap.containsKey(accountId)) { - HttpCallFilter filter = - new HttpCallFilter(redisClient, sync_threshold_count, sync_threshold_time); + HttpCallFilter filter = new HttpCallFilter(redisClient, sync_threshold_count, sync_threshold_time); httpCallFilterMap.put(accountId, filter); loggerMaker.infoAndAddToDb("New filter created for account: " + accountId); } @@ -123,12 +109,10 @@ public static void processRecords(ConsumerRecords records) { public static RedisClient createRedisClient() { String host = System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_HOST", "localhost"); - int port = - Integer.parseInt( - System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_PORT", "6379")); - int database = - Integer.parseInt( - System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_DB", "0")); + int port = Integer.parseInt( + System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_PORT", "6379")); + int database = Integer.parseInt( + System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_DB", "0")); return RedisClient.create("redis://" + host + ":" + port + "/" + database); } diff --git a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java index adc4e529bb..7b31a3f178 100644 --- a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java +++ b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java @@ -6,8 +6,8 @@ import com.akto.dto.HttpRequestParams; import com.akto.dto.HttpResponseParams; import com.akto.dto.monitoring.FilterConfig; -import com.akto.dto.threat_detection.SampleMaliciousRequest; import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier.Result; +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; import java.util.concurrent.ConcurrentHashMap; @@ -75,9 +75,8 @@ private static HttpResponseParams generateResponseParamsForStatusCode(int status public void testShouldNotify() throws InterruptedException { MemCache cache = new MemCache(); - WindowBasedThresholdNotifier notifier = - new WindowBasedThresholdNotifier( - cache, new WindowBasedThresholdNotifier.Config(10, 1)); + WindowBasedThresholdNotifier notifier = new WindowBasedThresholdNotifier( + cache, new WindowBasedThresholdNotifier.Config(10, 1)); boolean shouldNotify = false; String ip = "192.168.0.1"; @@ -86,11 +85,19 @@ public void testShouldNotify() throws InterruptedException { filterConfig.setId("4XX_FILTER"); for (int i = 0; i < 1000; i++) { - Result res = - notifier.shouldNotify( - ip + "|" + "4XX_FILTER", - new SampleMaliciousRequest( - filterConfig, ip, generateResponseParamsForStatusCode(400))); + HttpResponseParams responseParams = generateResponseParamsForStatusCode(400); + Result res = notifier.shouldNotify( + ip + "|" + "4XX_FILTER", + MaliciousEvent + .newBuilder() + .setActorId(ip) + .setIp(ip) + .setTimestamp(responseParams.getTime()) + .setApiCollectionId(responseParams.getRequestParams().getApiCollectionId()) + .setMethod(responseParams.getRequestParams().getMethod()) + .setUrl(responseParams.getRequestParams().getURL()) + .setPayload(responseParams.getOrig()) + .build()); shouldNotify = shouldNotify || res.shouldNotify(); } diff --git a/apps/pom.xml b/apps/pom.xml index ca5ec582af..52ff06ff91 100644 --- a/apps/pom.xml +++ b/apps/pom.xml @@ -136,6 +136,17 @@ testing-cli + + threat-protection-backend + + + threat-protection-backend/pom.xml + + + + threat-protection-backend + + diff --git a/apps/threat-protection-backend/pom.xml b/apps/threat-protection-backend/pom.xml new file mode 100644 index 0000000000..d63032154d --- /dev/null +++ b/apps/threat-protection-backend/pom.xml @@ -0,0 +1,182 @@ + + + 4.0.0 + + + com.akto.apps + apps + ${revision} + + + com.akto.apps.threat-protection-backend + threat-protection-backend + jar + + + + org.apache.commons + commons-lang3 + 3.12.0 + + + com.akto.libs.dao + dao + ${project.version} + + + com.akto.libs.utils + utils + ${project.version} + + + org.apache.httpcomponents + httpclient + 4.5.13 + + + + org.jetbrains + annotations + RELEASE + compile + + + org.junit.jupiter + junit-jupiter-api + 5.4.2 + test + + + com.akto.libs.utils + utils + test-jar + ${project.version} + test + + + com.akto.libs.protobuf + protobuf + 1.0-SNAPSHOT + compile + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 8 + 8 + + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.0.1 + + + copy-dependencies + package + + copy-dependencies + + + + + + + src/main/java + src/test/java + + + src/main/resources + true + + **/version.txt + + + + + + + + normal + + true + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + package + + single + + + + + + com.akto.threat.detection.Main + + + + + jar-with-dependencies + + + + + + + + + + + devcontainer + + + + org.apache.maven.plugins + maven-jar-plugin + + api-threat-detection-1.0-SNAPSHOT-jar-with-dependencies + + + true + com.akto.threat.detection.Main + dependency-jars/ + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + package + + copy-dependencies + + + ${project.build.directory}/dependency-jars/ + + + + + + + + + + \ No newline at end of file diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java new file mode 100644 index 0000000000..4037f50e22 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java @@ -0,0 +1,60 @@ +package com.akto.threat.protection; + +import io.grpc.Server; +import io.grpc.ServerBuilder; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import com.akto.DaoInit; +import com.akto.threat.protection.db.DBService; +import com.mongodb.ConnectionString; +import com.mongodb.ReadPreference; +import com.mongodb.client.MongoClient; + +public class BackendServer { + private final int port; + private final Server server; + + public BackendServer(int port) { + this.port = port; + + MongoClient mongoClient = DaoInit.createMongoClient( + new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO")), + ReadPreference.secondary()); + + DBService dbService = new DBService(mongoClient); + + this.server = ServerBuilder.forPort(port).addService(new ConsumerMaliciousEventService(dbService)).build(); + } + + public void stop() throws InterruptedException { + if (server != null) { + server.shutdown().awaitTermination(30, TimeUnit.SECONDS); + } + } + + public void start() throws IOException { + server.start(); + System.out.println("Server started, listening on " + port); + Runtime.getRuntime() + .addShutdownHook( + new Thread( + () -> { + System.err.println( + "*** shutting down gRPC server since JVM is shutting down"); + try { + BackendServer.this.stop(); + } catch (InterruptedException e) { + e.printStackTrace(System.err); + } + System.err.println("*** server shut down"); + })); + } + + public void blockUntilShutdown() throws InterruptedException { + if (server != null) { + server.awaitTermination(); + } + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java new file mode 100644 index 0000000000..20d5879618 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java @@ -0,0 +1,37 @@ +package com.akto.threat.protection; + +import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; +import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest; +import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse; +import com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest; +import com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse; +import com.akto.threat.protection.db.DBService; +import io.grpc.stub.StreamObserver; + +public class ConsumerMaliciousEventService + extends ConsumerServiceGrpc.ConsumerServiceImplBase { + + private final DBService dbService; + + public ConsumerMaliciousEventService(DBService dbService) { + this.dbService = dbService; + } + + @Override + public void saveMaliciousEvent( + SaveMaliciousEventRequest request, + StreamObserver responseObserver) { + this.dbService.saveMaliciousEvents(request.getAccountId() + "", request.getEventsList()); + responseObserver.onNext(SaveMaliciousEventResponse.newBuilder().build()); + responseObserver.onCompleted(); + } + + @Override + public void saveSmartEvent( + SaveSmartEventRequest request, + StreamObserver responseObserver) { + this.dbService.saveSmartEvent(request.getAccountId() + "", request.getEvent()); + responseObserver.onNext(SaveSmartEventResponse.newBuilder().build()); + responseObserver.onCompleted(); + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java new file mode 100644 index 0000000000..0ea4abda66 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -0,0 +1,16 @@ +package com.akto.threat.protection; + +import com.akto.DaoInit; +import com.mongodb.ConnectionString; + +public class Main { + public static void main(String[] args) throws Exception { + String mongoURI = System.getenv("AKTO_THREAT_DETECTION_MONGO_CONN"); + DaoInit.init(new ConnectionString(mongoURI)); + + int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); + BackendServer server = new BackendServer(port); + server.start(); + server.blockUntilShutdown(); + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/DBService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/DBService.java new file mode 100644 index 0000000000..3653882d2a --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/DBService.java @@ -0,0 +1,54 @@ +package com.akto.threat.protection.db; + +import java.util.ArrayList; +import java.util.List; + +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; +import com.akto.proto.threat_protection.consumer_service.v1.SmartEvent; +import com.mongodb.client.MongoClient; +import com.mongodb.client.model.BulkWriteOptions; +import com.mongodb.client.model.InsertOneModel; +import com.mongodb.client.model.WriteModel; + +public class DBService { + + // We are doing this way instead of using DaoInit.init since we are using + // separate mongo for saving events. + // Current Dao approach doesnt work well with 2 separate mongo connections. + private final MongoClient mongoClient; + + public DBService(MongoClient client) { + this.mongoClient = client; + } + + public void saveMaliciousEvents(String db, Iterable events) { + List> bulkUpdates = new ArrayList<>(); + events.forEach(event -> { + bulkUpdates.add( + new InsertOneModel<>( + new MaliciousEventModel( + event.getFilterId(), + event.getActorId(), + event.getIp(), + event.getUrl(), + event.getMethod(), + event.getPayload(), + event.getTimestamp()))); + }); + this.mongoClient + .getDatabase(db) + .getCollection("malicious_events", MaliciousEventModel.class) + .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); + } + + public void saveSmartEvent(String db, SmartEvent events) { + this.mongoClient + .getDatabase(db) + .getCollection("smart_events", SmartEventModel.class) + .insertOne( + new SmartEventModel( + events.getFilterId(), + events.getActorId(), + events.getDetectedAt())); + } +} diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java similarity index 59% rename from libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java rename to apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java index a88e6e891d..60d319d27d 100644 --- a/libs/dao/src/main/java/com/akto/dto/threat_detection/SampleMaliciousRequest.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java @@ -1,45 +1,47 @@ -package com.akto.dto.threat_detection; - -import com.akto.dto.HttpResponseParams; -import com.akto.dto.monitoring.FilterConfig; +package com.akto.threat.protection.db; import com.akto.dto.type.URLMethods.Method; import java.util.UUID; -public class SampleMaliciousRequest { +public class MaliciousEventModel { private String id; private String filterId; private String actor; + private String ip; private String url; private Method method; private String data; private int binId; private int expiry; - public SampleMaliciousRequest() {} + public MaliciousEventModel() { + } - public SampleMaliciousRequest( - FilterConfig filter, String actor, HttpResponseParams responseParams) { + public MaliciousEventModel( + String filterId, + String actor, + String ip, + String url, + String method, + String data, + long requestTime) { int now = (int) (System.currentTimeMillis() / 1000L); this.id = UUID.randomUUID().toString(); - this.filterId = filter.getId(); + this.ip = ip; + this.filterId = filterId; this.actor = actor; - this.data = responseParams.getOrig(); - this.binId = generateBinId(responseParams); - this.url = responseParams.getRequestParams().getURL(); - this.method = Method.fromString(responseParams.getRequestParams().getMethod()); + this.data = data; + this.binId = (int) requestTime / 60; + this.url = url; + this.method = Method.fromString(method); // For now, we are hardcoding it to 3 hrs. // But later we will read it through FilterConfig this.expiry = now + (3 * 60 * 60); } - public static int generateBinId(HttpResponseParams responseParam) { - return responseParam.getTime() / 60; - } - public String getId() { return id; } @@ -87,4 +89,28 @@ public void setBinId(int binId) { public void setExpiry(int expiry) { this.expiry = expiry; } + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public Method getMethod() { + return method; + } + + public void setMethod(Method method) { + this.method = method; + } + + public String getIp() { + return ip; + } + + public void setIp(String ip) { + this.ip = ip; + } } diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java similarity index 51% rename from libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java rename to apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java index 2a1bcbd9d3..d9748d5092 100644 --- a/libs/dao/src/main/java/com/akto/dto/threat_detection/DetectedThreatAlert.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java @@ -1,12 +1,9 @@ -package com.akto.dto.threat_detection; +package com.akto.threat.protection.db; -import java.util.List; -import java.util.Optional; import java.util.UUID; -import com.fasterxml.jackson.databind.ObjectMapper; +public class SmartEventModel { -public class DetectedThreatAlert { private String id; private String filterId; @@ -15,19 +12,14 @@ public class DetectedThreatAlert { private String actor; - private List bins; - - private final ObjectMapper objectMapper = new ObjectMapper(); - - public DetectedThreatAlert() { + public SmartEventModel() { } - public DetectedThreatAlert(String filterId, String actor, long detectedAt, List bins) { + public SmartEventModel(String filterId, String actor, long detectedAt) { this.id = UUID.randomUUID().toString(); this.filterId = filterId; this.detectedAt = detectedAt; this.actor = actor; - this.bins = bins; } public String getId() { @@ -62,20 +54,4 @@ public void setActor(String actor) { this.actor = actor; } - public List getBins() { - return bins; - } - - public void setBins(List bins) { - this.bins = bins; - } - - public Optional marshall() { - try { - return Optional.of(this.objectMapper.writeValueAsString(this)); - } catch (Exception e) { - e.printStackTrace(); - return Optional.empty(); - } - } } diff --git a/buf.gen.yaml b/buf.gen.yaml new file mode 100644 index 0000000000..ac36b4761c --- /dev/null +++ b/buf.gen.yaml @@ -0,0 +1,20 @@ +version: v2 + +clean: true + +managed: + enabled: true + + override: + - file_option: java_multiple_files + value: true + + - file_option: java_package_prefix + value: com.akto.proto + +plugins: + - remote: buf.build/grpc/java:v1.68.1 + out: libs/protobuf/src/main/java + + - remote: buf.build/protocolbuffers/java:v28.3 + out: libs/protobuf/src/main/java diff --git a/libs/dao/src/main/java/com/akto/DaoInit.java b/libs/dao/src/main/java/com/akto/DaoInit.java index 7e4ea5a2fe..04ced65abf 100644 --- a/libs/dao/src/main/java/com/akto/DaoInit.java +++ b/libs/dao/src/main/java/com/akto/DaoInit.java @@ -72,6 +72,8 @@ import com.mongodb.ConnectionString; import com.mongodb.MongoClientSettings; import com.mongodb.ReadPreference; +import com.mongodb.client.MongoClient; +import com.mongodb.WriteConcern; import com.mongodb.WriteConcern; import com.mongodb.client.MongoClients; @@ -88,327 +90,471 @@ public class DaoInit { - private static final Logger logger = LoggerFactory.getLogger(DaoInit.class); + private static final Logger logger = LoggerFactory.getLogger(DaoInit.class); - public static CodecRegistry createCodecRegistry(){ - ClassModel configClassModel = ClassModel.builder(Config.class).enableDiscriminator(true).build(); - ClassModel signupInfoClassModel = ClassModel.builder(SignupInfo.class).enableDiscriminator(true) - .build(); - ClassModel apiAuthClassModel = ClassModel.builder(APIAuth.class).enableDiscriminator(true).build(); - ClassModel attempResultModel = ClassModel.builder(AttemptResult.class).enableDiscriminator(true) - .build(); - ClassModel urlTemplateModel = ClassModel.builder(URLTemplate.class).enableDiscriminator(true) - .build(); - ClassModel pendingInviteCodeClassModel = ClassModel.builder(PendingInviteCode.class) - .enableDiscriminator(true).build(); - ClassModel rbacClassModel = ClassModel.builder(RBAC.class).enableDiscriminator(true).build(); - ClassModel singleTypeInfoClassModel = ClassModel.builder(SingleTypeInfo.class) - .enableDiscriminator(true).build(); - ClassModel kafkaHealthMetricClassModel = ClassModel.builder(KafkaHealthMetric.class) - .enableDiscriminator(true).build(); - ClassModel thirdPartyAccessClassModel = ClassModel.builder(ThirdPartyAccess.class) - .enableDiscriminator(true).build(); - ClassModel credentialClassModel = ClassModel.builder(Credential.class).enableDiscriminator(true) - .build(); - ClassModel apiTokenClassModel = ClassModel.builder(ApiToken.class).enableDiscriminator(true).build(); - ClassModel apiInfoClassModel = ClassModel.builder(ApiInfo.class).enableDiscriminator(true).build(); - ClassModel apiInfoKeyClassModel = ClassModel.builder(ApiInfo.ApiInfoKey.class) - .enableDiscriminator(true).build(); - ClassModel customFilterClassModel = ClassModel.builder(CustomFilter.class) - .enableDiscriminator(true).build(); - ClassModel fieldExistsFilterClassModel = ClassModel.builder(FieldExistsFilter.class) - .enableDiscriminator(true).build(); - ClassModel responseCodeRuntimeFilterClassModel = ClassModel - .builder(ResponseCodeRuntimeFilter.class).enableDiscriminator(true).build(); - ; - ClassModel runtimeFilterClassModel = ClassModel.builder(RuntimeFilter.class) - .enableDiscriminator(true).build(); - ClassModel filterSampleDataClassModel = ClassModel.builder(FilterSampleData.class) - .enableDiscriminator(true).build(); - ClassModel accountSettingsClassModel = ClassModel.builder(AccountSettings.class) - .enableDiscriminator(true).build(); - ClassModel predicateClassModel = ClassModel.builder(Predicate.class).enableDiscriminator(true) - .build(); - ClassModel regexPredicateClassModel = ClassModel.builder(RegexPredicate.class) - .enableDiscriminator(true).build(); - ClassModel startsWithPredicateClassModel = ClassModel.builder(StartsWithPredicate.class) - .enableDiscriminator(true).build(); - ClassModel endsWithPredicateClassModel = ClassModel.builder(EndsWithPredicate.class) - .enableDiscriminator(true).build(); - ClassModel equalsToPredicateClassModel = ClassModel.builder(EqualsToPredicate.class) - .enableDiscriminator(true).build(); - ClassModel isNumberPredicateClassModel = ClassModel.builder(IsNumberPredicate.class) - .enableDiscriminator(true).build(); - ClassModel conditionsClassModel = ClassModel.builder(Conditions.class).enableDiscriminator(true) - .build(); - ClassModel cappedListClassModel = ClassModel.builder(CappedList.class).enableDiscriminator(true) - .build(); - ClassModel testingRunClassModel = ClassModel.builder(TestingRun.class).enableDiscriminator(true) - .build(); - ClassModel testingRunResultClassModel = ClassModel.builder(TestingRunResult.class) - .enableDiscriminator(true).build(); - ClassModel testResultClassModel = ClassModel.builder(TestResult.class).enableDiscriminator(true) - .build(); - ClassModel multiExecTestResultClassModel = ClassModel.builder(MultiExecTestResult.class).enableDiscriminator(true) - .build(); - ClassModel genericTestResultClassModel = ClassModel.builder(GenericTestResult.class).enableDiscriminator(true) - .build(); - ClassModel authMechanismClassModel = ClassModel.builder(AuthMechanism.class) - .enableDiscriminator(true).build(); - ClassModel setupClassModel = ClassModel.builder(Setup.class) - .enableDiscriminator(true).build(); - ClassModel authParamClassModel = ClassModel.builder(AuthParam.class).enableDiscriminator(true) - .build(); - ClassModel hardcodedAuthParamClassModel = ClassModel.builder(HardcodedAuthParam.class) - .enableDiscriminator(true).build(); - ClassModel loginReqAuthParamClassModel = ClassModel.builder(LoginRequestAuthParam.class) - .enableDiscriminator(true).build(); - ClassModel testingEndpointsClassModel = ClassModel.builder(TestingEndpoints.class) - .enableDiscriminator(true).build(); - ClassModel customTestingEndpointsClassModel = ClassModel - .builder(CustomTestingEndpoints.class).enableDiscriminator(true).build(); - ClassModel allTestingEndpointsClassModel = ClassModel - .builder(AllTestingEndpoints.class).enableDiscriminator(true).build(); - ClassModel collectionWiseTestingEndpointsClassModel = ClassModel - .builder(CollectionWiseTestingEndpoints.class).enableDiscriminator(true).build(); - ClassModel workflowTestingEndpointsClassModel = ClassModel - .builder(WorkflowTestingEndpoints.class).enableDiscriminator(true).build(); - ClassModel workflowTestResultClassModel = ClassModel.builder(WorkflowTestResult.class) - .enableDiscriminator(true).build(); - ClassModel workflowTestClassModel = ClassModel.builder(WorkflowTest.class) - .enableDiscriminator(true).build(); - ClassModel cappedSetClassModel = ClassModel.builder(CappedSet.class).enableDiscriminator(true) - .build(); - ClassModel CustomWebhookClassModel = ClassModel.builder(CustomWebhook.class) - .enableDiscriminator(true).build(); - ClassModel WorkflowNodeDetailsClassModel = ClassModel.builder(WorkflowNodeDetails.class) - .enableDiscriminator(true).build(); - ClassModel CustomWebhookResultClassModel = ClassModel.builder(CustomWebhookResult.class) - .enableDiscriminator(true).build(); - ClassModel nodeResultClassModel = ClassModel - .builder(WorkflowTestResult.NodeResult.class).enableDiscriminator(true).build(); - ClassModel testingRunIssuesClassModel = ClassModel - .builder(TestingRunIssues.class).enableDiscriminator(true).build(); - ClassModel testingIssuesIdClassModel = ClassModel - .builder(TestingIssuesId.class).enableDiscriminator(true).build(); - ClassModel testSourceConfigClassModel = ClassModel - .builder(TestSourceConfig.class).enableDiscriminator(true).build(); - ClassModel endpointLogicalGroupClassModel = ClassModel - .builder(EndpointLogicalGroup.class).enableDiscriminator(true).build(); - ClassModel testRolesClassModel = ClassModel - .builder(TestRoles.class).enableDiscriminator(true).build(); - ClassModel logicalGroupTestingEndpointClassModel = ClassModel - .builder(LogicalGroupTestingEndpoint.class).enableDiscriminator(true).build(); - ClassModel customAuthTypeModel = ClassModel - .builder(CustomAuthType.class).enableDiscriminator(true).build(); - ClassModel containsPredicateClassModel = ClassModel - .builder(ContainsPredicate.class).enableDiscriminator(true).build(); - ClassModel notBelongsToPredicateClassModel = ClassModel - .builder(NotBelongsToPredicate.class).enableDiscriminator(true).build(); - ClassModel belongsToPredicateClassModel = ClassModel - .builder(BelongsToPredicate.class).enableDiscriminator(true).build(); - ClassModel yamlNodeDetails = ClassModel - .builder(YamlNodeDetails.class).enableDiscriminator(true).build(); - ClassModel unauthenticatedEndpointsClassModel = ClassModel - .builder(UnauthenticatedEndpoint.class).enableDiscriminator(true).build(); - ClassModel eventsExampleClassModel = ClassModel - .builder(EventsExample.class).enableDiscriminator(true).build(); - // ClassModel awsResourceModel = - // ClassModel.builder(AwsResource.class).enableDiscriminator(true) - // .build(); - ClassModel awsResourcesModel = ClassModel.builder(AwsResources.class).enableDiscriminator(true) - .build(); - ClassModel AktoDataTypeClassModel = ClassModel.builder(AktoDataType.class).enableDiscriminator(true).build(); - ClassModel testInfoClassModel = ClassModel.builder(TestInfo.class).enableDiscriminator(true).build(); - ClassModel bflaTestInfoClassModel = ClassModel.builder(BFLATestInfo.class).enableDiscriminator(true).build(); - ClassModel accessMatrixUrlToRoleClassModel = ClassModel.builder(AccessMatrixUrlToRole.class).enableDiscriminator(true).build(); - ClassModel accessMatrixTaskInfoClassModel = ClassModel.builder(AccessMatrixTaskInfo.class).enableDiscriminator(true).build(); - ClassModel loaderClassModel = ClassModel.builder(Loader.class).enableDiscriminator(true).build(); - ClassModel normalLoaderClassModel = ClassModel.builder(NormalLoader.class).enableDiscriminator(true).build(); - ClassModel postmanUploadLoaderClassModel = ClassModel.builder(PostmanUploadLoader.class).enableDiscriminator(true).build(); - ClassModel aktoGptConfigClassModel = ClassModel.builder(AktoGptConfig.class).enableDiscriminator(true).build(); + public static CodecRegistry createCodecRegistry() { + ClassModel configClassModel = + ClassModel.builder(Config.class).enableDiscriminator(true).build(); + ClassModel signupInfoClassModel = + ClassModel.builder(SignupInfo.class).enableDiscriminator(true).build(); + ClassModel apiAuthClassModel = + ClassModel.builder(APIAuth.class).enableDiscriminator(true).build(); + ClassModel attempResultModel = + ClassModel.builder(AttemptResult.class).enableDiscriminator(true).build(); + ClassModel urlTemplateModel = + ClassModel.builder(URLTemplate.class).enableDiscriminator(true).build(); + ClassModel pendingInviteCodeClassModel = + ClassModel.builder(PendingInviteCode.class).enableDiscriminator(true).build(); + ClassModel rbacClassModel = + ClassModel.builder(RBAC.class).enableDiscriminator(true).build(); + ClassModel singleTypeInfoClassModel = + ClassModel.builder(SingleTypeInfo.class).enableDiscriminator(true).build(); + ClassModel kafkaHealthMetricClassModel = + ClassModel.builder(KafkaHealthMetric.class).enableDiscriminator(true).build(); + ClassModel thirdPartyAccessClassModel = + ClassModel.builder(ThirdPartyAccess.class).enableDiscriminator(true).build(); + ClassModel credentialClassModel = + ClassModel.builder(Credential.class).enableDiscriminator(true).build(); + ClassModel apiTokenClassModel = + ClassModel.builder(ApiToken.class).enableDiscriminator(true).build(); + ClassModel apiInfoClassModel = + ClassModel.builder(ApiInfo.class).enableDiscriminator(true).build(); + ClassModel apiInfoKeyClassModel = + ClassModel.builder(ApiInfo.ApiInfoKey.class).enableDiscriminator(true).build(); + ClassModel customFilterClassModel = + ClassModel.builder(CustomFilter.class).enableDiscriminator(true).build(); + ClassModel fieldExistsFilterClassModel = + ClassModel.builder(FieldExistsFilter.class).enableDiscriminator(true).build(); + ClassModel responseCodeRuntimeFilterClassModel = + ClassModel.builder(ResponseCodeRuntimeFilter.class).enableDiscriminator(true).build(); + ; + ClassModel runtimeFilterClassModel = + ClassModel.builder(RuntimeFilter.class).enableDiscriminator(true).build(); + ClassModel filterSampleDataClassModel = + ClassModel.builder(FilterSampleData.class).enableDiscriminator(true).build(); + ClassModel accountSettingsClassModel = + ClassModel.builder(AccountSettings.class).enableDiscriminator(true).build(); + ClassModel predicateClassModel = + ClassModel.builder(Predicate.class).enableDiscriminator(true).build(); + ClassModel regexPredicateClassModel = + ClassModel.builder(RegexPredicate.class).enableDiscriminator(true).build(); + ClassModel startsWithPredicateClassModel = + ClassModel.builder(StartsWithPredicate.class).enableDiscriminator(true).build(); + ClassModel endsWithPredicateClassModel = + ClassModel.builder(EndsWithPredicate.class).enableDiscriminator(true).build(); + ClassModel equalsToPredicateClassModel = + ClassModel.builder(EqualsToPredicate.class).enableDiscriminator(true).build(); + ClassModel isNumberPredicateClassModel = + ClassModel.builder(IsNumberPredicate.class).enableDiscriminator(true).build(); + ClassModel conditionsClassModel = + ClassModel.builder(Conditions.class).enableDiscriminator(true).build(); + ClassModel cappedListClassModel = + ClassModel.builder(CappedList.class).enableDiscriminator(true).build(); + ClassModel testingRunClassModel = + ClassModel.builder(TestingRun.class).enableDiscriminator(true).build(); + ClassModel testingRunResultClassModel = + ClassModel.builder(TestingRunResult.class).enableDiscriminator(true).build(); + ClassModel testResultClassModel = + ClassModel.builder(TestResult.class).enableDiscriminator(true).build(); + ClassModel multiExecTestResultClassModel = + ClassModel.builder(MultiExecTestResult.class).enableDiscriminator(true).build(); + ClassModel genericTestResultClassModel = + ClassModel.builder(GenericTestResult.class).enableDiscriminator(true).build(); + ClassModel authMechanismClassModel = + ClassModel.builder(AuthMechanism.class).enableDiscriminator(true).build(); + ClassModel setupClassModel = + ClassModel.builder(Setup.class).enableDiscriminator(true).build(); + ClassModel authParamClassModel = + ClassModel.builder(AuthParam.class).enableDiscriminator(true).build(); + ClassModel hardcodedAuthParamClassModel = + ClassModel.builder(HardcodedAuthParam.class).enableDiscriminator(true).build(); + ClassModel loginReqAuthParamClassModel = + ClassModel.builder(LoginRequestAuthParam.class).enableDiscriminator(true).build(); + ClassModel testingEndpointsClassModel = + ClassModel.builder(TestingEndpoints.class).enableDiscriminator(true).build(); + ClassModel customTestingEndpointsClassModel = + ClassModel.builder(CustomTestingEndpoints.class).enableDiscriminator(true).build(); + ClassModel allTestingEndpointsClassModel = + ClassModel.builder(AllTestingEndpoints.class).enableDiscriminator(true).build(); + ClassModel collectionWiseTestingEndpointsClassModel = + ClassModel.builder(CollectionWiseTestingEndpoints.class).enableDiscriminator(true).build(); + ClassModel workflowTestingEndpointsClassModel = + ClassModel.builder(WorkflowTestingEndpoints.class).enableDiscriminator(true).build(); + ClassModel workflowTestResultClassModel = + ClassModel.builder(WorkflowTestResult.class).enableDiscriminator(true).build(); + ClassModel workflowTestClassModel = + ClassModel.builder(WorkflowTest.class).enableDiscriminator(true).build(); + ClassModel cappedSetClassModel = + ClassModel.builder(CappedSet.class).enableDiscriminator(true).build(); + ClassModel CustomWebhookClassModel = + ClassModel.builder(CustomWebhook.class).enableDiscriminator(true).build(); + ClassModel WorkflowNodeDetailsClassModel = + ClassModel.builder(WorkflowNodeDetails.class).enableDiscriminator(true).build(); + ClassModel CustomWebhookResultClassModel = + ClassModel.builder(CustomWebhookResult.class).enableDiscriminator(true).build(); + ClassModel nodeResultClassModel = + ClassModel.builder(WorkflowTestResult.NodeResult.class).enableDiscriminator(true).build(); + ClassModel testingRunIssuesClassModel = + ClassModel.builder(TestingRunIssues.class).enableDiscriminator(true).build(); + ClassModel testingIssuesIdClassModel = + ClassModel.builder(TestingIssuesId.class).enableDiscriminator(true).build(); + ClassModel testSourceConfigClassModel = + ClassModel.builder(TestSourceConfig.class).enableDiscriminator(true).build(); + ClassModel endpointLogicalGroupClassModel = + ClassModel.builder(EndpointLogicalGroup.class).enableDiscriminator(true).build(); + ClassModel testRolesClassModel = + ClassModel.builder(TestRoles.class).enableDiscriminator(true).build(); + ClassModel logicalGroupTestingEndpointClassModel = + ClassModel.builder(LogicalGroupTestingEndpoint.class).enableDiscriminator(true).build(); + ClassModel customAuthTypeModel = + ClassModel.builder(CustomAuthType.class).enableDiscriminator(true).build(); + ClassModel containsPredicateClassModel = + ClassModel.builder(ContainsPredicate.class).enableDiscriminator(true).build(); + ClassModel notBelongsToPredicateClassModel = + ClassModel.builder(NotBelongsToPredicate.class).enableDiscriminator(true).build(); + ClassModel belongsToPredicateClassModel = + ClassModel.builder(BelongsToPredicate.class).enableDiscriminator(true).build(); + ClassModel yamlNodeDetails = + ClassModel.builder(YamlNodeDetails.class).enableDiscriminator(true).build(); + ClassModel unauthenticatedEndpointsClassModel = + ClassModel.builder(UnauthenticatedEndpoint.class).enableDiscriminator(true).build(); + ClassModel eventsExampleClassModel = + ClassModel.builder(EventsExample.class).enableDiscriminator(true).build(); + // ClassModel awsResourceModel = + // ClassModel.builder(AwsResource.class).enableDiscriminator(true) + // .build(); + ClassModel awsResourcesModel = + ClassModel.builder(AwsResources.class).enableDiscriminator(true).build(); + ClassModel AktoDataTypeClassModel = + ClassModel.builder(AktoDataType.class).enableDiscriminator(true).build(); + ClassModel testInfoClassModel = + ClassModel.builder(TestInfo.class).enableDiscriminator(true).build(); + ClassModel bflaTestInfoClassModel = + ClassModel.builder(BFLATestInfo.class).enableDiscriminator(true).build(); + ClassModel accessMatrixUrlToRoleClassModel = + ClassModel.builder(AccessMatrixUrlToRole.class).enableDiscriminator(true).build(); + ClassModel accessMatrixTaskInfoClassModel = + ClassModel.builder(AccessMatrixTaskInfo.class).enableDiscriminator(true).build(); + ClassModel loaderClassModel = + ClassModel.builder(Loader.class).enableDiscriminator(true).build(); + ClassModel normalLoaderClassModel = + ClassModel.builder(NormalLoader.class).enableDiscriminator(true).build(); + ClassModel postmanUploadLoaderClassModel = + ClassModel.builder(PostmanUploadLoader.class).enableDiscriminator(true).build(); + ClassModel aktoGptConfigClassModel = + ClassModel.builder(AktoGptConfig.class).enableDiscriminator(true).build(); - ClassModel loginFlowStepsData = ClassModel.builder(LoginFlowStepsData.class) - .enableDiscriminator(true).build(); - ClassModel vulnerableRequestForTemplateClassModel = ClassModel.builder(VulnerableRequestForTemplate.class).enableDiscriminator(true).build(); - ClassModel trafficMetricsAlertClassModel = ClassModel.builder(TrafficMetricsAlert.class).enableDiscriminator(true).build(); - ClassModel jiraintegrationClassModel = ClassModel.builder(JiraIntegration.class).enableDiscriminator(true).build(); - ClassModel methodConditionClassModel = ClassModel.builder(MethodCondition.class).enableDiscriminator(true).build(); - ClassModel regexTestingEndpointsClassModel = ClassModel.builder(RegexTestingEndpoints.class).enableDiscriminator(true).build(); - ClassModel hostRegexTestingEndpointsClassModel = ClassModel.builder(HostRegexTestingEndpoints.class).enableDiscriminator(true).build(); - ClassModel dependencyNodeClassModel = ClassModel.builder(DependencyNode.class).enableDiscriminator(true).build(); - ClassModel paramInfoClassModel = ClassModel.builder(ParamInfo.class).enableDiscriminator(true).build(); - ClassModel nodeClassModel = ClassModel.builder(Node.class).enableDiscriminator(true).build(); - ClassModel connectionClassModel = ClassModel.builder(Connection.class).enableDiscriminator(true).build(); - ClassModel edgeClassModel = ClassModel.builder(Edge.class).enableDiscriminator(true).build(); - ClassModel cronTimersClassModel = ClassModel.builder(LastCronRunInfo.class) - .enableDiscriminator(true).build(); - ClassModel connectionInfoClassModel = ClassModel.builder(ConnectionInfo.class) - .enableDiscriminator(true).build(); - ClassModel testLibraryClassModel = ClassModel.builder(TestLibrary.class).enableDiscriminator(true).build(); + ClassModel loginFlowStepsData = + ClassModel.builder(LoginFlowStepsData.class).enableDiscriminator(true).build(); + ClassModel vulnerableRequestForTemplateClassModel = + ClassModel.builder(VulnerableRequestForTemplate.class).enableDiscriminator(true).build(); + ClassModel trafficMetricsAlertClassModel = + ClassModel.builder(TrafficMetricsAlert.class).enableDiscriminator(true).build(); + ClassModel jiraintegrationClassModel = + ClassModel.builder(JiraIntegration.class).enableDiscriminator(true).build(); + ClassModel methodConditionClassModel = + ClassModel.builder(MethodCondition.class).enableDiscriminator(true).build(); + ClassModel regexTestingEndpointsClassModel = + ClassModel.builder(RegexTestingEndpoints.class).enableDiscriminator(true).build(); + ClassModel hostRegexTestingEndpointsClassModel = + ClassModel.builder(HostRegexTestingEndpoints.class).enableDiscriminator(true).build(); + ClassModel dependencyNodeClassModel = + ClassModel.builder(DependencyNode.class).enableDiscriminator(true).build(); + ClassModel paramInfoClassModel = + ClassModel.builder(ParamInfo.class).enableDiscriminator(true).build(); + ClassModel nodeClassModel = + ClassModel.builder(Node.class).enableDiscriminator(true).build(); + ClassModel connectionClassModel = + ClassModel.builder(Connection.class).enableDiscriminator(true).build(); + ClassModel edgeClassModel = + ClassModel.builder(Edge.class).enableDiscriminator(true).build(); + ClassModel cronTimersClassModel = + ClassModel.builder(LastCronRunInfo.class).enableDiscriminator(true).build(); + ClassModel connectionInfoClassModel = + ClassModel.builder(ConnectionInfo.class).enableDiscriminator(true).build(); + ClassModel testLibraryClassModel = + ClassModel.builder(TestLibrary.class).enableDiscriminator(true).build(); - ClassModel UsageMetricClassModel = ClassModel.builder(UsageMetric.class).enableDiscriminator(true).build(); - ClassModel UsageMetricInfoClassModel = ClassModel.builder(UsageMetricInfo.class).enableDiscriminator(true).build(); - ClassModel UsageSyncClassModel = ClassModel.builder(UsageSync.class).enableDiscriminator(true).build(); - ClassModel OrganizationClassModel = ClassModel.builder(Organization.class).enableDiscriminator(true).build(); - ClassModel replaceDetailClassModel = ClassModel.builder(ReplaceDetail.class).enableDiscriminator(true).build(); - ClassModel modifyHostDetailClassModel = ClassModel.builder(ModifyHostDetail.class).enableDiscriminator(true).build(); - ClassModel fileUploadClassModel = ClassModel.builder(FileUpload.class).enableDiscriminator(true).build(); - ClassModel fileUploadLogClassModel = ClassModel.builder(FileUploadLog.class).enableDiscriminator(true).build(); - ClassModel codeAnalysisCollectionClassModel = ClassModel.builder(CodeAnalysisCollection.class).enableDiscriminator(true).build(); - ClassModel codeAnalysisApiLocationClassModel = ClassModel.builder(CodeAnalysisApiLocation.class).enableDiscriminator(true).build(); - ClassModel codeAnalysisApiInfoClassModel = ClassModel.builder(CodeAnalysisApiInfo.class).enableDiscriminator(true).build(); - ClassModel codeAnalysisApiInfoKeyClassModel = ClassModel.builder(CodeAnalysisApiInfo.CodeAnalysisApiInfoKey.class).enableDiscriminator(true).build(); - ClassModel riskScoreTestingEndpointsClassModel = ClassModel.builder(RiskScoreTestingEndpoints.class).enableDiscriminator(true).build(); - ClassModel OrganizationFlagsClassModel = ClassModel.builder(OrganizationFlags.class).enableDiscriminator(true).build(); - ClassModel sensitiveDataEndpointsClassModel = ClassModel.builder(SensitiveDataEndpoints.class).enableDiscriminator(true).build(); - ClassModel allApisGroupClassModel = ClassModel.builder(AllAPIsGroup.class).enableDiscriminator(true).build(); + ClassModel UsageMetricClassModel = + ClassModel.builder(UsageMetric.class).enableDiscriminator(true).build(); + ClassModel UsageMetricInfoClassModel = + ClassModel.builder(UsageMetricInfo.class).enableDiscriminator(true).build(); + ClassModel UsageSyncClassModel = + ClassModel.builder(UsageSync.class).enableDiscriminator(true).build(); + ClassModel OrganizationClassModel = + ClassModel.builder(Organization.class).enableDiscriminator(true).build(); + ClassModel replaceDetailClassModel = + ClassModel.builder(ReplaceDetail.class).enableDiscriminator(true).build(); + ClassModel modifyHostDetailClassModel = + ClassModel.builder(ModifyHostDetail.class).enableDiscriminator(true).build(); + ClassModel fileUploadClassModel = + ClassModel.builder(FileUpload.class).enableDiscriminator(true).build(); + ClassModel fileUploadLogClassModel = + ClassModel.builder(FileUploadLog.class).enableDiscriminator(true).build(); + ClassModel codeAnalysisCollectionClassModel = + ClassModel.builder(CodeAnalysisCollection.class).enableDiscriminator(true).build(); + ClassModel codeAnalysisApiLocationClassModel = + ClassModel.builder(CodeAnalysisApiLocation.class).enableDiscriminator(true).build(); + ClassModel codeAnalysisApiInfoClassModel = + ClassModel.builder(CodeAnalysisApiInfo.class).enableDiscriminator(true).build(); + ClassModel codeAnalysisApiInfoKeyClassModel = + ClassModel.builder(CodeAnalysisApiInfo.CodeAnalysisApiInfoKey.class) + .enableDiscriminator(true) + .build(); + ClassModel riskScoreTestingEndpointsClassModel = + ClassModel.builder(RiskScoreTestingEndpoints.class).enableDiscriminator(true).build(); + ClassModel OrganizationFlagsClassModel = + ClassModel.builder(OrganizationFlags.class).enableDiscriminator(true).build(); + ClassModel sensitiveDataEndpointsClassModel = + ClassModel.builder(SensitiveDataEndpoints.class).enableDiscriminator(true).build(); + ClassModel allApisGroupClassModel = + ClassModel.builder(AllAPIsGroup.class).enableDiscriminator(true).build(); - ClassModel RuntimeMetricsClassModel = ClassModel.builder(RuntimeMetrics.class).enableDiscriminator(true).build(); - ClassModel codeAnalysisApiModel = ClassModel.builder(CodeAnalysisApi.class).enableDiscriminator(true).build(); - ClassModel codeAnalysisRepoModel = ClassModel.builder(CodeAnalysisRepo.class).enableDiscriminator(true).build(); - ClassModel historicalDataClassModel = ClassModel.builder(HistoricalData.class).enableDiscriminator(true).build(); - ClassModel configSettingClassModel = ClassModel.builder(TestConfigsAdvancedSettings.class).enableDiscriminator(true).build(); - ClassModel configSettingsConditionTypeClassModel = ClassModel.builder(ConditionsType.class).enableDiscriminator(true).build(); - CodecRegistry pojoCodecRegistry = fromProviders(PojoCodecProvider.builder().register( - configClassModel, signupInfoClassModel, apiAuthClassModel, attempResultModel, urlTemplateModel, - pendingInviteCodeClassModel, rbacClassModel, kafkaHealthMetricClassModel, singleTypeInfoClassModel, - thirdPartyAccessClassModel, credentialClassModel, apiTokenClassModel, apiInfoClassModel, - apiInfoKeyClassModel, customFilterClassModel, runtimeFilterClassModel, filterSampleDataClassModel, - predicateClassModel, conditionsClassModel, regexPredicateClassModel, startsWithPredicateClassModel, - endsWithPredicateClassModel, - fieldExistsFilterClassModel, accountSettingsClassModel, responseCodeRuntimeFilterClassModel, - cappedListClassModel, - equalsToPredicateClassModel, isNumberPredicateClassModel, testingRunClassModel, - testingRunResultClassModel, testResultClassModel, genericTestResultClassModel, - authMechanismClassModel, authParamClassModel, hardcodedAuthParamClassModel, loginReqAuthParamClassModel, - testingEndpointsClassModel, customTestingEndpointsClassModel, collectionWiseTestingEndpointsClassModel, - workflowTestingEndpointsClassModel, workflowTestResultClassModel, - cappedSetClassModel, CustomWebhookClassModel, WorkflowNodeDetailsClassModel, CustomWebhookResultClassModel, - nodeResultClassModel, awsResourcesModel, AktoDataTypeClassModel, testingRunIssuesClassModel, - testingIssuesIdClassModel, testSourceConfigClassModel, endpointLogicalGroupClassModel, testRolesClassModel, - logicalGroupTestingEndpointClassModel, testInfoClassModel, bflaTestInfoClassModel, customAuthTypeModel, - containsPredicateClassModel, notBelongsToPredicateClassModel, belongsToPredicateClassModel, loginFlowStepsData, - accessMatrixUrlToRoleClassModel, accessMatrixTaskInfoClassModel, - loaderClassModel, normalLoaderClassModel, postmanUploadLoaderClassModel, aktoGptConfigClassModel, - vulnerableRequestForTemplateClassModel, trafficMetricsAlertClassModel,jiraintegrationClassModel, setupClassModel, - cronTimersClassModel, connectionInfoClassModel, testLibraryClassModel, - methodConditionClassModel, regexTestingEndpointsClassModel, hostRegexTestingEndpointsClassModel, allTestingEndpointsClassModel, - UsageMetricClassModel, UsageMetricInfoClassModel, UsageSyncClassModel, OrganizationClassModel, - yamlNodeDetails, multiExecTestResultClassModel, workflowTestClassModel, dependencyNodeClassModel, paramInfoClassModel, - nodeClassModel, connectionClassModel, edgeClassModel, replaceDetailClassModel, modifyHostDetailClassModel, fileUploadClassModel - ,fileUploadLogClassModel, codeAnalysisCollectionClassModel, codeAnalysisApiLocationClassModel, codeAnalysisApiInfoClassModel, codeAnalysisApiInfoKeyClassModel, - riskScoreTestingEndpointsClassModel, OrganizationFlagsClassModel, sensitiveDataEndpointsClassModel, unauthenticatedEndpointsClassModel, allApisGroupClassModel, - eventsExampleClassModel, RuntimeMetricsClassModel, codeAnalysisRepoModel, codeAnalysisApiModel, historicalDataClassModel, configSettingClassModel, configSettingsConditionTypeClassModel).automatic(true).build()); + ClassModel RuntimeMetricsClassModel = + ClassModel.builder(RuntimeMetrics.class).enableDiscriminator(true).build(); + ClassModel codeAnalysisApiModel = + ClassModel.builder(CodeAnalysisApi.class).enableDiscriminator(true).build(); + ClassModel codeAnalysisRepoModel = + ClassModel.builder(CodeAnalysisRepo.class).enableDiscriminator(true).build(); + ClassModel historicalDataClassModel = + ClassModel.builder(HistoricalData.class).enableDiscriminator(true).build(); + ClassModel configSettingClassModel = + ClassModel.builder(TestConfigsAdvancedSettings.class).enableDiscriminator(true).build(); + ClassModel configSettingsConditionTypeClassModel = + ClassModel.builder(ConditionsType.class).enableDiscriminator(true).build(); + CodecRegistry pojoCodecRegistry = + fromProviders( + PojoCodecProvider.builder() + .register( + configClassModel, + signupInfoClassModel, + apiAuthClassModel, + attempResultModel, + urlTemplateModel, + pendingInviteCodeClassModel, + rbacClassModel, + kafkaHealthMetricClassModel, + singleTypeInfoClassModel, + thirdPartyAccessClassModel, + credentialClassModel, + apiTokenClassModel, + apiInfoClassModel, + apiInfoKeyClassModel, + customFilterClassModel, + runtimeFilterClassModel, + filterSampleDataClassModel, + predicateClassModel, + conditionsClassModel, + regexPredicateClassModel, + startsWithPredicateClassModel, + endsWithPredicateClassModel, + fieldExistsFilterClassModel, + accountSettingsClassModel, + responseCodeRuntimeFilterClassModel, + cappedListClassModel, + equalsToPredicateClassModel, + isNumberPredicateClassModel, + testingRunClassModel, + testingRunResultClassModel, + testResultClassModel, + genericTestResultClassModel, + authMechanismClassModel, + authParamClassModel, + hardcodedAuthParamClassModel, + loginReqAuthParamClassModel, + testingEndpointsClassModel, + customTestingEndpointsClassModel, + collectionWiseTestingEndpointsClassModel, + workflowTestingEndpointsClassModel, + workflowTestResultClassModel, + cappedSetClassModel, + CustomWebhookClassModel, + WorkflowNodeDetailsClassModel, + CustomWebhookResultClassModel, + nodeResultClassModel, + awsResourcesModel, + AktoDataTypeClassModel, + testingRunIssuesClassModel, + testingIssuesIdClassModel, + testSourceConfigClassModel, + endpointLogicalGroupClassModel, + testRolesClassModel, + logicalGroupTestingEndpointClassModel, + testInfoClassModel, + bflaTestInfoClassModel, + customAuthTypeModel, + containsPredicateClassModel, + notBelongsToPredicateClassModel, + belongsToPredicateClassModel, + loginFlowStepsData, + accessMatrixUrlToRoleClassModel, + accessMatrixTaskInfoClassModel, + loaderClassModel, + normalLoaderClassModel, + postmanUploadLoaderClassModel, + aktoGptConfigClassModel, + vulnerableRequestForTemplateClassModel, + trafficMetricsAlertClassModel, + jiraintegrationClassModel, + setupClassModel, + cronTimersClassModel, + connectionInfoClassModel, + testLibraryClassModel, + methodConditionClassModel, + regexTestingEndpointsClassModel, + hostRegexTestingEndpointsClassModel, + allTestingEndpointsClassModel, + UsageMetricClassModel, + UsageMetricInfoClassModel, + UsageSyncClassModel, + OrganizationClassModel, + yamlNodeDetails, + multiExecTestResultClassModel, + workflowTestClassModel, + dependencyNodeClassModel, + paramInfoClassModel, + nodeClassModel, + connectionClassModel, + edgeClassModel, + replaceDetailClassModel, + modifyHostDetailClassModel, + fileUploadClassModel, + fileUploadLogClassModel, + codeAnalysisCollectionClassModel, + codeAnalysisApiLocationClassModel, + codeAnalysisApiInfoClassModel, + codeAnalysisApiInfoKeyClassModel, + riskScoreTestingEndpointsClassModel, + OrganizationFlagsClassModel, + sensitiveDataEndpointsClassModel, + unauthenticatedEndpointsClassModel, + allApisGroupClassModel, + eventsExampleClassModel, + RuntimeMetricsClassModel, + codeAnalysisRepoModel, + codeAnalysisApiModel, + historicalDataClassModel, + configSettingClassModel, + configSettingsConditionTypeClassModel) + .automatic(true) + .build()); - final CodecRegistry customEnumCodecs = CodecRegistries.fromCodecs( - new EnumCodec<>(Conditions.Operator.class), - new EnumCodec<>(SingleTypeInfo.SuperType.class), - new EnumCodec<>(Method.class), - new EnumCodec<>(RBAC.Role.class), - new EnumCodec<>(Credential.Type.class), - new EnumCodec<>(ApiToken.Utility.class), - new EnumCodec<>(ApiInfo.AuthType.class), - new EnumCodec<>(ApiInfo.ApiAccessType.class), - new EnumCodec<>(TestResult.TestError.class), - new EnumCodec<>(AuthParam.Location.class), - new EnumCodec<>(TestingEndpoints.Type.class), - new EnumCodec<>(TestingRun.State.class), - new EnumCodec<>(AccountSettings.SetupType.class), - new EnumCodec<>(WorkflowNodeDetails.Type.class), - new EnumCodec<>(SingleTypeInfo.Domain.class), - new EnumCodec<>(CustomWebhook.ActiveStatus.class), - new EnumCodec<>(TestResult.Confidence.class), - new EnumCodec<>(SingleTypeInfo.Position.class), - new EnumCodec<>(TestResult.Confidence.class), - new EnumCodec<>(GlobalEnums.TestRunIssueStatus.class), - new EnumCodec<>(GlobalEnums.TestErrorSource.class), - new EnumCodec<>(GlobalEnums.TestCategory.class), - new EnumCodec<>(GlobalEnums.IssueTags.class), - new EnumCodec<>(GlobalEnums.Severity.class), - new EnumCodec<>(TrafficMetrics.Name.class), - new EnumCodec<>(Loader.Type.class), - new EnumCodec<>(CustomWebhook.WebhookOptions.class), - new EnumCodec<>(GlobalEnums.YamlTemplateSource.class), - new EnumCodec<>(AktoGptConfigState.class), - new EnumCodec<>(CustomWebhook.WebhookOptions.class), - new EnumCodec<>(TestingEndpoints.Operator.class), - new EnumCodec<>(MetricTypes.class), - new EnumCodec<>(User.AktoUIMode.class), - new EnumCodec<>(TrafficMetricsAlert.FilterType.class), - new EnumCodec<>(KVPair.KVType.class), - new EnumCodec<>(ApiCollection.ENV_TYPE.class), - new EnumCodec<>(FileUpload.UploadType.class), - new EnumCodec<>(FileUpload.UploadStatus.class), - new EnumCodec<>(FileUploadLog.UploadLogStatus.class), - new EnumCodec<>(TestCollectionProperty.Id.class), - new EnumCodec<>(CustomAuthType.TypeOfToken.class), - new EnumCodec<>(TrafficAlerts.ALERT_TYPE.class), - new EnumCodec<>(ApiInfo.ApiType.class), - new EnumCodec<>(CodeAnalysisRepo.SourceCodeType.class) - ); + final CodecRegistry customEnumCodecs = + CodecRegistries.fromCodecs( + new EnumCodec<>(Conditions.Operator.class), + new EnumCodec<>(SingleTypeInfo.SuperType.class), + new EnumCodec<>(Method.class), + new EnumCodec<>(RBAC.Role.class), + new EnumCodec<>(Credential.Type.class), + new EnumCodec<>(ApiToken.Utility.class), + new EnumCodec<>(ApiInfo.AuthType.class), + new EnumCodec<>(ApiInfo.ApiAccessType.class), + new EnumCodec<>(TestResult.TestError.class), + new EnumCodec<>(AuthParam.Location.class), + new EnumCodec<>(TestingEndpoints.Type.class), + new EnumCodec<>(TestingRun.State.class), + new EnumCodec<>(AccountSettings.SetupType.class), + new EnumCodec<>(WorkflowNodeDetails.Type.class), + new EnumCodec<>(SingleTypeInfo.Domain.class), + new EnumCodec<>(CustomWebhook.ActiveStatus.class), + new EnumCodec<>(TestResult.Confidence.class), + new EnumCodec<>(SingleTypeInfo.Position.class), + new EnumCodec<>(TestResult.Confidence.class), + new EnumCodec<>(GlobalEnums.TestRunIssueStatus.class), + new EnumCodec<>(GlobalEnums.TestErrorSource.class), + new EnumCodec<>(GlobalEnums.TestCategory.class), + new EnumCodec<>(GlobalEnums.IssueTags.class), + new EnumCodec<>(GlobalEnums.Severity.class), + new EnumCodec<>(TrafficMetrics.Name.class), + new EnumCodec<>(Loader.Type.class), + new EnumCodec<>(CustomWebhook.WebhookOptions.class), + new EnumCodec<>(GlobalEnums.YamlTemplateSource.class), + new EnumCodec<>(AktoGptConfigState.class), + new EnumCodec<>(CustomWebhook.WebhookOptions.class), + new EnumCodec<>(TestingEndpoints.Operator.class), + new EnumCodec<>(MetricTypes.class), + new EnumCodec<>(User.AktoUIMode.class), + new EnumCodec<>(TrafficMetricsAlert.FilterType.class), + new EnumCodec<>(KVPair.KVType.class), + new EnumCodec<>(ApiCollection.ENV_TYPE.class), + new EnumCodec<>(FileUpload.UploadType.class), + new EnumCodec<>(FileUpload.UploadStatus.class), + new EnumCodec<>(FileUploadLog.UploadLogStatus.class), + new EnumCodec<>(TestCollectionProperty.Id.class), + new EnumCodec<>(CustomAuthType.TypeOfToken.class), + new EnumCodec<>(TrafficAlerts.ALERT_TYPE.class), + new EnumCodec<>(ApiInfo.ApiType.class), + new EnumCodec<>(CodeAnalysisRepo.SourceCodeType.class)); - return fromRegistries(MongoClientSettings.getDefaultCodecRegistry(), pojoCodecRegistry, - customEnumCodecs); - } + return fromRegistries( + MongoClientSettings.getDefaultCodecRegistry(), pojoCodecRegistry, customEnumCodecs); + } - public static void init(ConnectionString connectionString, ReadPreference readPreference, WriteConcern writeConcern) { - DbMode.refreshDbType(connectionString.getConnectionString()); - logger.info("DB type: {}", DbMode.dbType); - DbMode.refreshSetupType(connectionString); - logger.info("DB setup type: {}", DbMode.setupType); + public static MongoClient createMongoClient( + ConnectionString connectionString, ReadPreference readPreference, WriteConcern writeConcern) { + DbMode.refreshDbType(connectionString.getConnectionString()); + logger.info("DB type: {}", DbMode.dbType); + DbMode.refreshSetupType(connectionString); + logger.info("DB setup type: {}", DbMode.setupType); - CodecRegistry codecRegistry = createCodecRegistry(); + CodecRegistry codecRegistry = createCodecRegistry(); - MongoClientSettings clientSettings = MongoClientSettings.builder() - .readPreference(readPreference) - .writeConcern(writeConcern) - .applyConnectionString(connectionString) - .codecRegistry(codecRegistry) - .build(); + MongoClientSettings clientSettings = + MongoClientSettings.builder() + .readPreference(readPreference) + .writeConcern(writeConcern) + .applyConnectionString(connectionString) + .codecRegistry(codecRegistry) + .build(); - clients[0] = MongoClients.create(clientSettings); - } - public static void init(ConnectionString connectionString) { - init(connectionString, ReadPreference.secondary(), WriteConcern.ACKNOWLEDGED); - } + return MongoClients.create(clientSettings); + } - public static void createIndices() { - try { - TestingRunResultDao.instance.convertToCappedCollection(); - } catch (Exception e) { - logger.error("Error while converting TestingRunResults to capped collection: " + e.getMessage()); - } + public static void init( + ConnectionString connectionString, ReadPreference readPreference, WriteConcern writeConcern) { + clients[0] = createMongoClient(connectionString, readPreference, writeConcern); + } - OrganizationsDao.createIndexIfAbsent(); - UsageMetricsDao.createIndexIfAbsent(); - SingleTypeInfoDao.instance.createIndicesIfAbsent(); - TrafficMetricsDao.instance.createIndicesIfAbsent(); - TestRolesDao.instance.createIndicesIfAbsent(); - UsersDao.instance.createIndicesIfAbsent(); - AccountsDao.instance.createIndexIfAbsent(); + public static void init(ConnectionString connectionString) { + init(connectionString, ReadPreference.secondary(), WriteConcern.ACKNOWLEDGED); + } - ApiInfoDao.instance.createIndicesIfAbsent(); - RuntimeLogsDao.instance.createIndicesIfAbsent(); - LogsDao.instance.createIndicesIfAbsent(); - DashboardLogsDao.instance.createIndicesIfAbsent(); - AnalyserLogsDao.instance.createIndicesIfAbsent(); - SampleDataDao.instance.createIndicesIfAbsent(); - LoadersDao.instance.createIndicesIfAbsent(); - TestingRunResultDao.instance.createIndicesIfAbsent(); - TestingRunResultSummariesDao.instance.createIndicesIfAbsent(); - TestingRunDao.instance.createIndicesIfAbsent(); - TestingRunIssuesDao.instance.createIndicesIfAbsent(); - ApiCollectionsDao.instance.createIndicesIfAbsent(); - ActivitiesDao.instance.createIndicesIfAbsent(); - DependencyNodeDao.instance.createIndicesIfAbsent(); - DependencyFlowNodesDao.instance.createIndicesIfAbsent(); - CodeAnalysisCollectionDao.instance.createIndicesIfAbsent(); - CodeAnalysisApiInfoDao.instance.createIndicesIfAbsent(); - RBACDao.instance.createIndicesIfAbsent(); - TrafficAlertsDao.instance.createIndicesIfAbsent(); - RuntimeMetricsDao.instance.createIndicesIfAbsent(); - ApiAuditLogsDao.instance.createIndicesIfAbsent(); + public static void createIndices() { + try { + TestingRunResultDao.instance.convertToCappedCollection(); + } catch (Exception e) { + logger.error( + "Error while converting TestingRunResults to capped collection: " + e.getMessage()); } + OrganizationsDao.createIndexIfAbsent(); + UsageMetricsDao.createIndexIfAbsent(); + SingleTypeInfoDao.instance.createIndicesIfAbsent(); + TrafficMetricsDao.instance.createIndicesIfAbsent(); + TestRolesDao.instance.createIndicesIfAbsent(); + UsersDao.instance.createIndicesIfAbsent(); + AccountsDao.instance.createIndexIfAbsent(); + + ApiInfoDao.instance.createIndicesIfAbsent(); + RuntimeLogsDao.instance.createIndicesIfAbsent(); + LogsDao.instance.createIndicesIfAbsent(); + DashboardLogsDao.instance.createIndicesIfAbsent(); + AnalyserLogsDao.instance.createIndicesIfAbsent(); + SampleDataDao.instance.createIndicesIfAbsent(); + LoadersDao.instance.createIndicesIfAbsent(); + TestingRunResultDao.instance.createIndicesIfAbsent(); + TestingRunResultSummariesDao.instance.createIndicesIfAbsent(); + TestingRunDao.instance.createIndicesIfAbsent(); + TestingRunIssuesDao.instance.createIndicesIfAbsent(); + ApiCollectionsDao.instance.createIndicesIfAbsent(); + ActivitiesDao.instance.createIndicesIfAbsent(); + DependencyNodeDao.instance.createIndicesIfAbsent(); + DependencyFlowNodesDao.instance.createIndicesIfAbsent(); + CodeAnalysisCollectionDao.instance.createIndicesIfAbsent(); + CodeAnalysisApiInfoDao.instance.createIndicesIfAbsent(); + RBACDao.instance.createIndicesIfAbsent(); + TrafficAlertsDao.instance.createIndicesIfAbsent(); + RuntimeMetricsDao.instance.createIndicesIfAbsent(); + ApiAuditLogsDao.instance.createIndicesIfAbsent(); + } } diff --git a/libs/dao/src/main/java/com/akto/dao/threat_detection/DetectedThreatAlertDao.java b/libs/dao/src/main/java/com/akto/dao/threat_detection/DetectedThreatAlertDao.java deleted file mode 100644 index 8bec8986e7..0000000000 --- a/libs/dao/src/main/java/com/akto/dao/threat_detection/DetectedThreatAlertDao.java +++ /dev/null @@ -1,20 +0,0 @@ -package com.akto.dao.threat_detection; - -import com.akto.dao.AccountsContextDao; -import com.akto.dto.threat_detection.DetectedThreatAlert; - -public class DetectedThreatAlertDao extends AccountsContextDao { - - public static final DetectedThreatAlertDao instance = new DetectedThreatAlertDao(); - - @Override - public String getCollName() { - return "detected_threat_alerts"; - } - - @Override - public Class getClassT() { - return DetectedThreatAlert.class; - } - -} diff --git a/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleMaliciousRequestDao.java b/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleMaliciousRequestDao.java deleted file mode 100644 index 23fa950efa..0000000000 --- a/libs/dao/src/main/java/com/akto/dao/threat_detection/SampleMaliciousRequestDao.java +++ /dev/null @@ -1,19 +0,0 @@ -package com.akto.dao.threat_detection; - -import com.akto.dao.AccountsContextDao; -import com.akto.dto.threat_detection.SampleMaliciousRequest; - -public class SampleMaliciousRequestDao extends AccountsContextDao { - - public static final SampleMaliciousRequestDao instance = new SampleMaliciousRequestDao(); - - @Override - public String getCollName() { - return "sample_malicious_requests"; - } - - @Override - public Class getClassT() { - return SampleMaliciousRequest.class; - } -} diff --git a/libs/pom.xml b/libs/pom.xml index 8bce86a189..2dfb6103a8 100644 --- a/libs/pom.xml +++ b/libs/pom.xml @@ -18,6 +18,7 @@ dao utils integrations + protobuf diff --git a/libs/protobuf/pom.xml b/libs/protobuf/pom.xml new file mode 100644 index 0000000000..2542736113 --- /dev/null +++ b/libs/protobuf/pom.xml @@ -0,0 +1,60 @@ + + + 4.0.0 + + com.akto.libs + libs + ${revision} + + + com.akto.libs.protobuf + protobuf + jar + + + + + io.grpc + grpc-netty-shaded + 1.68.1 + + + io.grpc + grpc-core + 1.68.1 + + + io.grpc + grpc-protobuf + 1.68.1 + + + io.grpc + grpc-stub + 1.68.1 + + + com.google.protobuf + protobuf-java + 4.28.3 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 8 + 8 + + + + src/main/java + + + \ No newline at end of file diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceGrpc.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceGrpc.java new file mode 100644 index 0000000000..f927cfd0d2 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceGrpc.java @@ -0,0 +1,367 @@ +package com.akto.proto.threat_protection.consumer_service.v1; + +import static io.grpc.MethodDescriptor.generateFullMethodName; + +/** + */ +@javax.annotation.Generated( + value = "by gRPC proto compiler (version 1.68.1)", + comments = "Source: threat_protection/consumer_service/v1/consumer_service.proto") +@io.grpc.stub.annotations.GrpcGenerated +public final class ConsumerServiceGrpc { + + private ConsumerServiceGrpc() {} + + public static final java.lang.String SERVICE_NAME = "threat_protection.consumer_service.v1.ConsumerService"; + + // Static method descriptors that strictly reflect the proto. + private static volatile io.grpc.MethodDescriptor getSaveMaliciousEventMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "SaveMaliciousEvent", + requestType = com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.class, + responseType = com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getSaveMaliciousEventMethod() { + io.grpc.MethodDescriptor getSaveMaliciousEventMethod; + if ((getSaveMaliciousEventMethod = ConsumerServiceGrpc.getSaveMaliciousEventMethod) == null) { + synchronized (ConsumerServiceGrpc.class) { + if ((getSaveMaliciousEventMethod = ConsumerServiceGrpc.getSaveMaliciousEventMethod) == null) { + ConsumerServiceGrpc.getSaveMaliciousEventMethod = getSaveMaliciousEventMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "SaveMaliciousEvent")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.getDefaultInstance())) + .setSchemaDescriptor(new ConsumerServiceMethodDescriptorSupplier("SaveMaliciousEvent")) + .build(); + } + } + } + return getSaveMaliciousEventMethod; + } + + private static volatile io.grpc.MethodDescriptor getSaveSmartEventMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "SaveSmartEvent", + requestType = com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.class, + responseType = com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getSaveSmartEventMethod() { + io.grpc.MethodDescriptor getSaveSmartEventMethod; + if ((getSaveSmartEventMethod = ConsumerServiceGrpc.getSaveSmartEventMethod) == null) { + synchronized (ConsumerServiceGrpc.class) { + if ((getSaveSmartEventMethod = ConsumerServiceGrpc.getSaveSmartEventMethod) == null) { + ConsumerServiceGrpc.getSaveSmartEventMethod = getSaveSmartEventMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "SaveSmartEvent")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.getDefaultInstance())) + .setSchemaDescriptor(new ConsumerServiceMethodDescriptorSupplier("SaveSmartEvent")) + .build(); + } + } + } + return getSaveSmartEventMethod; + } + + /** + * Creates a new async stub that supports all call types for the service + */ + public static ConsumerServiceStub newStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public ConsumerServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceStub(channel, callOptions); + } + }; + return ConsumerServiceStub.newStub(factory, channel); + } + + /** + * Creates a new blocking-style stub that supports unary and streaming output calls on the service + */ + public static ConsumerServiceBlockingStub newBlockingStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public ConsumerServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceBlockingStub(channel, callOptions); + } + }; + return ConsumerServiceBlockingStub.newStub(factory, channel); + } + + /** + * Creates a new ListenableFuture-style stub that supports unary calls on the service + */ + public static ConsumerServiceFutureStub newFutureStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public ConsumerServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceFutureStub(channel, callOptions); + } + }; + return ConsumerServiceFutureStub.newStub(factory, channel); + } + + /** + */ + public interface AsyncService { + + /** + */ + default void saveMaliciousEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getSaveMaliciousEventMethod(), responseObserver); + } + + /** + */ + default void saveSmartEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getSaveSmartEventMethod(), responseObserver); + } + } + + /** + * Base class for the server implementation of the service ConsumerService. + */ + public static abstract class ConsumerServiceImplBase + implements io.grpc.BindableService, AsyncService { + + @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { + return ConsumerServiceGrpc.bindService(this); + } + } + + /** + * A stub to allow clients to do asynchronous rpc calls to service ConsumerService. + */ + public static final class ConsumerServiceStub + extends io.grpc.stub.AbstractAsyncStub { + private ConsumerServiceStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected ConsumerServiceStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceStub(channel, callOptions); + } + + /** + */ + public void saveMaliciousEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getSaveMaliciousEventMethod(), getCallOptions()), request, responseObserver); + } + + /** + */ + public void saveSmartEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getSaveSmartEventMethod(), getCallOptions()), request, responseObserver); + } + } + + /** + * A stub to allow clients to do synchronous rpc calls to service ConsumerService. + */ + public static final class ConsumerServiceBlockingStub + extends io.grpc.stub.AbstractBlockingStub { + private ConsumerServiceBlockingStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected ConsumerServiceBlockingStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceBlockingStub(channel, callOptions); + } + + /** + */ + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse saveMaliciousEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getSaveMaliciousEventMethod(), getCallOptions(), request); + } + + /** + */ + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse saveSmartEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getSaveSmartEventMethod(), getCallOptions(), request); + } + } + + /** + * A stub to allow clients to do ListenableFuture-style rpc calls to service ConsumerService. + */ + public static final class ConsumerServiceFutureStub + extends io.grpc.stub.AbstractFutureStub { + private ConsumerServiceFutureStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected ConsumerServiceFutureStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new ConsumerServiceFutureStub(channel, callOptions); + } + + /** + */ + public com.google.common.util.concurrent.ListenableFuture saveMaliciousEvent( + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getSaveMaliciousEventMethod(), getCallOptions()), request); + } + + /** + */ + public com.google.common.util.concurrent.ListenableFuture saveSmartEvent( + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getSaveSmartEventMethod(), getCallOptions()), request); + } + } + + private static final int METHODID_SAVE_MALICIOUS_EVENT = 0; + private static final int METHODID_SAVE_SMART_EVENT = 1; + + private static final class MethodHandlers implements + io.grpc.stub.ServerCalls.UnaryMethod, + io.grpc.stub.ServerCalls.ServerStreamingMethod, + io.grpc.stub.ServerCalls.ClientStreamingMethod, + io.grpc.stub.ServerCalls.BidiStreamingMethod { + private final AsyncService serviceImpl; + private final int methodId; + + MethodHandlers(AsyncService serviceImpl, int methodId) { + this.serviceImpl = serviceImpl; + this.methodId = methodId; + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + case METHODID_SAVE_MALICIOUS_EVENT: + serviceImpl.saveMaliciousEvent((com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + case METHODID_SAVE_SMART_EVENT: + serviceImpl.saveSmartEvent((com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + default: + throw new AssertionError(); + } + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public io.grpc.stub.StreamObserver invoke( + io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + default: + throw new AssertionError(); + } + } + } + + public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { + return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) + .addMethod( + getSaveMaliciousEventMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest, + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse>( + service, METHODID_SAVE_MALICIOUS_EVENT))) + .addMethod( + getSaveSmartEventMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest, + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse>( + service, METHODID_SAVE_SMART_EVENT))) + .build(); + } + + private static abstract class ConsumerServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { + ConsumerServiceBaseDescriptorSupplier() {} + + @java.lang.Override + public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.getDescriptor(); + } + + @java.lang.Override + public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { + return getFileDescriptor().findServiceByName("ConsumerService"); + } + } + + private static final class ConsumerServiceFileDescriptorSupplier + extends ConsumerServiceBaseDescriptorSupplier { + ConsumerServiceFileDescriptorSupplier() {} + } + + private static final class ConsumerServiceMethodDescriptorSupplier + extends ConsumerServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { + private final java.lang.String methodName; + + ConsumerServiceMethodDescriptorSupplier(java.lang.String methodName) { + this.methodName = methodName; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { + return getServiceDescriptor().findMethodByName(methodName); + } + } + + private static volatile io.grpc.ServiceDescriptor serviceDescriptor; + + public static io.grpc.ServiceDescriptor getServiceDescriptor() { + io.grpc.ServiceDescriptor result = serviceDescriptor; + if (result == null) { + synchronized (ConsumerServiceGrpc.class) { + result = serviceDescriptor; + if (result == null) { + serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) + .setSchemaDescriptor(new ConsumerServiceFileDescriptorSupplier()) + .addMethod(getSaveMaliciousEventMethod()) + .addMethod(getSaveSmartEventMethod()) + .build(); + } + } + } + return result; + } +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java new file mode 100644 index 0000000000..4e312c9884 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java @@ -0,0 +1,146 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +public final class ConsumerServiceProto { + private ConsumerServiceProto() {} + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + ConsumerServiceProto.class.getName()); + } + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_consumer_service_v1_SmartEvent_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_consumer_service_v1_SmartEvent_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_consumer_service_v1_MaliciousEvent_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_consumer_service_v1_MaliciousEvent_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor; + static final + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n builder) { + super(builder); + } + private MaliciousEvent() { + actorId_ = ""; + filterId_ = ""; + ip_ = ""; + url_ = ""; + method_ = ""; + payload_ = ""; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.class, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder.class); + } + + public static final int ACTOR_ID_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object actorId_ = ""; + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The actorId. + */ + @java.lang.Override + public java.lang.String getActorId() { + java.lang.Object ref = actorId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actorId_ = s; + return s; + } + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The bytes for actorId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getActorIdBytes() { + java.lang.Object ref = actorId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actorId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int IP_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object ip_ = ""; + /** + * string ip = 3 [json_name = "ip"]; + * @return The ip. + */ + @java.lang.Override + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } + } + /** + * string ip = 3 [json_name = "ip"]; + * @return The bytes for ip. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TIMESTAMP_FIELD_NUMBER = 4; + private long timestamp_ = 0L; + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + + public static final int URL_FIELD_NUMBER = 5; + @SuppressWarnings("serial") + private volatile java.lang.Object url_ = ""; + /** + * string url = 5 [json_name = "url"]; + * @return The url. + */ + @java.lang.Override + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } + } + /** + * string url = 5 [json_name = "url"]; + * @return The bytes for url. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int METHOD_FIELD_NUMBER = 6; + @SuppressWarnings("serial") + private volatile java.lang.Object method_ = ""; + /** + * string method = 6 [json_name = "method"]; + * @return The method. + */ + @java.lang.Override + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } + } + /** + * string method = 6 [json_name = "method"]; + * @return The bytes for method. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int API_COLLECTION_ID_FIELD_NUMBER = 7; + private int apiCollectionId_ = 0; + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + + public static final int PAYLOAD_FIELD_NUMBER = 8; + @SuppressWarnings("serial") + private volatile java.lang.Object payload_ = ""; + /** + * string payload = 8 [json_name = "payload"]; + * @return The payload. + */ + @java.lang.Override + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } + } + /** + * string payload = 8 [json_name = "payload"]; + * @return The bytes for payload. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actorId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, actorId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 3, ip_); + } + if (timestamp_ != 0L) { + output.writeInt64(4, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 5, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 6, method_); + } + if (apiCollectionId_ != 0) { + output.writeInt32(7, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 8, payload_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actorId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actorId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(3, ip_); + } + if (timestamp_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(4, timestamp_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(5, url_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(6, method_); + } + if (apiCollectionId_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(7, apiCollectionId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(8, payload_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent other = (com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent) obj; + + if (!getActorId() + .equals(other.getActorId())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (!getIp() + .equals(other.getIp())) return false; + if (getTimestamp() + != other.getTimestamp()) return false; + if (!getUrl() + .equals(other.getUrl())) return false; + if (!getMethod() + .equals(other.getMethod())) return false; + if (getApiCollectionId() + != other.getApiCollectionId()) return false; + if (!getPayload() + .equals(other.getPayload())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACTOR_ID_FIELD_NUMBER; + hash = (53 * hash) + getActorId().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (37 * hash) + IP_FIELD_NUMBER; + hash = (53 * hash) + getIp().hashCode(); + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getTimestamp()); + hash = (37 * hash) + URL_FIELD_NUMBER; + hash = (53 * hash) + getUrl().hashCode(); + hash = (37 * hash) + METHOD_FIELD_NUMBER; + hash = (53 * hash) + getMethod().hashCode(); + hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; + hash = (53 * hash) + getApiCollectionId(); + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.consumer_service.v1.MaliciousEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.MaliciousEvent) + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.class, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + actorId_ = ""; + filterId_ = ""; + ip_ = ""; + timestamp_ = 0L; + url_ = ""; + method_ = ""; + apiCollectionId_ = 0; + payload_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getDefaultInstanceForType() { + return com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent build() { + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent buildPartial() { + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent result = new com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.actorId_ = actorId_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.filterId_ = filterId_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.ip_ = ip_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.timestamp_ = timestamp_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.url_ = url_; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.method_ = method_; + } + if (((from_bitField0_ & 0x00000040) != 0)) { + result.apiCollectionId_ = apiCollectionId_; + } + if (((from_bitField0_ & 0x00000080) != 0)) { + result.payload_ = payload_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent) { + return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent other) { + if (other == com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.getDefaultInstance()) return this; + if (!other.getActorId().isEmpty()) { + actorId_ = other.actorId_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (!other.getIp().isEmpty()) { + ip_ = other.ip_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (other.getTimestamp() != 0L) { + setTimestamp(other.getTimestamp()); + } + if (!other.getUrl().isEmpty()) { + url_ = other.url_; + bitField0_ |= 0x00000010; + onChanged(); + } + if (!other.getMethod().isEmpty()) { + method_ = other.method_; + bitField0_ |= 0x00000020; + onChanged(); + } + if (other.getApiCollectionId() != 0) { + setApiCollectionId(other.getApiCollectionId()); + } + if (!other.getPayload().isEmpty()) { + payload_ = other.payload_; + bitField0_ |= 0x00000080; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + actorId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + ip_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 32: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000008; + break; + } // case 32 + case 42: { + url_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000010; + break; + } // case 42 + case 50: { + method_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000020; + break; + } // case 50 + case 56: { + apiCollectionId_ = input.readInt32(); + bitField0_ |= 0x00000040; + break; + } // case 56 + case 66: { + payload_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000080; + break; + } // case 66 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object actorId_ = ""; + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The actorId. + */ + public java.lang.String getActorId() { + java.lang.Object ref = actorId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actorId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The bytes for actorId. + */ + public com.google.protobuf.ByteString + getActorIdBytes() { + java.lang.Object ref = actorId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actorId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @param value The actorId to set. + * @return This builder for chaining. + */ + public Builder setActorId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + actorId_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return This builder for chaining. + */ + public Builder clearActorId() { + actorId_ = getDefaultInstance().getActorId(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @param value The bytes for actorId to set. + * @return This builder for chaining. + */ + public Builder setActorIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + actorId_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private java.lang.Object ip_ = ""; + /** + * string ip = 3 [json_name = "ip"]; + * @return The ip. + */ + public java.lang.String getIp() { + java.lang.Object ref = ip_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ip_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string ip = 3 [json_name = "ip"]; + * @return The bytes for ip. + */ + public com.google.protobuf.ByteString + getIpBytes() { + java.lang.Object ref = ip_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ip_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string ip = 3 [json_name = "ip"]; + * @param value The ip to set. + * @return This builder for chaining. + */ + public Builder setIp( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ip_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string ip = 3 [json_name = "ip"]; + * @return This builder for chaining. + */ + public Builder clearIp() { + ip_ = getDefaultInstance().getIp(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string ip = 3 [json_name = "ip"]; + * @param value The bytes for ip to set. + * @return This builder for chaining. + */ + public Builder setIpBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ip_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private long timestamp_ ; + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + + timestamp_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000008); + timestamp_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object url_ = ""; + /** + * string url = 5 [json_name = "url"]; + * @return The url. + */ + public java.lang.String getUrl() { + java.lang.Object ref = url_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + url_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string url = 5 [json_name = "url"]; + * @return The bytes for url. + */ + public com.google.protobuf.ByteString + getUrlBytes() { + java.lang.Object ref = url_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + url_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string url = 5 [json_name = "url"]; + * @param value The url to set. + * @return This builder for chaining. + */ + public Builder setUrl( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + url_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * string url = 5 [json_name = "url"]; + * @return This builder for chaining. + */ + public Builder clearUrl() { + url_ = getDefaultInstance().getUrl(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + return this; + } + /** + * string url = 5 [json_name = "url"]; + * @param value The bytes for url to set. + * @return This builder for chaining. + */ + public Builder setUrlBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + url_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + + private java.lang.Object method_ = ""; + /** + * string method = 6 [json_name = "method"]; + * @return The method. + */ + public java.lang.String getMethod() { + java.lang.Object ref = method_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + method_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string method = 6 [json_name = "method"]; + * @return The bytes for method. + */ + public com.google.protobuf.ByteString + getMethodBytes() { + java.lang.Object ref = method_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + method_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string method = 6 [json_name = "method"]; + * @param value The method to set. + * @return This builder for chaining. + */ + public Builder setMethod( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + method_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * string method = 6 [json_name = "method"]; + * @return This builder for chaining. + */ + public Builder clearMethod() { + method_ = getDefaultInstance().getMethod(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + return this; + } + /** + * string method = 6 [json_name = "method"]; + * @param value The bytes for method to set. + * @return This builder for chaining. + */ + public Builder setMethodBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + method_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + + private int apiCollectionId_ ; + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + @java.lang.Override + public int getApiCollectionId() { + return apiCollectionId_; + } + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @param value The apiCollectionId to set. + * @return This builder for chaining. + */ + public Builder setApiCollectionId(int value) { + + apiCollectionId_ = value; + bitField0_ |= 0x00000040; + onChanged(); + return this; + } + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @return This builder for chaining. + */ + public Builder clearApiCollectionId() { + bitField0_ = (bitField0_ & ~0x00000040); + apiCollectionId_ = 0; + onChanged(); + return this; + } + + private java.lang.Object payload_ = ""; + /** + * string payload = 8 [json_name = "payload"]; + * @return The payload. + */ + public java.lang.String getPayload() { + java.lang.Object ref = payload_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + payload_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string payload = 8 [json_name = "payload"]; + * @return The bytes for payload. + */ + public com.google.protobuf.ByteString + getPayloadBytes() { + java.lang.Object ref = payload_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + payload_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string payload = 8 [json_name = "payload"]; + * @param value The payload to set. + * @return This builder for chaining. + */ + public Builder setPayload( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + payload_ = value; + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + /** + * string payload = 8 [json_name = "payload"]; + * @return This builder for chaining. + */ + public Builder clearPayload() { + payload_ = getDefaultInstance().getPayload(); + bitField0_ = (bitField0_ & ~0x00000080); + onChanged(); + return this; + } + /** + * string payload = 8 [json_name = "payload"]; + * @param value The bytes for payload to set. + * @return This builder for chaining. + */ + public Builder setPayloadBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + payload_ = value; + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.MaliciousEvent) + } + + // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.MaliciousEvent) + private static final com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent(); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MaliciousEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEventOrBuilder.java new file mode 100644 index 0000000000..5abab768f8 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEventOrBuilder.java @@ -0,0 +1,95 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +public interface MaliciousEventOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.MaliciousEvent) + com.google.protobuf.MessageOrBuilder { + + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The actorId. + */ + java.lang.String getActorId(); + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The bytes for actorId. + */ + com.google.protobuf.ByteString + getActorIdBytes(); + + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + java.lang.String getFilterId(); + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + com.google.protobuf.ByteString + getFilterIdBytes(); + + /** + * string ip = 3 [json_name = "ip"]; + * @return The ip. + */ + java.lang.String getIp(); + /** + * string ip = 3 [json_name = "ip"]; + * @return The bytes for ip. + */ + com.google.protobuf.ByteString + getIpBytes(); + + /** + * int64 timestamp = 4 [json_name = "timestamp"]; + * @return The timestamp. + */ + long getTimestamp(); + + /** + * string url = 5 [json_name = "url"]; + * @return The url. + */ + java.lang.String getUrl(); + /** + * string url = 5 [json_name = "url"]; + * @return The bytes for url. + */ + com.google.protobuf.ByteString + getUrlBytes(); + + /** + * string method = 6 [json_name = "method"]; + * @return The method. + */ + java.lang.String getMethod(); + /** + * string method = 6 [json_name = "method"]; + * @return The bytes for method. + */ + com.google.protobuf.ByteString + getMethodBytes(); + + /** + * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; + * @return The apiCollectionId. + */ + int getApiCollectionId(); + + /** + * string payload = 8 [json_name = "payload"]; + * @return The payload. + */ + java.lang.String getPayload(); + /** + * string payload = 8 [json_name = "payload"]; + * @return The bytes for payload. + */ + com.google.protobuf.ByteString + getPayloadBytes(); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java new file mode 100644 index 0000000000..b5e27a2559 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java @@ -0,0 +1,785 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +/** + * Protobuf type {@code threat_protection.consumer_service.v1.SaveMaliciousEventRequest} + */ +public final class SaveMaliciousEventRequest extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) + SaveMaliciousEventRequestOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SaveMaliciousEventRequest.class.getName()); + } + // Use SaveMaliciousEventRequest.newBuilder() to construct. + private SaveMaliciousEventRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SaveMaliciousEventRequest() { + events_ = java.util.Collections.emptyList(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.Builder.class); + } + + public static final int ACCOUNT_ID_FIELD_NUMBER = 1; + private int accountId_ = 0; + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @return The accountId. + */ + @java.lang.Override + public int getAccountId() { + return accountId_; + } + + public static final int EVENTS_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private java.util.List events_; + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + @java.lang.Override + public java.util.List getEventsList() { + return events_; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + @java.lang.Override + public java.util.List + getEventsOrBuilderList() { + return events_; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + @java.lang.Override + public int getEventsCount() { + return events_.size(); + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getEvents(int index) { + return events_.get(index); + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder getEventsOrBuilder( + int index) { + return events_.get(index); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (accountId_ != 0) { + output.writeInt32(1, accountId_); + } + for (int i = 0; i < events_.size(); i++) { + output.writeMessage(2, events_.get(i)); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (accountId_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(1, accountId_); + } + for (int i = 0; i < events_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, events_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest other = (com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest) obj; + + if (getAccountId() + != other.getAccountId()) return false; + if (!getEventsList() + .equals(other.getEventsList())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACCOUNT_ID_FIELD_NUMBER; + hash = (53 * hash) + getAccountId(); + if (getEventsCount() > 0) { + hash = (37 * hash) + EVENTS_FIELD_NUMBER; + hash = (53 * hash) + getEventsList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.consumer_service.v1.SaveMaliciousEventRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + accountId_ = 0; + if (eventsBuilder_ == null) { + events_ = java.util.Collections.emptyList(); + } else { + events_ = null; + eventsBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest build() { + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest buildPartial() { + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result = new com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest(this); + buildPartialRepeatedFields(result); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartialRepeatedFields(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result) { + if (eventsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0)) { + events_ = java.util.Collections.unmodifiableList(events_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.events_ = events_; + } else { + result.events_ = eventsBuilder_.build(); + } + } + + private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.accountId_ = accountId_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest) { + return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest other) { + if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.getDefaultInstance()) return this; + if (other.getAccountId() != 0) { + setAccountId(other.getAccountId()); + } + if (eventsBuilder_ == null) { + if (!other.events_.isEmpty()) { + if (events_.isEmpty()) { + events_ = other.events_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureEventsIsMutable(); + events_.addAll(other.events_); + } + onChanged(); + } + } else { + if (!other.events_.isEmpty()) { + if (eventsBuilder_.isEmpty()) { + eventsBuilder_.dispose(); + eventsBuilder_ = null; + events_ = other.events_; + bitField0_ = (bitField0_ & ~0x00000002); + eventsBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getEventsFieldBuilder() : null; + } else { + eventsBuilder_.addAllMessages(other.events_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + accountId_ = input.readInt32(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 18: { + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent m = + input.readMessage( + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.parser(), + extensionRegistry); + if (eventsBuilder_ == null) { + ensureEventsIsMutable(); + events_.add(m); + } else { + eventsBuilder_.addMessage(m); + } + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private int accountId_ ; + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @return The accountId. + */ + @java.lang.Override + public int getAccountId() { + return accountId_; + } + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @param value The accountId to set. + * @return This builder for chaining. + */ + public Builder setAccountId(int value) { + + accountId_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @return This builder for chaining. + */ + public Builder clearAccountId() { + bitField0_ = (bitField0_ & ~0x00000001); + accountId_ = 0; + onChanged(); + return this; + } + + private java.util.List events_ = + java.util.Collections.emptyList(); + private void ensureEventsIsMutable() { + if (!((bitField0_ & 0x00000002) != 0)) { + events_ = new java.util.ArrayList(events_); + bitField0_ |= 0x00000002; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder> eventsBuilder_; + + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public java.util.List getEventsList() { + if (eventsBuilder_ == null) { + return java.util.Collections.unmodifiableList(events_); + } else { + return eventsBuilder_.getMessageList(); + } + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public int getEventsCount() { + if (eventsBuilder_ == null) { + return events_.size(); + } else { + return eventsBuilder_.getCount(); + } + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getEvents(int index) { + if (eventsBuilder_ == null) { + return events_.get(index); + } else { + return eventsBuilder_.getMessage(index); + } + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder setEvents( + int index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent value) { + if (eventsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEventsIsMutable(); + events_.set(index, value); + onChanged(); + } else { + eventsBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder setEvents( + int index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder builderForValue) { + if (eventsBuilder_ == null) { + ensureEventsIsMutable(); + events_.set(index, builderForValue.build()); + onChanged(); + } else { + eventsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder addEvents(com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent value) { + if (eventsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEventsIsMutable(); + events_.add(value); + onChanged(); + } else { + eventsBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder addEvents( + int index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent value) { + if (eventsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEventsIsMutable(); + events_.add(index, value); + onChanged(); + } else { + eventsBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder addEvents( + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder builderForValue) { + if (eventsBuilder_ == null) { + ensureEventsIsMutable(); + events_.add(builderForValue.build()); + onChanged(); + } else { + eventsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder addEvents( + int index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder builderForValue) { + if (eventsBuilder_ == null) { + ensureEventsIsMutable(); + events_.add(index, builderForValue.build()); + onChanged(); + } else { + eventsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder addAllEvents( + java.lang.Iterable values) { + if (eventsBuilder_ == null) { + ensureEventsIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, events_); + onChanged(); + } else { + eventsBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder clearEvents() { + if (eventsBuilder_ == null) { + events_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + } else { + eventsBuilder_.clear(); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public Builder removeEvents(int index) { + if (eventsBuilder_ == null) { + ensureEventsIsMutable(); + events_.remove(index); + onChanged(); + } else { + eventsBuilder_.remove(index); + } + return this; + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder getEventsBuilder( + int index) { + return getEventsFieldBuilder().getBuilder(index); + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder getEventsOrBuilder( + int index) { + if (eventsBuilder_ == null) { + return events_.get(index); } else { + return eventsBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public java.util.List + getEventsOrBuilderList() { + if (eventsBuilder_ != null) { + return eventsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(events_); + } + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder addEventsBuilder() { + return getEventsFieldBuilder().addBuilder( + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.getDefaultInstance()); + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder addEventsBuilder( + int index) { + return getEventsFieldBuilder().addBuilder( + index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.getDefaultInstance()); + } + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + public java.util.List + getEventsBuilderList() { + return getEventsFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder> + getEventsFieldBuilder() { + if (eventsBuilder_ == null) { + eventsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder>( + events_, + ((bitField0_ & 0x00000002) != 0), + getParentForChildren(), + isClean()); + events_ = null; + } + return eventsBuilder_; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) + private static final com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest(); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SaveMaliciousEventRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java new file mode 100644 index 0000000000..55c6786ad1 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java @@ -0,0 +1,41 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +public interface SaveMaliciousEventRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @return The accountId. + */ + int getAccountId(); + + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + java.util.List + getEventsList(); + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getEvents(int index); + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + int getEventsCount(); + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + java.util.List + getEventsOrBuilderList(); + /** + * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; + */ + com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder getEventsOrBuilder( + int index); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponse.java new file mode 100644 index 0000000000..c4c28a8bba --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponse.java @@ -0,0 +1,358 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +/** + * Protobuf type {@code threat_protection.consumer_service.v1.SaveMaliciousEventResponse} + */ +public final class SaveMaliciousEventResponse extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) + SaveMaliciousEventResponseOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SaveMaliciousEventResponse.class.getName()); + } + // Use SaveMaliciousEventResponse.newBuilder() to construct. + private SaveMaliciousEventResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SaveMaliciousEventResponse() { + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.Builder.class); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse other = (com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse) obj; + + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.consumer_service.v1.SaveMaliciousEventResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse getDefaultInstanceForType() { + return com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse build() { + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse buildPartial() { + com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse result = new com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse) { + return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse other) { + if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) + } + + // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) + private static final com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse(); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SaveMaliciousEventResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponseOrBuilder.java new file mode 100644 index 0000000000..9c481b6591 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponseOrBuilder.java @@ -0,0 +1,11 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +public interface SaveMaliciousEventResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) + com.google.protobuf.MessageOrBuilder { +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java new file mode 100644 index 0000000000..48fb08d291 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java @@ -0,0 +1,624 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +/** + * Protobuf type {@code threat_protection.consumer_service.v1.SaveSmartEventRequest} + */ +public final class SaveSmartEventRequest extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SaveSmartEventRequest) + SaveSmartEventRequestOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SaveSmartEventRequest.class.getName()); + } + // Use SaveSmartEventRequest.newBuilder() to construct. + private SaveSmartEventRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SaveSmartEventRequest() { + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.Builder.class); + } + + private int bitField0_; + public static final int ACCOUNT_ID_FIELD_NUMBER = 1; + private int accountId_ = 0; + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @return The accountId. + */ + @java.lang.Override + public int getAccountId() { + return accountId_; + } + + public static final int EVENT_FIELD_NUMBER = 2; + private com.akto.proto.threat_protection.consumer_service.v1.SmartEvent event_; + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + * @return Whether the event field is set. + */ + @java.lang.Override + public boolean hasEvent() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + * @return The event. + */ + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getEvent() { + return event_ == null ? com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance() : event_; + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder getEventOrBuilder() { + return event_ == null ? com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance() : event_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (accountId_ != 0) { + output.writeInt32(1, accountId_); + } + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(2, getEvent()); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (accountId_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(1, accountId_); + } + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getEvent()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest other = (com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest) obj; + + if (getAccountId() + != other.getAccountId()) return false; + if (hasEvent() != other.hasEvent()) return false; + if (hasEvent()) { + if (!getEvent() + .equals(other.getEvent())) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACCOUNT_ID_FIELD_NUMBER; + hash = (53 * hash) + getAccountId(); + if (hasEvent()) { + hash = (37 * hash) + EVENT_FIELD_NUMBER; + hash = (53 * hash) + getEvent().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.consumer_service.v1.SaveSmartEventRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SaveSmartEventRequest) + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage + .alwaysUseFieldBuilders) { + getEventFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + accountId_ = 0; + event_ = null; + if (eventBuilder_ != null) { + eventBuilder_.dispose(); + eventBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest getDefaultInstanceForType() { + return com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest build() { + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest buildPartial() { + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest result = new com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.accountId_ = accountId_; + } + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000002) != 0)) { + result.event_ = eventBuilder_ == null + ? event_ + : eventBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest) { + return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest other) { + if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.getDefaultInstance()) return this; + if (other.getAccountId() != 0) { + setAccountId(other.getAccountId()); + } + if (other.hasEvent()) { + mergeEvent(other.getEvent()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + accountId_ = input.readInt32(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 18: { + input.readMessage( + getEventFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private int accountId_ ; + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @return The accountId. + */ + @java.lang.Override + public int getAccountId() { + return accountId_; + } + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @param value The accountId to set. + * @return This builder for chaining. + */ + public Builder setAccountId(int value) { + + accountId_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @return This builder for chaining. + */ + public Builder clearAccountId() { + bitField0_ = (bitField0_ & ~0x00000001); + accountId_ = 0; + onChanged(); + return this; + } + + private com.akto.proto.threat_protection.consumer_service.v1.SmartEvent event_; + private com.google.protobuf.SingleFieldBuilder< + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder> eventBuilder_; + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + * @return Whether the event field is set. + */ + public boolean hasEvent() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + * @return The event. + */ + public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getEvent() { + if (eventBuilder_ == null) { + return event_ == null ? com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance() : event_; + } else { + return eventBuilder_.getMessage(); + } + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + public Builder setEvent(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent value) { + if (eventBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + event_ = value; + } else { + eventBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + public Builder setEvent( + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder builderForValue) { + if (eventBuilder_ == null) { + event_ = builderForValue.build(); + } else { + eventBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + public Builder mergeEvent(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent value) { + if (eventBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + event_ != null && + event_ != com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance()) { + getEventBuilder().mergeFrom(value); + } else { + event_ = value; + } + } else { + eventBuilder_.mergeFrom(value); + } + if (event_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + public Builder clearEvent() { + bitField0_ = (bitField0_ & ~0x00000002); + event_ = null; + if (eventBuilder_ != null) { + eventBuilder_.dispose(); + eventBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder getEventBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getEventFieldBuilder().getBuilder(); + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + public com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder getEventOrBuilder() { + if (eventBuilder_ != null) { + return eventBuilder_.getMessageOrBuilder(); + } else { + return event_ == null ? + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance() : event_; + } + } + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + private com.google.protobuf.SingleFieldBuilder< + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder> + getEventFieldBuilder() { + if (eventBuilder_ == null) { + eventBuilder_ = new com.google.protobuf.SingleFieldBuilder< + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder>( + getEvent(), + getParentForChildren(), + isClean()); + event_ = null; + } + return eventBuilder_; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SaveSmartEventRequest) + } + + // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SaveSmartEventRequest) + private static final com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest(); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SaveSmartEventRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java new file mode 100644 index 0000000000..73f2b031ba --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java @@ -0,0 +1,32 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +public interface SaveSmartEventRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveSmartEventRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * int32 account_id = 1 [json_name = "accountId"]; + * @return The accountId. + */ + int getAccountId(); + + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + * @return Whether the event field is set. + */ + boolean hasEvent(); + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + * @return The event. + */ + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getEvent(); + /** + * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; + */ + com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder getEventOrBuilder(); +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponse.java new file mode 100644 index 0000000000..809931e36a --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponse.java @@ -0,0 +1,358 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +/** + * Protobuf type {@code threat_protection.consumer_service.v1.SaveSmartEventResponse} + */ +public final class SaveSmartEventResponse extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SaveSmartEventResponse) + SaveSmartEventResponseOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SaveSmartEventResponse.class.getName()); + } + // Use SaveSmartEventResponse.newBuilder() to construct. + private SaveSmartEventResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SaveSmartEventResponse() { + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.class, com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.Builder.class); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse other = (com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse) obj; + + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.consumer_service.v1.SaveSmartEventResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SaveSmartEventResponse) + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.class, com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse getDefaultInstanceForType() { + return com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse build() { + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse buildPartial() { + com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse result = new com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse) { + return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse other) { + if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SaveSmartEventResponse) + } + + // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SaveSmartEventResponse) + private static final com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse(); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SaveSmartEventResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponseOrBuilder.java new file mode 100644 index 0000000000..a551db5627 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponseOrBuilder.java @@ -0,0 +1,11 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +public interface SaveSmartEventResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveSmartEventResponse) + com.google.protobuf.MessageOrBuilder { +} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEvent.java new file mode 100644 index 0000000000..9d74c8d817 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEvent.java @@ -0,0 +1,704 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +/** + * Protobuf type {@code threat_protection.consumer_service.v1.SmartEvent} + */ +public final class SmartEvent extends + com.google.protobuf.GeneratedMessage implements + // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SmartEvent) + SmartEventOrBuilder { +private static final long serialVersionUID = 0L; + static { + com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( + com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, + /* major= */ 4, + /* minor= */ 28, + /* patch= */ 3, + /* suffix= */ "", + SmartEvent.class.getName()); + } + // Use SmartEvent.newBuilder() to construct. + private SmartEvent(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + } + private SmartEvent() { + actorId_ = ""; + filterId_ = ""; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.class, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder.class); + } + + public static final int ACTOR_ID_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object actorId_ = ""; + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The actorId. + */ + @java.lang.Override + public java.lang.String getActorId() { + java.lang.Object ref = actorId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actorId_ = s; + return s; + } + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The bytes for actorId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getActorIdBytes() { + java.lang.Object ref = actorId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actorId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FILTER_ID_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + @java.lang.Override + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DETECTED_AT_FIELD_NUMBER = 3; + private long detectedAt_ = 0L; + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actorId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 1, actorId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); + } + if (detectedAt_ != 0L) { + output.writeInt64(3, detectedAt_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actorId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actorId_); + } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); + } + if (detectedAt_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(3, detectedAt_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SmartEvent)) { + return super.equals(obj); + } + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent other = (com.akto.proto.threat_protection.consumer_service.v1.SmartEvent) obj; + + if (!getActorId() + .equals(other.getActorId())) return false; + if (!getFilterId() + .equals(other.getFilterId())) return false; + if (getDetectedAt() + != other.getDetectedAt()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ACTOR_ID_FIELD_NUMBER; + hash = (53 * hash) + getActorId().hashCode(); + hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; + hash = (53 * hash) + getFilterId().hashCode(); + hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getDetectedAt()); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input); + } + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessage + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code threat_protection.consumer_service.v1.SmartEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder implements + // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SmartEvent) + com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.class, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder.class); + } + + // Construct using com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + actorId_ = ""; + filterId_ = ""; + detectedAt_ = 0L; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_descriptor; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getDefaultInstanceForType() { + return com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance(); + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent build() { + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent buildPartial() { + com.akto.proto.threat_protection.consumer_service.v1.SmartEvent result = new com.akto.proto.threat_protection.consumer_service.v1.SmartEvent(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.actorId_ = actorId_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.filterId_ = filterId_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.detectedAt_ = detectedAt_; + } + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SmartEvent) { + return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SmartEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent other) { + if (other == com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance()) return this; + if (!other.getActorId().isEmpty()) { + actorId_ = other.actorId_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getFilterId().isEmpty()) { + filterId_ = other.filterId_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (other.getDetectedAt() != 0L) { + setDetectedAt(other.getDetectedAt()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + actorId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + filterId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + detectedAt_ = input.readInt64(); + bitField0_ |= 0x00000004; + break; + } // case 24 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object actorId_ = ""; + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The actorId. + */ + public java.lang.String getActorId() { + java.lang.Object ref = actorId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + actorId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The bytes for actorId. + */ + public com.google.protobuf.ByteString + getActorIdBytes() { + java.lang.Object ref = actorId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + actorId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @param value The actorId to set. + * @return This builder for chaining. + */ + public Builder setActorId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + actorId_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return This builder for chaining. + */ + public Builder clearActorId() { + actorId_ = getDefaultInstance().getActorId(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string actor_id = 1 [json_name = "actorId"]; + * @param value The bytes for actorId to set. + * @return This builder for chaining. + */ + public Builder setActorIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + actorId_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object filterId_ = ""; + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + public java.lang.String getFilterId() { + java.lang.Object ref = filterId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + filterId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + public com.google.protobuf.ByteString + getFilterIdBytes() { + java.lang.Object ref = filterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + filterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return This builder for chaining. + */ + public Builder clearFilterId() { + filterId_ = getDefaultInstance().getFilterId(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string filter_id = 2 [json_name = "filterId"]; + * @param value The bytes for filterId to set. + * @return This builder for chaining. + */ + public Builder setFilterIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + filterId_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private long detectedAt_ ; + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + @java.lang.Override + public long getDetectedAt() { + return detectedAt_; + } + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @param value The detectedAt to set. + * @return This builder for chaining. + */ + public Builder setDetectedAt(long value) { + + detectedAt_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return This builder for chaining. + */ + public Builder clearDetectedAt() { + bitField0_ = (bitField0_ & ~0x00000004); + detectedAt_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SmartEvent) + } + + // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SmartEvent) + private static final com.akto.proto.threat_protection.consumer_service.v1.SmartEvent DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SmartEvent(); + } + + public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SmartEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java new file mode 100644 index 0000000000..e1e1eab5db --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java @@ -0,0 +1,41 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// NO CHECKED-IN PROTOBUF GENCODE +// source: threat_protection/consumer_service/v1/consumer_service.proto +// Protobuf Java Version: 4.28.3 + +package com.akto.proto.threat_protection.consumer_service.v1; + +public interface SmartEventOrBuilder extends + // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SmartEvent) + com.google.protobuf.MessageOrBuilder { + + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The actorId. + */ + java.lang.String getActorId(); + /** + * string actor_id = 1 [json_name = "actorId"]; + * @return The bytes for actorId. + */ + com.google.protobuf.ByteString + getActorIdBytes(); + + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The filterId. + */ + java.lang.String getFilterId(); + /** + * string filter_id = 2 [json_name = "filterId"]; + * @return The bytes for filterId. + */ + com.google.protobuf.ByteString + getFilterIdBytes(); + + /** + * int64 detected_at = 3 [json_name = "detectedAt"]; + * @return The detectedAt. + */ + long getDetectedAt(); +} diff --git a/protobuf/buf.yaml b/protobuf/buf.yaml new file mode 100644 index 0000000000..534fcc9218 --- /dev/null +++ b/protobuf/buf.yaml @@ -0,0 +1,7 @@ +version: v2 +breaking: + use: + - FILE +lint: + use: + - STANDARD diff --git a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto new file mode 100644 index 0000000000..304165794d --- /dev/null +++ b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto @@ -0,0 +1,47 @@ +syntax = "proto3"; + +package threat_protection.consumer_service.v1; + +// This is a consumer service for saving malicious and smart events. +// For dashboard purposes we will have a separate service to retrieve these events. +option java_outer_classname = "ConsumerServiceProto"; +option java_package = "threat_protection.v1.consumer_service"; + +message SaveMaliciousEventResponse { +} + +message SaveSmartEventResponse { +} + +message SmartEvent { + string actor_id= 1; + string filter_id = 2; + int64 detected_at = 3; +} + +message MaliciousEvent { + string actor_id= 1; + string filter_id = 2; + string ip = 3; + int64 timestamp = 4; + string url = 5; + string method = 6; + int32 api_collection_id = 7; + string payload = 8; +} + +message SaveMaliciousEventRequest { + int32 account_id = 1; + repeated MaliciousEvent events = 2; +} + +message SaveSmartEventRequest { + int32 account_id = 1; + SmartEvent event = 2; +} + +service ConsumerService { + rpc SaveMaliciousEvent(SaveMaliciousEventRequest) returns (SaveMaliciousEventResponse) {} + + rpc SaveSmartEvent(SaveSmartEventRequest) returns (SaveSmartEventResponse) {} +} \ No newline at end of file From cc8c31fe70126f78e548cf4b84550b69ccf5d0c2 Mon Sep 17 00:00:00 2001 From: ayushaga14 Date: Fri, 22 Nov 2024 11:15:37 +0530 Subject: [PATCH 20/73] apply aggregations --- .../java/com/akto/filters/HttpCallFilter.java | 50 ++++--- .../WindowBasedThresholdNotifier.java | 8 +- .../v1/ConsumerServiceProto.java | 61 ++++---- .../consumer_service/v1/SmartEvent.java | 136 ++++++++++++++++++ .../v1/SmartEventOrBuilder.java | 12 ++ .../v1/consumer_service.proto | 1 + 6 files changed, 216 insertions(+), 52 deletions(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 9b11828a97..117209df1b 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -8,6 +8,9 @@ import com.akto.data_actor.DataActor; import com.akto.data_actor.DataActorFactory; import com.akto.dto.ApiInfo.ApiInfoKey; +import com.akto.dto.api_protection_parse_layer.AggregationRules; +import com.akto.dto.api_protection_parse_layer.Condition; +import com.akto.dto.api_protection_parse_layer.Rule; import com.akto.dto.HttpResponseParams; import com.akto.dto.RawApi; import com.akto.dto.monitoring.FilterConfig; @@ -101,6 +104,11 @@ public void filterFunction(List responseParams) { // But regardless of whether request falls in aggregation or not, // we still push malicious requests to kafka + // todo: modify fetch yaml and read aggregate rules from it + List rules = new ArrayList<>(); + rules.add(new Rule("Lfi Rule 1", new Condition(100, 10))); + AggregationRules aggRules = new AggregationRules(); + SourceIPKeyGenerator.instance .generate(responseParam) .ifPresent( @@ -123,26 +131,30 @@ public void filterFunction(List responseParams) { new Message( responseParam.getAccountId(), maliciousEvent)); - - WindowBasedThresholdNotifier.Result result = this.windowBasedThresholdNotifier - .shouldNotify( - aggKey, maliciousEvent); - - if (result.shouldNotify()) { - this.consumerServiceBlockingStub.saveSmartEvent( - SaveSmartEventRequest - .newBuilder() - .setAccountId( - Integer.parseInt(responseParam.getAccountId())) - .setEvent( - SmartEvent.newBuilder() - .setFilterId(apiFilter.getId()) - .setActorId(actor) - .setDetectedAt(responseParam.getTime()) - .build()) - .build()); + + for (Rule rule: aggRules.getRule()) { + WindowBasedThresholdNotifier.Result result = this.windowBasedThresholdNotifier + .shouldNotify( + aggKey, maliciousEvent, rule); + + if (result.shouldNotify()) { + this.consumerServiceBlockingStub.saveSmartEvent( + SaveSmartEventRequest + .newBuilder() + .setAccountId( + Integer.parseInt(responseParam.getAccountId())) + .setEvent( + SmartEvent.newBuilder() + .setFilterId(apiFilter.getId()) + .setActorId(actor) + .setDetectedAt(responseParam.getTime()) + .setRuleId(actor) + .build()) + .build()); + } } - }); + }); + } } } diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java index af7e7d6572..9fa8748d2f 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java @@ -6,6 +6,8 @@ import java.util.concurrent.ConcurrentHashMap; import com.akto.cache.CounterCache; +import com.akto.dto.api_protection_parse_layer.AggregationRules; +import com.akto.dto.api_protection_parse_layer.Rule; import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; public class WindowBasedThresholdNotifier { @@ -66,18 +68,18 @@ public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.notifiedMap = new ConcurrentHashMap<>(); } - public Result shouldNotify(String aggKey, MaliciousEvent maliciousEvent) { + public Result shouldNotify(String aggKey, MaliciousEvent maliciousEvent, Rule rule) { int binId = (int) maliciousEvent.getTimestamp() / 60; String cacheKey = aggKey + "|" + binId; this.cache.increment(cacheKey); long windowCount = 0L; - List bins = getBins(aggKey, binId - this.config.getWindowSizeInMinutes() + 1, binId); + List bins = getBins(aggKey, binId - rule.getCondition().getWindowThreshold() + 1, binId); for (Bin data : bins) { windowCount += data.getCount(); } - boolean thresholdBreached = windowCount >= this.config.getThreshold(); + boolean thresholdBreached = windowCount >= rule.getCondition().getMatchCount(); long now = System.currentTimeMillis() / 1000L; long lastNotified = this.notifiedMap.getOrDefault(aggKey, 0L); diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java index 4e312c9884..b30b967d85 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java @@ -67,37 +67,38 @@ public static void registerAllExtensions( "\n builder) { private SmartEvent() { actorId_ = ""; filterId_ = ""; + ruleId_ = ""; } public static final com.google.protobuf.Descriptors.Descriptor @@ -133,6 +134,45 @@ public long getDetectedAt() { return detectedAt_; } + public static final int RULE_ID_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object ruleId_ = ""; + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The ruleId. + */ + @java.lang.Override + public java.lang.String getRuleId() { + java.lang.Object ref = ruleId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ruleId_ = s; + return s; + } + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The bytes for ruleId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getRuleIdBytes() { + java.lang.Object ref = ruleId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ruleId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + private byte memoizedIsInitialized = -1; @java.lang.Override public final boolean isInitialized() { @@ -156,6 +196,9 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (detectedAt_ != 0L) { output.writeInt64(3, detectedAt_); } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ruleId_)) { + com.google.protobuf.GeneratedMessage.writeString(output, 4, ruleId_); + } getUnknownFields().writeTo(output); } @@ -175,6 +218,9 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt64Size(3, detectedAt_); } + if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ruleId_)) { + size += com.google.protobuf.GeneratedMessage.computeStringSize(4, ruleId_); + } size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; @@ -196,6 +242,8 @@ public boolean equals(final java.lang.Object obj) { .equals(other.getFilterId())) return false; if (getDetectedAt() != other.getDetectedAt()) return false; + if (!getRuleId() + .equals(other.getRuleId())) return false; if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -214,6 +262,8 @@ public int hashCode() { hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getDetectedAt()); + hash = (37 * hash) + RULE_ID_FIELD_NUMBER; + hash = (53 * hash) + getRuleId().hashCode(); hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -348,6 +398,7 @@ public Builder clear() { actorId_ = ""; filterId_ = ""; detectedAt_ = 0L; + ruleId_ = ""; return this; } @@ -390,6 +441,9 @@ private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1. if (((from_bitField0_ & 0x00000004) != 0)) { result.detectedAt_ = detectedAt_; } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.ruleId_ = ruleId_; + } } @java.lang.Override @@ -417,6 +471,11 @@ public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.Sm if (other.getDetectedAt() != 0L) { setDetectedAt(other.getDetectedAt()); } + if (!other.getRuleId().isEmpty()) { + ruleId_ = other.ruleId_; + bitField0_ |= 0x00000008; + onChanged(); + } this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; @@ -458,6 +517,11 @@ public Builder mergeFrom( bitField0_ |= 0x00000004; break; } // case 24 + case 34: { + ruleId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 default: { if (!super.parseUnknownField(input, extensionRegistry, tag)) { done = true; // was an endgroup tag @@ -651,6 +715,78 @@ public Builder clearDetectedAt() { return this; } + private java.lang.Object ruleId_ = ""; + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The ruleId. + */ + public java.lang.String getRuleId() { + java.lang.Object ref = ruleId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + ruleId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The bytes for ruleId. + */ + public com.google.protobuf.ByteString + getRuleIdBytes() { + java.lang.Object ref = ruleId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + ruleId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @param value The ruleId to set. + * @return This builder for chaining. + */ + public Builder setRuleId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + ruleId_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return This builder for chaining. + */ + public Builder clearRuleId() { + ruleId_ = getDefaultInstance().getRuleId(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @param value The bytes for ruleId to set. + * @return This builder for chaining. + */ + public Builder setRuleIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + ruleId_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SmartEvent) } diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java index e1e1eab5db..fc243b8af8 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java @@ -38,4 +38,16 @@ public interface SmartEventOrBuilder extends * @return The detectedAt. */ long getDetectedAt(); + + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The ruleId. + */ + java.lang.String getRuleId(); + /** + * string rule_id = 4 [json_name = "ruleId"]; + * @return The bytes for ruleId. + */ + com.google.protobuf.ByteString + getRuleIdBytes(); } diff --git a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto index 304165794d..abfed1d029 100644 --- a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto +++ b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto @@ -17,6 +17,7 @@ message SmartEvent { string actor_id= 1; string filter_id = 2; int64 detected_at = 3; + string rule_id = 4; } message MaliciousEvent { From bf81ce207d809798fe1b8c23a32dc01cf397046b Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 22 Nov 2024 12:04:08 +0530 Subject: [PATCH 21/73] removed db service and using mongo client everywhere --- .../akto/threat/protection/BackendServer.java | 5 +- .../ConsumerMaliciousEventService.java | 45 ++++++- .../java/com/akto/threat/protection/Main.java | 2 +- .../protection/db/CleanupAuditModel.java | 43 ++++++ .../akto/threat/protection/db/DBService.java | 54 -------- .../threat/protection/tasks/CleanupTask.java | 123 ++++++++++++++++++ 6 files changed, 207 insertions(+), 65 deletions(-) create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/CleanupAuditModel.java delete mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/DBService.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java index 4037f50e22..23618660e8 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java @@ -7,7 +7,6 @@ import java.util.concurrent.TimeUnit; import com.akto.DaoInit; -import com.akto.threat.protection.db.DBService; import com.mongodb.ConnectionString; import com.mongodb.ReadPreference; import com.mongodb.client.MongoClient; @@ -23,9 +22,7 @@ public BackendServer(int port) { new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO")), ReadPreference.secondary()); - DBService dbService = new DBService(mongoClient); - - this.server = ServerBuilder.forPort(port).addService(new ConsumerMaliciousEventService(dbService)).build(); + this.server = ServerBuilder.forPort(port).addService(new ConsumerMaliciousEventService(mongoClient)).build(); } public void stop() throws InterruptedException { diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java index 20d5879618..3872a9de7e 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java @@ -1,27 +1,53 @@ package com.akto.threat.protection; +import java.util.ArrayList; +import java.util.List; + import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest; import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse; import com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest; import com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse; -import com.akto.threat.protection.db.DBService; +import com.akto.proto.threat_protection.consumer_service.v1.SmartEvent; +import com.akto.threat.protection.db.MaliciousEventModel; +import com.akto.threat.protection.db.SmartEventModel; +import com.mongodb.client.MongoClient; +import com.mongodb.client.model.BulkWriteOptions; +import com.mongodb.client.model.InsertOneModel; +import com.mongodb.client.model.WriteModel; + import io.grpc.stub.StreamObserver; public class ConsumerMaliciousEventService extends ConsumerServiceGrpc.ConsumerServiceImplBase { - private final DBService dbService; + private MongoClient mongoClient; - public ConsumerMaliciousEventService(DBService dbService) { - this.dbService = dbService; + public ConsumerMaliciousEventService(MongoClient mongoClient) { + this.mongoClient = mongoClient; } @Override public void saveMaliciousEvent( SaveMaliciousEventRequest request, StreamObserver responseObserver) { - this.dbService.saveMaliciousEvents(request.getAccountId() + "", request.getEventsList()); + + List> bulkUpdates = new ArrayList<>(); + request.getEventsList().forEach(event -> { + bulkUpdates.add( + new InsertOneModel<>( + new MaliciousEventModel( + event.getFilterId(), + event.getActorId(), + event.getIp(), + event.getUrl(), + event.getMethod(), + event.getPayload(), + event.getTimestamp()))); + }); + this.mongoClient.getDatabase(request.getAccountId() + "") + .getCollection("malicious_events", MaliciousEventModel.class) + .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); responseObserver.onNext(SaveMaliciousEventResponse.newBuilder().build()); responseObserver.onCompleted(); } @@ -30,7 +56,14 @@ public void saveMaliciousEvent( public void saveSmartEvent( SaveSmartEventRequest request, StreamObserver responseObserver) { - this.dbService.saveSmartEvent(request.getAccountId() + "", request.getEvent()); + SmartEvent event = request.getEvent(); + this.mongoClient.getDatabase(request.getAccountId() + "") + .getCollection("smart_events", SmartEventModel.class) + .insertOne( + new SmartEventModel( + event.getFilterId(), + event.getActorId(), + event.getDetectedAt())); responseObserver.onNext(SaveSmartEventResponse.newBuilder().build()); responseObserver.onCompleted(); } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index 0ea4abda66..2927286301 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -5,7 +5,7 @@ public class Main { public static void main(String[] args) throws Exception { - String mongoURI = System.getenv("AKTO_THREAT_DETECTION_MONGO_CONN"); + String mongoURI = System.getenv("AKTO_MONGO_CONN");; DaoInit.init(new ConnectionString(mongoURI)); int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/CleanupAuditModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/CleanupAuditModel.java new file mode 100644 index 0000000000..2be4e6e7d8 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/CleanupAuditModel.java @@ -0,0 +1,43 @@ +package com.akto.threat.protection.db; + +import org.bson.types.ObjectId; + +public class CleanupAuditModel { + + private String id; + private long alertWindowStart; + private long alertWindowEnd; + + public CleanupAuditModel() {} + + public CleanupAuditModel(long alertWindowStart, long alertWindowEnd) { + this.id = new ObjectId().toString(); + this.alertWindowStart = alertWindowStart; + this.alertWindowEnd = alertWindowEnd; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public long getAlertWindowStart() { + return alertWindowStart; + } + + public void setAlertWindowStart(long alertWindowStart) { + this.alertWindowStart = alertWindowStart; + } + + public long getAlertWindowEnd() { + return alertWindowEnd; + } + + public void setAlertWindowEnd(long alertWindowEnd) { + this.alertWindowEnd = alertWindowEnd; + } +} + diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/DBService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/DBService.java deleted file mode 100644 index 3653882d2a..0000000000 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/DBService.java +++ /dev/null @@ -1,54 +0,0 @@ -package com.akto.threat.protection.db; - -import java.util.ArrayList; -import java.util.List; - -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; -import com.akto.proto.threat_protection.consumer_service.v1.SmartEvent; -import com.mongodb.client.MongoClient; -import com.mongodb.client.model.BulkWriteOptions; -import com.mongodb.client.model.InsertOneModel; -import com.mongodb.client.model.WriteModel; - -public class DBService { - - // We are doing this way instead of using DaoInit.init since we are using - // separate mongo for saving events. - // Current Dao approach doesnt work well with 2 separate mongo connections. - private final MongoClient mongoClient; - - public DBService(MongoClient client) { - this.mongoClient = client; - } - - public void saveMaliciousEvents(String db, Iterable events) { - List> bulkUpdates = new ArrayList<>(); - events.forEach(event -> { - bulkUpdates.add( - new InsertOneModel<>( - new MaliciousEventModel( - event.getFilterId(), - event.getActorId(), - event.getIp(), - event.getUrl(), - event.getMethod(), - event.getPayload(), - event.getTimestamp()))); - }); - this.mongoClient - .getDatabase(db) - .getCollection("malicious_events", MaliciousEventModel.class) - .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); - } - - public void saveSmartEvent(String db, SmartEvent events) { - this.mongoClient - .getDatabase(db) - .getCollection("smart_events", SmartEventModel.class) - .insertOne( - new SmartEventModel( - events.getFilterId(), - events.getActorId(), - events.getDetectedAt())); - } -} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java new file mode 100644 index 0000000000..f72d603d92 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java @@ -0,0 +1,123 @@ +package com.akto.threat.protection.tasks; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +import org.bson.conversions.Bson; + +import com.akto.dao.threat_detection.CleanupAuditDao; +import com.akto.dto.Account; +import com.akto.dto.threat_detection.CleanupAudit; +import com.akto.threat.protection.db.CleanupAuditModel; +import com.akto.threat.protection.db.MaliciousEventModel; +import com.akto.util.AccountTask; +import com.mongodb.BasicDBList; +import com.mongodb.BasicDBObject; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.model.Accumulators; +import com.mongodb.client.model.Aggregates; +import com.mongodb.client.model.Filters; +import com.mongodb.client.model.Projections; + +public class CleanupTask { + private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(2); + + private final MongoClient mongoClient; + + public CleanupTask(MongoClient mongoClient) { + this.mongoClient = mongoClient; + } + + public void init() { + this.executor.scheduleAtFixedRate(this::cleanUp, 0, 5, TimeUnit.HOURS); + } + + public void cleanUp() { + AccountTask.instance.executeTask( + new Consumer() { + @Override + public void accept(Account account) { + cleanUpForAccount(account.getId()); + } + }, + "cleanup-malicious-requests"); + } + + public Optional getLatestEntry(String accountId) { + return Optional.ofNullable(this.mongoClient.getDatabase(accountId + "") + .getCollection("cleanup_malicious_requests_audit", CleanupAuditModel.class) + .find( + new BasicDBObject("sort", new BasicDBObject("alertWindowEnd", -1))) + .first()); + } + + public void cleanUpForAccount(int accountId) { + // Remove all the requests that have passed their expiry. + // AND those requests whose actor and filter that don't have any alerts + // associated with + // them. + long now = System.currentTimeMillis() / 1000L; + + // Get the latest cleanup audit if exists + Optional audit = this.getLatestEntry(accountId + ""); + long start = audit.map(CleanupAuditModel::getAlertWindowEnd).orElse(0L); + + List pipeline = Arrays.asList( + Aggregates.match( + Filters.and( + Filters.gte("detectedAt", start), + Filters.lt("detectedAt", now))), + Aggregates.group( + 0, + Accumulators.addToSet( + "validFilters", + new BasicDBObject("filterId", "$filterId") + .append("actor", "$actor"))), + Aggregates.project( + Projections.fields( + Projections.include("validFilters"), + Projections.excludeId()))); + + try (MongoCursor result = this.mongoClient.getDatabase(accountId + "") + .getCollection("smart_events") + .aggregate(pipeline, BasicDBObject.class) + .cursor()) { + + BasicDBObject validFilters = result.tryNext(); + if (validFilters == null) { + return; + } + + BasicDBList filters = (BasicDBList) validFilters.get("validFilters"); + List filterList = new ArrayList<>(); + for (Object filter : filters) { + BasicDBObject filterObj = (BasicDBObject) filter; + filterList.add( + new BasicDBObject("filterId", filterObj.getString("filterId")) + .append("actor", filterObj.getString("actor"))); + } + + // Remove all the requests that have passed their expiry. + this.mongoClient.getDatabase(accountId + "") + .getCollection("malicious_events", MaliciousEventModel.class) + .deleteMany( + Filters.and( + Filters.lt("expiry", now), + Filters.nor( + filterList.stream() + .map(Filters::and) + .toArray(Bson[]::new)))); + + // TODO: For any given filter, only keep last 1000 requests + + CleanupAuditDao.instance.insertOne(new CleanupAudit(start, now)); + } + } +} \ No newline at end of file From 740413ec6889a9f34842694a40fc90dbb7c978f3 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 22 Nov 2024 14:49:16 +0530 Subject: [PATCH 22/73] moved clean up task to threat protection service --- .../WindowBasedThresholdNotifierTest.java | 158 +++++++++--------- .../akto/threat/protection/BackendServer.java | 10 +- .../java/com/akto/threat/protection/Main.java | 11 +- .../threat/protection/tasks/CleanupTask.java | 6 +- .../dao/threat_detection/CleanupAuditDao.java | 29 ---- .../dto/threat_detection/CleanupAudit.java | 42 ----- 6 files changed, 93 insertions(+), 163 deletions(-) delete mode 100644 libs/dao/src/main/java/com/akto/dao/threat_detection/CleanupAuditDao.java delete mode 100644 libs/dao/src/main/java/com/akto/dto/threat_detection/CleanupAudit.java diff --git a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java index 7b31a3f178..6294b37d1f 100644 --- a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java +++ b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java @@ -1,10 +1,14 @@ package com.akto.filters.aggregators.window_based; import static org.junit.Assert.assertEquals; + import java.util.HashMap; import java.util.Map; + import com.akto.dto.HttpRequestParams; import com.akto.dto.HttpResponseParams; +import com.akto.dto.api_protection_parse_layer.Condition; +import com.akto.dto.api_protection_parse_layer.Rule; import com.akto.dto.monitoring.FilterConfig; import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier.Result; import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; @@ -17,95 +21,91 @@ class MemCache implements CounterCache { - private final ConcurrentHashMap cache; + private final ConcurrentHashMap cache; - public MemCache() { - this.cache = new ConcurrentHashMap<>(); - } + public MemCache() { + this.cache = new ConcurrentHashMap<>(); + } - @Override - public void incrementBy(String key, long val) { - cache.put(key, cache.getOrDefault(key, 0L) + val); - } + @Override + public void incrementBy(String key, long val) { + cache.put(key, cache.getOrDefault(key, 0L) + val); + } - @Override - public void increment(String key) { - incrementBy(key, 1); - } + @Override + public void increment(String key) { + incrementBy(key, 1); + } - @Override - public long get(String key) { - return cache.getOrDefault(key, 0L); - } + @Override + public long get(String key) { + return cache.getOrDefault(key, 0L); + } - @Override - public boolean exists(String key) { - return cache.containsKey(key); - } + @Override + public boolean exists(String key) { + return cache.containsKey(key); + } - public Map internalCache() { - return cache; - } + public Map internalCache() { + return cache; + } } public class WindowBasedThresholdNotifierTest { - private static HttpResponseParams generateResponseParamsForStatusCode(int statusCode) { - return new HttpResponseParams( - "HTTP/1.1", - statusCode, - "Bad Request", - new HashMap<>(), - "{'error': 'Bad Request'}", - new HttpRequestParams( - "POST", - "/api/v1/endpoint", - "HTTP/1.1", - new HashMap<>(), - "{'error': 'Bad Request'}", - 1), - (int) (System.currentTimeMillis() / 1000L), - "100000", - false, - HttpResponseParams.Source.OTHER, - "", - "192.168.0.1"); + private static HttpResponseParams generateResponseParamsForStatusCode(int statusCode) { + return new HttpResponseParams( + "HTTP/1.1", + statusCode, + "Bad Request", + new HashMap<>(), + "{'error': 'Bad Request'}", + new HttpRequestParams( + "POST", "/api/v1/endpoint", "HTTP/1.1", new HashMap<>(), "{'error': 'Bad Request'}", 1), + (int) (System.currentTimeMillis() / 1000L), + "100000", + false, + HttpResponseParams.Source.OTHER, + "", + "192.168.0.1"); + } + + @Test + public void testShouldNotify() throws InterruptedException { + + MemCache cache = new MemCache(); + WindowBasedThresholdNotifier notifier = + new WindowBasedThresholdNotifier(cache, new WindowBasedThresholdNotifier.Config(10, 1)); + + boolean shouldNotify = false; + String ip = "192.168.0.1"; + + FilterConfig filterConfig = new FilterConfig(); + filterConfig.setId("4XX_FILTER"); + + for (int i = 0; i < 1000; i++) { + HttpResponseParams responseParams = generateResponseParamsForStatusCode(400); + Result res = + notifier.shouldNotify( + ip + "|" + "4XX_FILTER", + MaliciousEvent.newBuilder() + .setActorId(ip) + .setIp(ip) + .setTimestamp(responseParams.getTime()) + .setApiCollectionId(responseParams.getRequestParams().getApiCollectionId()) + .setMethod(responseParams.getRequestParams().getMethod()) + .setUrl(responseParams.getRequestParams().getURL()) + .setPayload(responseParams.getOrig()) + .build(), + new Rule("4XX_FILTER", new Condition(10, 1))); + shouldNotify = shouldNotify || res.shouldNotify(); } - @Test - public void testShouldNotify() throws InterruptedException { - - MemCache cache = new MemCache(); - WindowBasedThresholdNotifier notifier = new WindowBasedThresholdNotifier( - cache, new WindowBasedThresholdNotifier.Config(10, 1)); - - boolean shouldNotify = false; - String ip = "192.168.0.1"; - - FilterConfig filterConfig = new FilterConfig(); - filterConfig.setId("4XX_FILTER"); - - for (int i = 0; i < 1000; i++) { - HttpResponseParams responseParams = generateResponseParamsForStatusCode(400); - Result res = notifier.shouldNotify( - ip + "|" + "4XX_FILTER", - MaliciousEvent - .newBuilder() - .setActorId(ip) - .setIp(ip) - .setTimestamp(responseParams.getTime()) - .setApiCollectionId(responseParams.getRequestParams().getApiCollectionId()) - .setMethod(responseParams.getRequestParams().getMethod()) - .setUrl(responseParams.getRequestParams().getURL()) - .setPayload(responseParams.getOrig()) - .build()); - shouldNotify = shouldNotify || res.shouldNotify(); - } - - long count = 0; - for (Map.Entry entry : cache.internalCache().entrySet()) { - count += entry.getValue(); - } - - assertEquals(1000, count); + long count = 0; + for (Map.Entry entry : cache.internalCache().entrySet()) { + count += entry.getValue(); } + + assertEquals(1000, count); + } } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java index 23618660e8..ac5261d8dc 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java @@ -6,22 +6,14 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; -import com.akto.DaoInit; -import com.mongodb.ConnectionString; -import com.mongodb.ReadPreference; import com.mongodb.client.MongoClient; public class BackendServer { private final int port; private final Server server; - public BackendServer(int port) { + public BackendServer(int port, MongoClient mongoClient) { this.port = port; - - MongoClient mongoClient = DaoInit.createMongoClient( - new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO")), - ReadPreference.secondary()); - this.server = ServerBuilder.forPort(port).addService(new ConsumerMaliciousEventService(mongoClient)).build(); } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index 2927286301..fb3a007d72 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -1,15 +1,24 @@ package com.akto.threat.protection; import com.akto.DaoInit; +import com.akto.threat.protection.tasks.CleanupTask; import com.mongodb.ConnectionString; +import com.mongodb.ReadPreference; +import com.mongodb.client.MongoClient; public class Main { public static void main(String[] args) throws Exception { String mongoURI = System.getenv("AKTO_MONGO_CONN");; DaoInit.init(new ConnectionString(mongoURI)); + MongoClient threatProtectionMongo = DaoInit.createMongoClient( + new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO")), + ReadPreference.secondary()); + + new CleanupTask(threatProtectionMongo).init(); + int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); - BackendServer server = new BackendServer(port); + BackendServer server = new BackendServer(port, threatProtectionMongo); server.start(); server.blockUntilShutdown(); } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java index f72d603d92..cea1a3b298 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java @@ -11,9 +11,7 @@ import org.bson.conversions.Bson; -import com.akto.dao.threat_detection.CleanupAuditDao; import com.akto.dto.Account; -import com.akto.dto.threat_detection.CleanupAudit; import com.akto.threat.protection.db.CleanupAuditModel; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.util.AccountTask; @@ -117,7 +115,9 @@ public void cleanUpForAccount(int accountId) { // TODO: For any given filter, only keep last 1000 requests - CleanupAuditDao.instance.insertOne(new CleanupAudit(start, now)); + this.mongoClient.getDatabase(accountId + "") + .getCollection("cleanup_malicious_requests_audit", CleanupAuditModel.class) + .insertOne(new CleanupAuditModel(start, now)); } } } \ No newline at end of file diff --git a/libs/dao/src/main/java/com/akto/dao/threat_detection/CleanupAuditDao.java b/libs/dao/src/main/java/com/akto/dao/threat_detection/CleanupAuditDao.java deleted file mode 100644 index 90e8444480..0000000000 --- a/libs/dao/src/main/java/com/akto/dao/threat_detection/CleanupAuditDao.java +++ /dev/null @@ -1,29 +0,0 @@ -package com.akto.dao.threat_detection; - -import com.akto.dao.AccountsContextDao; -import com.akto.dto.threat_detection.CleanupAudit; -import com.mongodb.BasicDBObject; - -import java.util.Optional; - -public class CleanupAuditDao extends AccountsContextDao { - - public static final CleanupAuditDao instance = new CleanupAuditDao(); - - @Override - public String getCollName() { - return "cleanup_malicious_requests_audit"; - } - - @Override - public Class getClassT() { - return CleanupAudit.class; - } - - public Optional getLatestEntry() { - return Optional.ofNullable( - findOne( - new BasicDBObject(), - new BasicDBObject("sort", new BasicDBObject("alertWindowEnd", -1)))); - } -} diff --git a/libs/dao/src/main/java/com/akto/dto/threat_detection/CleanupAudit.java b/libs/dao/src/main/java/com/akto/dto/threat_detection/CleanupAudit.java deleted file mode 100644 index a705ae6eeb..0000000000 --- a/libs/dao/src/main/java/com/akto/dto/threat_detection/CleanupAudit.java +++ /dev/null @@ -1,42 +0,0 @@ -package com.akto.dto.threat_detection; - -import org.bson.types.ObjectId; - -public class CleanupAudit { - - private String id; - private long alertWindowStart; - private long alertWindowEnd; - - public CleanupAudit() {} - - public CleanupAudit(long alertWindowStart, long alertWindowEnd) { - this.id = new ObjectId().toString(); - this.alertWindowStart = alertWindowStart; - this.alertWindowEnd = alertWindowEnd; - } - - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - - public long getAlertWindowStart() { - return alertWindowStart; - } - - public void setAlertWindowStart(long alertWindowStart) { - this.alertWindowStart = alertWindowStart; - } - - public long getAlertWindowEnd() { - return alertWindowEnd; - } - - public void setAlertWindowEnd(long alertWindowEnd) { - this.alertWindowEnd = alertWindowEnd; - } -} From e19b55e71dbd8912a2eaa5009ea9871d2b5cfcfb Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 22 Nov 2024 17:32:09 +0530 Subject: [PATCH 23/73] added authorization interceptor for grpc server --- .../akto/threat/protection/BackendServer.java | 74 ++++++++------- .../ConsumerMaliciousEventService.java | 91 ++++++++++--------- .../java/com/akto/threat/protection/Main.java | 27 +++--- .../AuthenticationInterceptor.java | 72 +++++++++++++++ .../protection/interceptors/Constants.java | 16 ++++ 5 files changed, 189 insertions(+), 91 deletions(-) create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/AuthenticationInterceptor.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/Constants.java diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java index ac5261d8dc..2fb64553e7 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java @@ -1,5 +1,6 @@ package com.akto.threat.protection; +import com.akto.threat.protection.interceptors.AuthenticationInterceptor; import io.grpc.Server; import io.grpc.ServerBuilder; @@ -9,41 +10,44 @@ import com.mongodb.client.MongoClient; public class BackendServer { - private final int port; - private final Server server; - - public BackendServer(int port, MongoClient mongoClient) { - this.port = port; - this.server = ServerBuilder.forPort(port).addService(new ConsumerMaliciousEventService(mongoClient)).build(); - } - - public void stop() throws InterruptedException { - if (server != null) { - server.shutdown().awaitTermination(30, TimeUnit.SECONDS); - } + private final int port; + private final Server server; + + public BackendServer(int port, MongoClient mongoClient) { + this.port = port; + this.server = + ServerBuilder.forPort(port) + .addService(new ConsumerMaliciousEventService(mongoClient)) + .intercept(new AuthenticationInterceptor()) + .build(); + } + + public void stop() throws InterruptedException { + if (server != null) { + server.shutdown().awaitTermination(30, TimeUnit.SECONDS); } - - public void start() throws IOException { - server.start(); - System.out.println("Server started, listening on " + port); - Runtime.getRuntime() - .addShutdownHook( - new Thread( - () -> { - System.err.println( - "*** shutting down gRPC server since JVM is shutting down"); - try { - BackendServer.this.stop(); - } catch (InterruptedException e) { - e.printStackTrace(System.err); - } - System.err.println("*** server shut down"); - })); - } - - public void blockUntilShutdown() throws InterruptedException { - if (server != null) { - server.awaitTermination(); - } + } + + public void start() throws IOException { + server.start(); + System.out.println("Server started, listening on " + port); + Runtime.getRuntime() + .addShutdownHook( + new Thread( + () -> { + System.err.println("*** shutting down gRPC server since JVM is shutting down"); + try { + BackendServer.this.stop(); + } catch (InterruptedException e) { + e.printStackTrace(System.err); + } + System.err.println("*** server shut down"); + })); + } + + public void blockUntilShutdown() throws InterruptedException { + if (server != null) { + server.awaitTermination(); } + } } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java index 3872a9de7e..781dd3557e 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java @@ -11,6 +11,7 @@ import com.akto.proto.threat_protection.consumer_service.v1.SmartEvent; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.db.SmartEventModel; +import com.akto.threat.protection.interceptors.Constants; import com.mongodb.client.MongoClient; import com.mongodb.client.model.BulkWriteOptions; import com.mongodb.client.model.InsertOneModel; @@ -18,53 +19,55 @@ import io.grpc.stub.StreamObserver; -public class ConsumerMaliciousEventService - extends ConsumerServiceGrpc.ConsumerServiceImplBase { +public class ConsumerMaliciousEventService extends ConsumerServiceGrpc.ConsumerServiceImplBase { - private MongoClient mongoClient; + private final MongoClient mongoClient; - public ConsumerMaliciousEventService(MongoClient mongoClient) { - this.mongoClient = mongoClient; - } + public ConsumerMaliciousEventService(MongoClient mongoClient) { + this.mongoClient = mongoClient; + } - @Override - public void saveMaliciousEvent( - SaveMaliciousEventRequest request, - StreamObserver responseObserver) { + @Override + public void saveMaliciousEvent( + SaveMaliciousEventRequest request, + StreamObserver responseObserver) { - List> bulkUpdates = new ArrayList<>(); - request.getEventsList().forEach(event -> { - bulkUpdates.add( - new InsertOneModel<>( - new MaliciousEventModel( - event.getFilterId(), - event.getActorId(), - event.getIp(), - event.getUrl(), - event.getMethod(), - event.getPayload(), - event.getTimestamp()))); - }); - this.mongoClient.getDatabase(request.getAccountId() + "") - .getCollection("malicious_events", MaliciousEventModel.class) - .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); - responseObserver.onNext(SaveMaliciousEventResponse.newBuilder().build()); - responseObserver.onCompleted(); - } + List> bulkUpdates = new ArrayList<>(); + request + .getEventsList() + .forEach( + event -> { + bulkUpdates.add( + new InsertOneModel<>( + new MaliciousEventModel( + event.getFilterId(), + event.getActorId(), + event.getIp(), + event.getUrl(), + event.getMethod(), + event.getPayload(), + event.getTimestamp()))); + }); + int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); + this.mongoClient + .getDatabase(accountId + "") + .getCollection("malicious_events", MaliciousEventModel.class) + .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); + responseObserver.onNext(SaveMaliciousEventResponse.newBuilder().build()); + responseObserver.onCompleted(); + } - @Override - public void saveSmartEvent( - SaveSmartEventRequest request, - StreamObserver responseObserver) { - SmartEvent event = request.getEvent(); - this.mongoClient.getDatabase(request.getAccountId() + "") - .getCollection("smart_events", SmartEventModel.class) - .insertOne( - new SmartEventModel( - event.getFilterId(), - event.getActorId(), - event.getDetectedAt())); - responseObserver.onNext(SaveSmartEventResponse.newBuilder().build()); - responseObserver.onCompleted(); - } + @Override + public void saveSmartEvent( + SaveSmartEventRequest request, StreamObserver responseObserver) { + SmartEvent event = request.getEvent(); + int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); + this.mongoClient + .getDatabase(accountId + "") + .getCollection("smart_events", SmartEventModel.class) + .insertOne( + new SmartEventModel(event.getFilterId(), event.getActorId(), event.getDetectedAt())); + responseObserver.onNext(SaveSmartEventResponse.newBuilder().build()); + responseObserver.onCompleted(); + } } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index fb3a007d72..9f800c4db3 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -7,19 +7,22 @@ import com.mongodb.client.MongoClient; public class Main { - public static void main(String[] args) throws Exception { - String mongoURI = System.getenv("AKTO_MONGO_CONN");; - DaoInit.init(new ConnectionString(mongoURI)); + public static void main(String[] args) throws Exception { + String mongoURI = System.getenv("AKTO_MONGO_CONN"); - MongoClient threatProtectionMongo = DaoInit.createMongoClient( - new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO")), - ReadPreference.secondary()); + DaoInit.init(new ConnectionString(mongoURI)); - new CleanupTask(threatProtectionMongo).init(); + MongoClient threatProtectionMongo = + DaoInit.createMongoClient( + new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")), + ReadPreference.secondary()); - int port = Integer.parseInt(System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); - BackendServer server = new BackendServer(port, threatProtectionMongo); - server.start(); - server.blockUntilShutdown(); - } + new CleanupTask(threatProtectionMongo).init(); + int port = + Integer.parseInt( + System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); + BackendServer server = new BackendServer(port, threatProtectionMongo); + server.start(); + server.blockUntilShutdown(); + } } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/AuthenticationInterceptor.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/AuthenticationInterceptor.java new file mode 100644 index 0000000000..18c428e654 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/AuthenticationInterceptor.java @@ -0,0 +1,72 @@ +package com.akto.threat.protection.interceptors; + +import com.akto.dao.ConfigsDao; +import com.akto.dto.Config; +import io.grpc.*; +import io.jsonwebtoken.Claims; +import io.jsonwebtoken.Jws; +import io.jsonwebtoken.Jwts; + +import java.io.IOException; +import java.security.KeyFactory; +import java.security.NoSuchAlgorithmException; +import java.security.PublicKey; +import java.security.spec.InvalidKeySpecException; +import java.security.spec.X509EncodedKeySpec; +import java.util.Base64; + +public class AuthenticationInterceptor implements ServerInterceptor { + @Override + public ServerCall.Listener interceptCall( + ServerCall call, Metadata metadata, ServerCallHandler next) { + String value = metadata.get(Constants.AUTHORIZATION_METADATA_KEY); + + if (value == null) { + call.close( + Status.UNAUTHENTICATED.withDescription("Authorization token is required"), metadata); + return null; + } + + try { + PublicKey publicKey = getPublicKey(); + Jws claims = + Jwts.parserBuilder().setSigningKey(publicKey).build().parseClaimsJws(value); + int accountId = (int) claims.getBody().get("accountId"); + Context ctx = Context.current().withValue(Constants.ACCOUNT_ID_CONTEXT_KEY, accountId); + return Contexts.interceptCall(ctx, call, metadata, next); + } catch (Exception e) { + e.printStackTrace(); + call.close(Status.UNAUTHENTICATED.withDescription("Invalid token"), metadata); + } + + return null; + } + + private static PublicKey getPublicKey() + throws NoSuchAlgorithmException, InvalidKeySpecException, IOException { + Config.HybridSaasConfig config; + try { + config = + (Config.HybridSaasConfig) + ConfigsDao.instance.findOne("_id", Config.ConfigType.HYBRID_SAAS.name()); + } catch (Exception e) { + System.out.println(e); + throw e; + } + String rsaPublicKey = config.getPublicKey(); + + rsaPublicKey = rsaPublicKey.replace("-----BEGIN PUBLIC KEY-----", ""); + rsaPublicKey = rsaPublicKey.replace("-----END PUBLIC KEY-----", ""); + rsaPublicKey = rsaPublicKey.replace("\n", ""); + byte[] decoded = Base64.getDecoder().decode(rsaPublicKey); + X509EncodedKeySpec keySpec = new X509EncodedKeySpec(decoded); + KeyFactory kf = KeyFactory.getInstance("RSA"); + + try { + return kf.generatePublic(keySpec); + } catch (Exception e) { + System.out.println(e); + throw e; + } + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/Constants.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/Constants.java new file mode 100644 index 0000000000..33979da98c --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/Constants.java @@ -0,0 +1,16 @@ +package com.akto.threat.protection.interceptors; + +import io.grpc.Context; +import io.grpc.Metadata; + +import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; + +public class Constants { + public static final Metadata.Key AUTHORIZATION_METADATA_KEY = + Metadata.Key.of("Authorization", ASCII_STRING_MARSHALLER); + public static final Context.Key ACCOUNT_ID_CONTEXT_KEY = Context.key("accountId"); + + private Constants() { + throw new AssertionError(); + } +} From 856278883a5d72edfddd810c7ec06d09540a6d77 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 22 Nov 2024 19:41:53 +0530 Subject: [PATCH 24/73] removed account_id field from consume event service requests (threat-detection-backend) Getting account_id from JWT token now --- .../java/com/akto/filters/HttpCallFilter.java | 300 +++++++++--------- .../akto/suspect_data/FlushMessagesTask.java | 154 ++++----- .../v1/ConsumerServiceProto.java | 48 ++- .../v1/SaveMaliciousEventRequest.java | 84 +---- .../SaveMaliciousEventRequestOrBuilder.java | 6 - .../v1/SaveSmartEventRequest.java | 84 +---- .../v1/SaveSmartEventRequestOrBuilder.java | 6 - .../v1/consumer_service.proto | 2 - 8 files changed, 264 insertions(+), 420 deletions(-) diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index 117209df1b..ac8a2c36f0 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -40,175 +40,167 @@ import io.lettuce.core.RedisClient; public class HttpCallFilter { - private static final LoggerMaker loggerMaker = new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); + private static final LoggerMaker loggerMaker = + new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); - private Map apiFilters; - private final HttpCallParser httpCallParser; - private final Kafka kafka; + private Map apiFilters; + private final HttpCallParser httpCallParser; + private final Kafka kafka; - private static final int KAFKA_BATCH_SIZE = 1000; - private static final int KAFKA_BATCH_LINGER_MS = 1000; - private static final String KAFKA_MALICIOUS_TOPIC = "akto.malicious"; + private static final int KAFKA_BATCH_SIZE = 1000; + private static final int KAFKA_BATCH_LINGER_MS = 1000; + private static final String KAFKA_MALICIOUS_TOPIC = "akto.malicious"; - private static final int FILTER_REFRESH_INTERVAL = 10 * 60; - private int lastFilterFetch; + private static final int FILTER_REFRESH_INTERVAL = 10 * 60; + private int lastFilterFetch; - private static final DataActor dataActor = DataActorFactory.fetchInstance(); + private static final DataActor dataActor = DataActorFactory.fetchInstance(); - private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; + private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; - private final ConsumerServiceBlockingStub consumerServiceBlockingStub; + private final ConsumerServiceBlockingStub consumerServiceBlockingStub; - public HttpCallFilter( - RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { - this.apiFilters = new HashMap<>(); - this.lastFilterFetch = 0; - this.httpCallParser = new HttpCallParser(sync_threshold_count, sync_threshold_time); + public HttpCallFilter( + RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { + this.apiFilters = new HashMap<>(); + this.lastFilterFetch = 0; + this.httpCallParser = new HttpCallParser(sync_threshold_count, sync_threshold_time); - String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); - this.kafka = new Kafka(kafkaBrokerUrl, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); - this.windowBasedThresholdNotifier = new WindowBasedThresholdNotifier( - new RedisBackedCounterCache(redisClient, "wbt"), - new WindowBasedThresholdNotifier.Config(100, 10 * 60)); + String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); + this.kafka = new Kafka(kafkaBrokerUrl, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); + this.windowBasedThresholdNotifier = + new WindowBasedThresholdNotifier( + new RedisBackedCounterCache(redisClient, "wbt"), + new WindowBasedThresholdNotifier.Config(100, 10 * 60)); - String target = "localhost:8980"; - ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) - .build(); - this.consumerServiceBlockingStub = ConsumerServiceGrpc.newBlockingStub(channel); - } - - public void filterFunction(List responseParams) { + String target = "localhost:8980"; + ManagedChannel channel = + Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); + this.consumerServiceBlockingStub = ConsumerServiceGrpc.newBlockingStub(channel); + } - int now = Context.now(); - if ((lastFilterFetch + FILTER_REFRESH_INTERVAL) < now) { - // TODO: add support for only active templates. - List templates = dataActor.fetchFilterYamlTemplates(); - apiFilters = FilterYamlTemplateDao.fetchFilterConfig(false, templates, false); - lastFilterFetch = now; - } + public void filterFunction(List responseParams) { - if (apiFilters == null || apiFilters.isEmpty()) { - return; - } - - List maliciousSamples = new ArrayList<>(); - for (HttpResponseParams responseParam : responseParams) { - for (FilterConfig apiFilter : apiFilters.values()) { - boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); - - // If a request passes any of the filter, then it's a malicious request, - // and so we push it to kafka - if (hasPassedFilter) { - // Later we will also add aggregation support - // Eg: 100 4xx requests in last 10 minutes. - // But regardless of whether request falls in aggregation or not, - // we still push malicious requests to kafka - - // todo: modify fetch yaml and read aggregate rules from it - List rules = new ArrayList<>(); - rules.add(new Rule("Lfi Rule 1", new Condition(100, 10))); - AggregationRules aggRules = new AggregationRules(); - - SourceIPKeyGenerator.instance - .generate(responseParam) - .ifPresent( - actor -> { - String groupKey = apiFilter.getId(); - String aggKey = actor + "|" + groupKey; - - MaliciousEvent maliciousEvent = MaliciousEvent.newBuilder().setActorId(actor) - .setFilterId(apiFilter.getId()) - .setUrl(responseParam.getRequestParams().getURL()) - .setMethod(responseParam.getRequestParams().getMethod()) - .setPayload(responseParam.getOrig()) - .setIp(actor) // For now using actor as IP - .setApiCollectionId( - responseParam.getRequestParams().getApiCollectionId()) - .setTimestamp(responseParam.getTime()) - .build(); - - maliciousSamples.add( - new Message( - responseParam.getAccountId(), - maliciousEvent)); - - for (Rule rule: aggRules.getRule()) { - WindowBasedThresholdNotifier.Result result = this.windowBasedThresholdNotifier - .shouldNotify( - aggKey, maliciousEvent, rule); - - if (result.shouldNotify()) { - this.consumerServiceBlockingStub.saveSmartEvent( - SaveSmartEventRequest - .newBuilder() - .setAccountId( - Integer.parseInt(responseParam.getAccountId())) - .setEvent( - SmartEvent.newBuilder() - .setFilterId(apiFilter.getId()) - .setActorId(actor) - .setDetectedAt(responseParam.getTime()) - .setRuleId(actor) - .build()) - .build()); - } - } - }); - - } - } - } + int now = Context.now(); + if ((lastFilterFetch + FILTER_REFRESH_INTERVAL) < now) { + // TODO: add support for only active templates. + List templates = dataActor.fetchFilterYamlTemplates(); + apiFilters = FilterYamlTemplateDao.fetchFilterConfig(false, templates, false); + lastFilterFetch = now; + } - // Should we push all the messages in one go - // or call kafka.send for each HttpRequestParams - try { - maliciousSamples.forEach( - sample -> { - try { - String data = JsonFormat.printer().print(null); - kafka.send(data, KAFKA_MALICIOUS_TOPIC); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); - } - }); - } catch (Exception e) { - e.printStackTrace(); - } + if (apiFilters == null || apiFilters.isEmpty()) { + return; } - private boolean validateFilterForRequest( - HttpResponseParams responseParam, FilterConfig apiFilter) { - try { - String message = responseParam.getOrig(); - RawApi rawApi = RawApi.buildFromMessage(message); - int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); - responseParam.requestParams.setApiCollectionId(apiCollectionId); - String url = responseParam.getRequestParams().getURL(); - Method method = Method.fromString(responseParam.getRequestParams().getMethod()); - ApiInfoKey apiInfoKey = new ApiInfoKey(apiCollectionId, url, method); - Map varMap = apiFilter.resolveVarMap(); - VariableResolver.resolveWordList( - varMap, - new HashMap>() { - { - put(apiInfoKey, Collections.singletonList(message)); - } - }, - apiInfoKey); - String filterExecutionLogId = UUID.randomUUID().toString(); - ValidationResult res = TestPlugin.validateFilter( - apiFilter.getFilter().getNode(), - rawApi, - apiInfoKey, - varMap, - filterExecutionLogId); - - return res.getIsValid(); - } catch (Exception e) { - loggerMaker.errorAndAddToDb( - e, String.format("Error in httpCallFilter %s", e.toString())); + List maliciousSamples = new ArrayList<>(); + for (HttpResponseParams responseParam : responseParams) { + for (FilterConfig apiFilter : apiFilters.values()) { + boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); + + // If a request passes any of the filter, then it's a malicious request, + // and so we push it to kafka + if (hasPassedFilter) { + // Later we will also add aggregation support + // Eg: 100 4xx requests in last 10 minutes. + // But regardless of whether request falls in aggregation or not, + // we still push malicious requests to kafka + + // todo: modify fetch yaml and read aggregate rules from it + List rules = new ArrayList<>(); + rules.add(new Rule("Lfi Rule 1", new Condition(100, 10))); + AggregationRules aggRules = new AggregationRules(); + + SourceIPKeyGenerator.instance + .generate(responseParam) + .ifPresent( + actor -> { + String groupKey = apiFilter.getId(); + String aggKey = actor + "|" + groupKey; + + MaliciousEvent maliciousEvent = + MaliciousEvent.newBuilder() + .setActorId(actor) + .setFilterId(apiFilter.getId()) + .setUrl(responseParam.getRequestParams().getURL()) + .setMethod(responseParam.getRequestParams().getMethod()) + .setPayload(responseParam.getOrig()) + .setIp(actor) // For now using actor as IP + .setApiCollectionId( + responseParam.getRequestParams().getApiCollectionId()) + .setTimestamp(responseParam.getTime()) + .build(); + + maliciousSamples.add(new Message(responseParam.getAccountId(), maliciousEvent)); + + for (Rule rule : aggRules.getRule()) { + WindowBasedThresholdNotifier.Result result = + this.windowBasedThresholdNotifier.shouldNotify( + aggKey, maliciousEvent, rule); + + if (result.shouldNotify()) { + SmartEvent smartEvent = + SmartEvent.newBuilder() + .setFilterId(apiFilter.getId()) + .setActorId(actor) + .setDetectedAt(responseParam.getTime()) + .setRuleId(rule.getName()) + .build(); + this.consumerServiceBlockingStub.saveSmartEvent( + SaveSmartEventRequest.newBuilder().setEvent(smartEvent).build()); + } + } + }); } + } + } - return false; + // Should we push all the messages in one go + // or call kafka.send for each HttpRequestParams + try { + maliciousSamples.forEach( + sample -> { + try { + String data = JsonFormat.printer().print(null); + kafka.send(data, KAFKA_MALICIOUS_TOPIC); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } + }); + } catch (Exception e) { + e.printStackTrace(); } + } + + private boolean validateFilterForRequest( + HttpResponseParams responseParam, FilterConfig apiFilter) { + try { + String message = responseParam.getOrig(); + RawApi rawApi = RawApi.buildFromMessage(message); + int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); + responseParam.requestParams.setApiCollectionId(apiCollectionId); + String url = responseParam.getRequestParams().getURL(); + Method method = Method.fromString(responseParam.getRequestParams().getMethod()); + ApiInfoKey apiInfoKey = new ApiInfoKey(apiCollectionId, url, method); + Map varMap = apiFilter.resolveVarMap(); + VariableResolver.resolveWordList( + varMap, + new HashMap>() { + { + put(apiInfoKey, Collections.singletonList(message)); + } + }, + apiInfoKey); + String filterExecutionLogId = UUID.randomUUID().toString(); + ValidationResult res = + TestPlugin.validateFilter( + apiFilter.getFilter().getNode(), rawApi, apiInfoKey, varMap, filterExecutionLogId); + + return res.getIsValid(); + } catch (Exception e) { + loggerMaker.errorAndAddToDb(e, String.format("Error in httpCallFilter %s", e.toString())); + } + + return false; + } } diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java index 2ff4551c3c..fdec77aef3 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java @@ -33,86 +33,86 @@ public class FlushMessagesTask { - private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); - private final Consumer consumer; - private final ConsumerServiceStub asyncStub; - - private FlushMessagesTask() { - String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); - String groupId = "akto-flush-malicious-messages"; - - Properties properties = Utils.configProperties(kafkaBrokerUrl, groupId, 100); - this.consumer = new KafkaConsumer<>(properties); - - // String target = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); - // TODO: Secure this connection - String target = "localhost:8980"; - ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) - .build(); - - this.asyncStub = ConsumerServiceGrpc.newStub(channel); + private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); + private final Consumer consumer; + private final ConsumerServiceStub asyncStub; + + private FlushMessagesTask() { + String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); + String groupId = "akto-flush-malicious-messages"; + + Properties properties = Utils.configProperties(kafkaBrokerUrl, groupId, 100); + this.consumer = new KafkaConsumer<>(properties); + + String target = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); + // TODO: Secure this connection + ManagedChannel channel = + Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); + + this.asyncStub = ConsumerServiceGrpc.newStub(channel); + } + + public static FlushMessagesTask instance = new FlushMessagesTask(); + + public void init() { + consumer.subscribe(Collections.singletonList("akto.malicious")); + pollingExecutor.execute( + new Runnable() { + @Override + public void run() { + while (true) { + try { + ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); + processRecords(records); + } catch (Exception e) { + e.printStackTrace(); + consumer.close(); + } + } + } + }); + } + + public void processRecords(ConsumerRecords records) { + Map> accWiseMessages = new HashMap<>(); + for (ConsumerRecord record : records) { + try { + MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); + JsonFormat.parser().merge(record.value(), builder); + MaliciousEvent event = builder.build(); + accWiseMessages.computeIfAbsent(record.key(), k -> new ArrayList<>()).add(event); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } } - public static FlushMessagesTask instance = new FlushMessagesTask(); - - public void init() { - consumer.subscribe(Collections.singletonList("akto.malicious")); - pollingExecutor.execute( - new Runnable() { - @Override - public void run() { - while (true) { - try { - ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); - processRecords(records); - } catch (Exception e) { - e.printStackTrace(); - consumer.close(); - } - } - } - }); - } + for (Map.Entry> entry : accWiseMessages.entrySet()) { + int accountId = Integer.parseInt(entry.getKey()); + List events = entry.getValue(); + Context.accountId.set(accountId); + + this.asyncStub.saveMaliciousEvent( + SaveMaliciousEventRequest.newBuilder().addAllEvents(events).build(), + new StreamObserver() { + @Override + public void onNext(SaveMaliciousEventResponse value) { + // Do nothing + } - public void processRecords(ConsumerRecords records) { - Map> accWiseMessages = new HashMap<>(); - for (ConsumerRecord record : records) { - try { - MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); - JsonFormat.parser().merge(record.value(), builder); - MaliciousEvent event = builder.build(); - accWiseMessages.computeIfAbsent(record.key(), k -> new ArrayList<>()).add(event); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); + @Override + public void onError(Throwable t) { + t.printStackTrace(); + } + + @Override + public void onCompleted() { + // Do nothing + System.out.println( + String.format( + "Saved malicious events for account: %d. Saved event counts: %d", + accountId, events.size())); } - } - - for (Map.Entry> entry : accWiseMessages.entrySet()) { - int accountId = Integer.parseInt(entry.getKey()); - List events = entry.getValue(); - Context.accountId.set(accountId); - - this.asyncStub.saveMaliciousEvent( - SaveMaliciousEventRequest.newBuilder().setAccountId(accountId).addAllEvents(events).build(), - new StreamObserver() { - @Override - public void onNext(SaveMaliciousEventResponse value) { - // Do nothing - } - - @Override - public void onError(Throwable t) { - t.printStackTrace(); - } - - @Override - public void onCompleted() { - // Do nothing - System.out.println(String.format( - "Saved malicious events for account: %d. Saved event counts: %d", accountId, - events.size())); - } - }); - } + }); } + } } diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java index b30b967d85..79e87f5a06 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java @@ -76,29 +76,27 @@ public static void registerAllExtensions( "\022\016\n\002ip\030\003 \001(\tR\002ip\022\034\n\ttimestamp\030\004 \001(\003R\ttim" + "estamp\022\020\n\003url\030\005 \001(\tR\003url\022\026\n\006method\030\006 \001(\t" + "R\006method\022*\n\021api_collection_id\030\007 \001(\005R\017api" + - "CollectionId\022\030\n\007payload\030\010 \001(\tR\007payload\"\211" + - "\001\n\031SaveMaliciousEventRequest\022\035\n\naccount_" + - "id\030\001 \001(\005R\taccountId\022M\n\006events\030\002 \003(\01325.th" + - "reat_protection.consumer_service.v1.Mali" + - "ciousEventR\006events\"\177\n\025SaveSmartEventRequ" + - "est\022\035\n\naccount_id\030\001 \001(\005R\taccountId\022G\n\005ev" + - "ent\030\002 \001(\01321.threat_protection.consumer_s" + - "ervice.v1.SmartEventR\005event2\301\002\n\017Consumer" + - "Service\022\233\001\n\022SaveMaliciousEvent\022@.threat_" + - "protection.consumer_service.v1.SaveMalic" + - "iousEventRequest\032A.threat_protection.con" + - "sumer_service.v1.SaveMaliciousEventRespo" + - "nse\"\000\022\217\001\n\016SaveSmartEvent\022<.threat_protec" + - "tion.consumer_service.v1.SaveSmartEventR" + - "equest\032=.threat_protection.consumer_serv" + - "ice.v1.SaveSmartEventResponse\"\000B\372\001\n4com." + - "akto.proto.threat_protection.consumer_se" + - "rvice.v1B\024ConsumerServiceProtoP\001\242\002\003TCX\252\002" + - "#ThreatProtection.ConsumerService.V1\312\002#T" + - "hreatProtection\\ConsumerService\\V1\342\002/Thr" + - "eatProtection\\ConsumerService\\V1\\GPBMeta" + - "data\352\002%ThreatProtection::ConsumerService" + - "::V1b\006proto3" + "CollectionId\022\030\n\007payload\030\010 \001(\tR\007payload\"j" + + "\n\031SaveMaliciousEventRequest\022M\n\006events\030\002 " + + "\003(\01325.threat_protection.consumer_service" + + ".v1.MaliciousEventR\006events\"`\n\025SaveSmartE" + + "ventRequest\022G\n\005event\030\002 \001(\01321.threat_prot" + + "ection.consumer_service.v1.SmartEventR\005e" + + "vent2\301\002\n\017ConsumerService\022\233\001\n\022SaveMalicio" + + "usEvent\022@.threat_protection.consumer_ser" + + "vice.v1.SaveMaliciousEventRequest\032A.thre" + + "at_protection.consumer_service.v1.SaveMa" + + "liciousEventResponse\"\000\022\217\001\n\016SaveSmartEven" + + "t\022<.threat_protection.consumer_service.v" + + "1.SaveSmartEventRequest\032=.threat_protect" + + "ion.consumer_service.v1.SaveSmartEventRe" + + "sponse\"\000B\372\001\n4com.akto.proto.threat_prote" + + "ction.consumer_service.v1B\024ConsumerServi" + + "ceProtoP\001\242\002\003TCX\252\002#ThreatProtection.Consu" + + "merService.V1\312\002#ThreatProtection\\Consume" + + "rService\\V1\342\002/ThreatProtection\\ConsumerS" + + "ervice\\V1\\GPBMetadata\352\002%ThreatProtection" + + "::ConsumerService::V1b\006proto3" }; descriptor = com.google.protobuf.Descriptors.FileDescriptor .internalBuildGeneratedFileFrom(descriptorData, @@ -133,13 +131,13 @@ public static void registerAllExtensions( internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor, - new java.lang.String[] { "AccountId", "Events", }); + new java.lang.String[] { "Events", }); internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor, - new java.lang.String[] { "AccountId", "Event", }); + new java.lang.String[] { "Event", }); descriptor.resolveAllFeaturesImmutable(); } diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java index b5e27a2559..cd6a177be9 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java @@ -43,17 +43,6 @@ private SaveMaliciousEventRequest() { com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.Builder.class); } - public static final int ACCOUNT_ID_FIELD_NUMBER = 1; - private int accountId_ = 0; - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @return The accountId. - */ - @java.lang.Override - public int getAccountId() { - return accountId_; - } - public static final int EVENTS_FIELD_NUMBER = 2; @SuppressWarnings("serial") private java.util.List events_; @@ -109,9 +98,6 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - if (accountId_ != 0) { - output.writeInt32(1, accountId_); - } for (int i = 0; i < events_.size(); i++) { output.writeMessage(2, events_.get(i)); } @@ -124,10 +110,6 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - if (accountId_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(1, accountId_); - } for (int i = 0; i < events_.size(); i++) { size += com.google.protobuf.CodedOutputStream .computeMessageSize(2, events_.get(i)); @@ -147,8 +129,6 @@ public boolean equals(final java.lang.Object obj) { } com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest other = (com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest) obj; - if (getAccountId() - != other.getAccountId()) return false; if (!getEventsList() .equals(other.getEventsList())) return false; if (!getUnknownFields().equals(other.getUnknownFields())) return false; @@ -162,8 +142,6 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACCOUNT_ID_FIELD_NUMBER; - hash = (53 * hash) + getAccountId(); if (getEventsCount() > 0) { hash = (37 * hash) + EVENTS_FIELD_NUMBER; hash = (53 * hash) + getEventsList().hashCode(); @@ -299,14 +277,13 @@ private Builder( public Builder clear() { super.clear(); bitField0_ = 0; - accountId_ = 0; if (eventsBuilder_ == null) { events_ = java.util.Collections.emptyList(); } else { events_ = null; eventsBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000001); return this; } @@ -341,9 +318,9 @@ public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRe private void buildPartialRepeatedFields(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result) { if (eventsBuilder_ == null) { - if (((bitField0_ & 0x00000002) != 0)) { + if (((bitField0_ & 0x00000001) != 0)) { events_ = java.util.Collections.unmodifiableList(events_); - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000001); } result.events_ = events_; } else { @@ -353,9 +330,6 @@ private void buildPartialRepeatedFields(com.akto.proto.threat_protection.consume private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result) { int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.accountId_ = accountId_; - } } @java.lang.Override @@ -370,14 +344,11 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest other) { if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.getDefaultInstance()) return this; - if (other.getAccountId() != 0) { - setAccountId(other.getAccountId()); - } if (eventsBuilder_ == null) { if (!other.events_.isEmpty()) { if (events_.isEmpty()) { events_ = other.events_; - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000001); } else { ensureEventsIsMutable(); events_.addAll(other.events_); @@ -390,7 +361,7 @@ public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.Sa eventsBuilder_.dispose(); eventsBuilder_ = null; events_ = other.events_; - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000001); eventsBuilder_ = com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? getEventsFieldBuilder() : null; @@ -425,11 +396,6 @@ public Builder mergeFrom( case 0: done = true; break; - case 8: { - accountId_ = input.readInt32(); - bitField0_ |= 0x00000001; - break; - } // case 8 case 18: { com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent m = input.readMessage( @@ -460,44 +426,12 @@ public Builder mergeFrom( } private int bitField0_; - private int accountId_ ; - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @return The accountId. - */ - @java.lang.Override - public int getAccountId() { - return accountId_; - } - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @param value The accountId to set. - * @return This builder for chaining. - */ - public Builder setAccountId(int value) { - - accountId_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @return This builder for chaining. - */ - public Builder clearAccountId() { - bitField0_ = (bitField0_ & ~0x00000001); - accountId_ = 0; - onChanged(); - return this; - } - private java.util.List events_ = java.util.Collections.emptyList(); private void ensureEventsIsMutable() { - if (!((bitField0_ & 0x00000002) != 0)) { + if (!((bitField0_ & 0x00000001) != 0)) { events_ = new java.util.ArrayList(events_); - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000001; } } @@ -647,7 +581,7 @@ public Builder addAllEvents( public Builder clearEvents() { if (eventsBuilder_ == null) { events_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000001); onChanged(); } else { eventsBuilder_.clear(); @@ -724,7 +658,7 @@ public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Build eventsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder>( events_, - ((bitField0_ & 0x00000002) != 0), + ((bitField0_ & 0x00000001) != 0), getParentForChildren(), isClean()); events_ = null; diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java index 55c6786ad1..2cbb8407ea 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java @@ -9,12 +9,6 @@ public interface SaveMaliciousEventRequestOrBuilder extends // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) com.google.protobuf.MessageOrBuilder { - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @return The accountId. - */ - int getAccountId(); - /** * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; */ diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java index 48fb08d291..bb4f037719 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java @@ -43,17 +43,6 @@ private SaveSmartEventRequest() { } private int bitField0_; - public static final int ACCOUNT_ID_FIELD_NUMBER = 1; - private int accountId_ = 0; - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @return The accountId. - */ - @java.lang.Override - public int getAccountId() { - return accountId_; - } - public static final int EVENT_FIELD_NUMBER = 2; private com.akto.proto.threat_protection.consumer_service.v1.SmartEvent event_; /** @@ -94,9 +83,6 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - if (accountId_ != 0) { - output.writeInt32(1, accountId_); - } if (((bitField0_ & 0x00000001) != 0)) { output.writeMessage(2, getEvent()); } @@ -109,10 +95,6 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - if (accountId_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(1, accountId_); - } if (((bitField0_ & 0x00000001) != 0)) { size += com.google.protobuf.CodedOutputStream .computeMessageSize(2, getEvent()); @@ -132,8 +114,6 @@ public boolean equals(final java.lang.Object obj) { } com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest other = (com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest) obj; - if (getAccountId() - != other.getAccountId()) return false; if (hasEvent() != other.hasEvent()) return false; if (hasEvent()) { if (!getEvent() @@ -150,8 +130,6 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACCOUNT_ID_FIELD_NUMBER; - hash = (53 * hash) + getAccountId(); if (hasEvent()) { hash = (37 * hash) + EVENT_FIELD_NUMBER; hash = (53 * hash) + getEvent().hashCode(); @@ -293,7 +271,6 @@ private void maybeForceBuilderInitialization() { public Builder clear() { super.clear(); bitField0_ = 0; - accountId_ = 0; event_ = null; if (eventBuilder_ != null) { eventBuilder_.dispose(); @@ -332,11 +309,8 @@ public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventReques private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest result) { int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.accountId_ = accountId_; - } int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000002) != 0)) { + if (((from_bitField0_ & 0x00000001) != 0)) { result.event_ = eventBuilder_ == null ? event_ : eventBuilder_.build(); @@ -357,9 +331,6 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest other) { if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.getDefaultInstance()) return this; - if (other.getAccountId() != 0) { - setAccountId(other.getAccountId()); - } if (other.hasEvent()) { mergeEvent(other.getEvent()); } @@ -389,16 +360,11 @@ public Builder mergeFrom( case 0: done = true; break; - case 8: { - accountId_ = input.readInt32(); - bitField0_ |= 0x00000001; - break; - } // case 8 case 18: { input.readMessage( getEventFieldBuilder().getBuilder(), extensionRegistry); - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000001; break; } // case 18 default: { @@ -418,38 +384,6 @@ public Builder mergeFrom( } private int bitField0_; - private int accountId_ ; - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @return The accountId. - */ - @java.lang.Override - public int getAccountId() { - return accountId_; - } - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @param value The accountId to set. - * @return This builder for chaining. - */ - public Builder setAccountId(int value) { - - accountId_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @return This builder for chaining. - */ - public Builder clearAccountId() { - bitField0_ = (bitField0_ & ~0x00000001); - accountId_ = 0; - onChanged(); - return this; - } - private com.akto.proto.threat_protection.consumer_service.v1.SmartEvent event_; private com.google.protobuf.SingleFieldBuilder< com.akto.proto.threat_protection.consumer_service.v1.SmartEvent, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder> eventBuilder_; @@ -458,7 +392,7 @@ public Builder clearAccountId() { * @return Whether the event field is set. */ public boolean hasEvent() { - return ((bitField0_ & 0x00000002) != 0); + return ((bitField0_ & 0x00000001) != 0); } /** * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; @@ -483,7 +417,7 @@ public Builder setEvent(com.akto.proto.threat_protection.consumer_service.v1.Sma } else { eventBuilder_.setMessage(value); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000001; onChanged(); return this; } @@ -497,7 +431,7 @@ public Builder setEvent( } else { eventBuilder_.setMessage(builderForValue.build()); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000001; onChanged(); return this; } @@ -506,7 +440,7 @@ public Builder setEvent( */ public Builder mergeEvent(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent value) { if (eventBuilder_ == null) { - if (((bitField0_ & 0x00000002) != 0) && + if (((bitField0_ & 0x00000001) != 0) && event_ != null && event_ != com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance()) { getEventBuilder().mergeFrom(value); @@ -517,7 +451,7 @@ public Builder mergeEvent(com.akto.proto.threat_protection.consumer_service.v1.S eventBuilder_.mergeFrom(value); } if (event_ != null) { - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000001; onChanged(); } return this; @@ -526,7 +460,7 @@ public Builder mergeEvent(com.akto.proto.threat_protection.consumer_service.v1.S * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; */ public Builder clearEvent() { - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000001); event_ = null; if (eventBuilder_ != null) { eventBuilder_.dispose(); @@ -539,7 +473,7 @@ public Builder clearEvent() { * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; */ public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder getEventBuilder() { - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000001; onChanged(); return getEventFieldBuilder().getBuilder(); } diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java index 73f2b031ba..5fcd871a80 100644 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java +++ b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java @@ -9,12 +9,6 @@ public interface SaveSmartEventRequestOrBuilder extends // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveSmartEventRequest) com.google.protobuf.MessageOrBuilder { - /** - * int32 account_id = 1 [json_name = "accountId"]; - * @return The accountId. - */ - int getAccountId(); - /** * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; * @return Whether the event field is set. diff --git a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto index abfed1d029..fffa2d49ef 100644 --- a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto +++ b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto @@ -32,12 +32,10 @@ message MaliciousEvent { } message SaveMaliciousEventRequest { - int32 account_id = 1; repeated MaliciousEvent events = 2; } message SaveSmartEventRequest { - int32 account_id = 1; SmartEvent event = 2; } From 3abaa088d7a8ad58c34934f341d5a5666f8c6446 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Sat, 23 Nov 2024 11:22:34 +0530 Subject: [PATCH 25/73] added client side authentication support while saving malicious and smart events --- .../java/com/akto/auth/grpc/AuthToken.java | 35 +++ .../java/com/akto/filters/HttpCallFilter.java | 60 +++-- .../akto/suspect_data/FlushMessagesTask.java | 26 +- .../com/akto/suspect_data/KafkaMessage.java | 56 +++++ .../java/com/akto/suspect_data/Message.java | 33 --- .../java/com/akto/traffic/KafkaRunner.java | 238 +++++++++--------- 6 files changed, 267 insertions(+), 181 deletions(-) create mode 100644 apps/api-threat-detection/src/main/java/com/akto/auth/grpc/AuthToken.java create mode 100644 apps/api-threat-detection/src/main/java/com/akto/suspect_data/KafkaMessage.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java diff --git a/apps/api-threat-detection/src/main/java/com/akto/auth/grpc/AuthToken.java b/apps/api-threat-detection/src/main/java/com/akto/auth/grpc/AuthToken.java new file mode 100644 index 0000000000..8fbe579cfd --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/auth/grpc/AuthToken.java @@ -0,0 +1,35 @@ +package com.akto.auth.grpc; + +import io.grpc.CallCredentials; +import io.grpc.Metadata; +import io.grpc.Status; + +import java.util.concurrent.Executor; + +import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; + +public class AuthToken extends CallCredentials { + + private final String token; + public static final Metadata.Key AUTHORIZATION_METADATA_KEY = + Metadata.Key.of("Authorization", ASCII_STRING_MARSHALLER); + + public AuthToken(String token) { + this.token = token; + } + + @Override + public void applyRequestMetadata( + RequestInfo requestInfo, Executor appExecutor, MetadataApplier applier) { + appExecutor.execute( + () -> { + try { + Metadata headers = new Metadata(); + headers.put(AUTHORIZATION_METADATA_KEY, token); + applier.apply(headers); + } catch (Throwable e) { + applier.fail(Status.UNAUTHENTICATED.withCause(e)); + } + }); + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java index ac8a2c36f0..0d40950feb 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java @@ -2,6 +2,7 @@ import java.util.*; +import com.akto.auth.grpc.AuthToken; import com.akto.cache.RedisBackedCounterCache; import com.akto.dao.context.Context; import com.akto.dao.monitoring.FilterYamlTemplateDao; @@ -22,13 +23,9 @@ import com.akto.kafka.Kafka; import com.akto.log.LoggerMaker; import com.akto.log.LoggerMaker.LogDb; -import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc.ConsumerServiceBlockingStub; -import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; -import com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest; -import com.akto.proto.threat_protection.consumer_service.v1.SmartEvent; +import com.akto.proto.threat_protection.consumer_service.v1.*; import com.akto.rules.TestPlugin; -import com.akto.suspect_data.Message; +import com.akto.suspect_data.KafkaMessage; import com.akto.test_editor.execution.VariableResolver; import com.akto.test_editor.filter.data_operands_impl.ValidationResult; import com.google.protobuf.InvalidProtocolBufferException; @@ -37,6 +34,7 @@ import io.grpc.Grpc; import io.grpc.InsecureChannelCredentials; import io.grpc.ManagedChannel; +import io.grpc.stub.StreamObserver; import io.lettuce.core.RedisClient; public class HttpCallFilter { @@ -58,7 +56,7 @@ public class HttpCallFilter { private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; - private final ConsumerServiceBlockingStub consumerServiceBlockingStub; + private final ConsumerServiceGrpc.ConsumerServiceStub consumerServiceStub; public HttpCallFilter( RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { @@ -76,7 +74,10 @@ public HttpCallFilter( String target = "localhost:8980"; ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - this.consumerServiceBlockingStub = ConsumerServiceGrpc.newBlockingStub(channel); + this.consumerServiceStub = + ConsumerServiceGrpc.newStub(channel) + .withCallCredentials( + new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); } public void filterFunction(List responseParams) { @@ -93,7 +94,7 @@ public void filterFunction(List responseParams) { return; } - List maliciousSamples = new ArrayList<>(); + List maliciousSamples = new ArrayList<>(); for (HttpResponseParams responseParam : responseParams) { for (FilterConfig apiFilter : apiFilters.values()) { boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); @@ -110,6 +111,7 @@ public void filterFunction(List responseParams) { List rules = new ArrayList<>(); rules.add(new Rule("Lfi Rule 1", new Condition(100, 10))); AggregationRules aggRules = new AggregationRules(); + aggRules.setRule(rules); SourceIPKeyGenerator.instance .generate(responseParam) @@ -131,7 +133,13 @@ public void filterFunction(List responseParams) { .setTimestamp(responseParam.getTime()) .build(); - maliciousSamples.add(new Message(responseParam.getAccountId(), maliciousEvent)); + try { + String data = JsonFormat.printer().print(maliciousEvent); + maliciousSamples.add(new KafkaMessage(responseParam.getAccountId(), data)); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + return; + } for (Rule rule : aggRules.getRule()) { WindowBasedThresholdNotifier.Result result = @@ -146,8 +154,24 @@ public void filterFunction(List responseParams) { .setDetectedAt(responseParam.getTime()) .setRuleId(rule.getName()) .build(); - this.consumerServiceBlockingStub.saveSmartEvent( - SaveSmartEventRequest.newBuilder().setEvent(smartEvent).build()); + this.consumerServiceStub.saveSmartEvent( + SaveSmartEventRequest.newBuilder().setEvent(smartEvent).build(), + new StreamObserver() { + @Override + public void onNext(SaveSmartEventResponse value) { + // Do nothing + } + + @Override + public void onError(Throwable t) { + t.printStackTrace(); + } + + @Override + public void onCompleted() { + // Do nothing + } + }); } } }); @@ -160,12 +184,12 @@ public void filterFunction(List responseParams) { try { maliciousSamples.forEach( sample -> { - try { - String data = JsonFormat.printer().print(null); - kafka.send(data, KAFKA_MALICIOUS_TOPIC); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); - } + sample + .marshal() + .ifPresent( + data -> { + kafka.send(data, KAFKA_MALICIOUS_TOPIC); + }); }); } catch (Exception e) { e.printStackTrace(); diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java index fdec77aef3..ade6e44c02 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java @@ -1,15 +1,11 @@ package com.akto.suspect_data; import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; +import java.util.*; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import com.akto.auth.grpc.AuthToken; import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc.ConsumerServiceStub; import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; @@ -21,7 +17,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; -import com.akto.dao.context.Context; import com.akto.runtime.utils.Utils; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; @@ -45,11 +40,13 @@ private FlushMessagesTask() { this.consumer = new KafkaConsumer<>(properties); String target = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); - // TODO: Secure this connection ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - this.asyncStub = ConsumerServiceGrpc.newStub(channel); + this.asyncStub = + ConsumerServiceGrpc.newStub(channel) + .withCallCredentials( + new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); } public static FlushMessagesTask instance = new FlushMessagesTask(); @@ -66,7 +63,6 @@ public void run() { processRecords(records); } catch (Exception e) { e.printStackTrace(); - consumer.close(); } } } @@ -78,9 +74,14 @@ public void processRecords(ConsumerRecords records) { for (ConsumerRecord record : records) { try { MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); - JsonFormat.parser().merge(record.value(), builder); + KafkaMessage m = KafkaMessage.unmarshal(record.value()).orElse(null); + if (m == null) { + continue; + } + + JsonFormat.parser().merge(m.getData(), builder); MaliciousEvent event = builder.build(); - accWiseMessages.computeIfAbsent(record.key(), k -> new ArrayList<>()).add(event); + accWiseMessages.computeIfAbsent(m.getAccountId(), k -> new ArrayList<>()).add(event); } catch (InvalidProtocolBufferException e) { e.printStackTrace(); } @@ -89,7 +90,6 @@ public void processRecords(ConsumerRecords records) { for (Map.Entry> entry : accWiseMessages.entrySet()) { int accountId = Integer.parseInt(entry.getKey()); List events = entry.getValue(); - Context.accountId.set(accountId); this.asyncStub.saveMaliciousEvent( SaveMaliciousEventRequest.newBuilder().addAllEvents(events).build(), diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/KafkaMessage.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/KafkaMessage.java new file mode 100644 index 0000000000..5b371c2495 --- /dev/null +++ b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/KafkaMessage.java @@ -0,0 +1,56 @@ +package com.akto.suspect_data; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.util.Optional; + +// Kafka Message Wrapper for suspect data +public class KafkaMessage { + private String accountId; + private String data; + + private static ObjectMapper objectMapper = new ObjectMapper(); + + public KafkaMessage() {} + + public KafkaMessage(String accountId, String data) { + this.accountId = accountId; + this.data = data; + } + + public String getAccountId() { + return accountId; + } + + public void setAccountId(String accountId) { + this.accountId = accountId; + } + + public String getData() { + return data; + } + + public void setData(String data) { + this.data = data; + } + + public Optional marshal() { + try { + return Optional.ofNullable(objectMapper.writeValueAsString(this)); + } catch (Exception e) { + e.printStackTrace(); + } + + return Optional.empty(); + } + + public static Optional unmarshal(String message) { + try { + return Optional.ofNullable(objectMapper.readValue(message, KafkaMessage.class)); + } catch (Exception e) { + e.printStackTrace(); + } + + return Optional.empty(); + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java deleted file mode 100644 index 334590a39f..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/Message.java +++ /dev/null @@ -1,33 +0,0 @@ -package com.akto.suspect_data; - -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; - -// Kafka Message Wrapper for suspect data -public class Message { - private String accountId; - private MaliciousEvent data; - - public Message() { - } - - public Message(String accountId, MaliciousEvent data) { - this.accountId = accountId; - this.data = data; - } - - public String getAccountId() { - return accountId; - } - - public void setAccountId(String accountId) { - this.accountId = accountId; - } - - public MaliciousEvent getData() { - return data; - } - - public void setData(MaliciousEvent data) { - this.data = data; - } -} diff --git a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java index f31e35a496..571982f9aa 100644 --- a/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java +++ b/libs/utils/src/main/java/com/akto/traffic/KafkaRunner.java @@ -9,6 +9,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import com.akto.DaoInit; +import com.mongodb.ConnectionString; import org.apache.commons.lang3.function.FailableFunction; import org.apache.kafka.clients.consumer.*; import org.apache.kafka.common.Metric; @@ -25,139 +27,141 @@ import com.akto.runtime.utils.Utils; public class KafkaRunner { - private Consumer consumer; - private static final LoggerMaker loggerMaker = new LoggerMaker(KafkaRunner.class, LogDb.RUNTIME); - private static final DataActor dataActor = DataActorFactory.fetchInstance(); - private static final String KAFKA_GROUP_ID = "akto-threat-detection"; - - public static final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2); - - private static Properties generateKafkaProperties() { - String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); - int maxPollRecords = Integer.parseInt( - System.getenv().getOrDefault("AKTO_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); - - return Utils.configProperties(kafkaBrokerUrl, KAFKA_GROUP_ID, maxPollRecords); + private Consumer consumer; + private static final LoggerMaker loggerMaker = new LoggerMaker(KafkaRunner.class, LogDb.RUNTIME); + private static final DataActor dataActor = DataActorFactory.fetchInstance(); + private static final String KAFKA_GROUP_ID = "akto-threat-detection"; + + public static final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2); + + private static Properties generateKafkaProperties() { + String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); + int maxPollRecords = + Integer.parseInt(System.getenv().getOrDefault("AKTO_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); + + return Utils.configProperties(kafkaBrokerUrl, KAFKA_GROUP_ID, maxPollRecords); + } + + public static void consume( + LogDb module, + List topics, + FailableFunction, Void, Exception> recordProcessor) { + + loggerMaker.setDb(module); + + final KafkaRunner main = new KafkaRunner(); + main.consumer = new KafkaConsumer<>(generateKafkaProperties()); + + boolean hybridSaas = RuntimeMode.isHybridDeployment(); + boolean connected = false; + if (hybridSaas) { + AccountSettings accountSettings = dataActor.fetchAccountSettings(); + if (accountSettings != null) { + int acc = accountSettings.getId(); + Context.accountId.set(acc); + connected = true; + } + } else { + String mongoURI = System.getenv("AKTO_MONGO_CONN"); + DaoInit.init(new ConnectionString(mongoURI)); + Context.accountId.set(1_000_000); + connected = true; } - public static void consume(LogDb module, List topics, - FailableFunction, Void, Exception> recordProcessor) { - - loggerMaker.setDb(module); - - final KafkaRunner main = new KafkaRunner(); - main.consumer = new KafkaConsumer<>(generateKafkaProperties()); - - boolean hybridSaas = RuntimeMode.isHybridDeployment(); - boolean connected = false; - if (!hybridSaas) { - throw new RuntimeException("Hybrid mode is required for this module"); - } - - AccountSettings accountSettings = dataActor.fetchAccountSettings(); - if (accountSettings != null) { - int acc = accountSettings.getId(); - Context.accountId.set(acc); - connected = true; - } - - if (connected) { - loggerMaker.infoAndAddToDb(String.format("Starting module for account : %d", Context.accountId.get())); - AllMetrics.instance.init(module); - } + if (connected) { + loggerMaker.infoAndAddToDb( + String.format("Starting module for account : %d", Context.accountId.get())); + AllMetrics.instance.init(module); + } - final Thread mainThread = Thread.currentThread(); - final AtomicBoolean exceptionOnCommitSync = new AtomicBoolean(false); + final Thread mainThread = Thread.currentThread(); + final AtomicBoolean exceptionOnCommitSync = new AtomicBoolean(false); - Runtime.getRuntime().addShutdownHook(new Thread() { - public void run() { + Runtime.getRuntime() + .addShutdownHook( + new Thread() { + public void run() { main.consumer.wakeup(); try { - if (!exceptionOnCommitSync.get()) { - mainThread.join(); - } + if (!exceptionOnCommitSync.get()) { + mainThread.join(); + } } catch (InterruptedException e) { - e.printStackTrace(); + e.printStackTrace(); } catch (Error e) { - loggerMaker.errorAndAddToDb("Error in main thread: " + e.getMessage()); + loggerMaker.errorAndAddToDb("Error in main thread: " + e.getMessage()); } - } - }); - - scheduler.scheduleAtFixedRate(() -> { - main.logKafkaMetrics(module); - }, 0, 1, TimeUnit.MINUTES); - + } + }); + + scheduler.scheduleAtFixedRate( + () -> { + main.logKafkaMetrics(module); + }, + 0, + 1, + TimeUnit.MINUTES); + + try { + main.consumer.subscribe(topics); + loggerMaker.infoAndAddToDb( + String.format("Consumer subscribed for topics : %s", topics.toString())); + while (true) { + ConsumerRecords records = main.consumer.poll(Duration.ofMillis(10000)); try { - main.consumer.subscribe(topics); - loggerMaker.infoAndAddToDb( - String.format("Consumer subscribed for topics : %s", topics.toString())); - while (true) { - ConsumerRecords records = main.consumer.poll(Duration.ofMillis(10000)); - try { - main.consumer.commitSync(); - } catch (Exception e) { - throw e; - } - - try { - recordProcessor.apply(records); - } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, "Error while processing kafka messages " + e); - } - } - } catch (WakeupException ignored) { - // nothing to catch. This exception is called from the shutdown hook. + main.consumer.commitSync(); } catch (Exception e) { - exceptionOnCommitSync.set(true); - Utils.printL(e); - loggerMaker.errorAndAddToDb("Error in Kafka consumer: " + e.getMessage()); - e.printStackTrace(); - System.exit(0); - } finally { - main.consumer.close(); + throw e; } - } - public void logKafkaMetrics(LogDb module) { try { - Map metrics = this.consumer.metrics(); - for (Map.Entry entry : metrics.entrySet()) { - MetricName key = entry.getKey(); - Metric value = entry.getValue(); - - if (key.name().equals("records-lag-max")) { - double val = value.metricValue().equals(Double.NaN) - ? 0d - : (double) value.metricValue(); - AllMetrics.instance.setKafkaRecordsLagMax((float) val); - } - if (key.name().equals("records-consumed-rate")) { - double val = value.metricValue().equals(Double.NaN) - ? 0d - : (double) value.metricValue(); - AllMetrics.instance.setKafkaRecordsConsumedRate((float) val); - } + recordProcessor.apply(records); + } catch (Exception e) { + loggerMaker.errorAndAddToDb(e, "Error while processing kafka messages " + e); + } + } + } catch (WakeupException ignored) { + // nothing to catch. This exception is called from the shutdown hook. + } catch (Exception e) { + exceptionOnCommitSync.set(true); + Utils.printL(e); + loggerMaker.errorAndAddToDb("Error in Kafka consumer: " + e.getMessage()); + e.printStackTrace(); + System.exit(0); + } finally { + main.consumer.close(); + } + } + + public void logKafkaMetrics(LogDb module) { + try { + Map metrics = this.consumer.metrics(); + for (Map.Entry entry : metrics.entrySet()) { + MetricName key = entry.getKey(); + Metric value = entry.getValue(); + + if (key.name().equals("records-lag-max")) { + double val = value.metricValue().equals(Double.NaN) ? 0d : (double) value.metricValue(); + AllMetrics.instance.setKafkaRecordsLagMax((float) val); + } + if (key.name().equals("records-consumed-rate")) { + double val = value.metricValue().equals(Double.NaN) ? 0d : (double) value.metricValue(); + AllMetrics.instance.setKafkaRecordsConsumedRate((float) val); + } - if (key.name().equals("fetch-latency-avg")) { - double val = value.metricValue().equals(Double.NaN) - ? 0d - : (double) value.metricValue(); - AllMetrics.instance.setKafkaFetchAvgLatency((float) val); - } + if (key.name().equals("fetch-latency-avg")) { + double val = value.metricValue().equals(Double.NaN) ? 0d : (double) value.metricValue(); + AllMetrics.instance.setKafkaFetchAvgLatency((float) val); + } - if (key.name().equals("bytes-consumed-rate")) { - double val = value.metricValue().equals(Double.NaN) - ? 0d - : (double) value.metricValue(); - AllMetrics.instance.setKafkaBytesConsumedRate((float) val); - } - } - } catch (Exception e) { - loggerMaker.errorAndAddToDb( - e, - String.format( - "Failed to get kafka metrics for %s error: %s", module.name(), e)); + if (key.name().equals("bytes-consumed-rate")) { + double val = value.metricValue().equals(Double.NaN) ? 0d : (double) value.metricValue(); + AllMetrics.instance.setKafkaBytesConsumedRate((float) val); } + } + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + e, String.format("Failed to get kafka metrics for %s error: %s", module.name(), e)); } + } } From 99a0675489fdb9efeb767efa918da54a9f1f6aca Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Mon, 25 Nov 2024 13:18:21 +0530 Subject: [PATCH 26/73] added new api threat detection removed old api-threat-detection-module --- .gitignore | 1 + .../java/com/akto/cache/CounterCache.java | 12 - .../java/com/akto/cache/LongValueCodec.java | 33 - .../akto/cache/RedisBackedCounterCache.java | 108 -- .../java/com/akto/filters/HttpCallFilter.java | 230 --- .../key_generator/KeyGenerator.java | 13 - .../key_generator/SourceIPKeyGenerator.java | 25 - .../filters/aggregators/window_based/Bin.java | 19 - .../aggregators/window_based/Data.java | 52 - .../WindowBasedThresholdNotifier.java | 108 -- .../akto/suspect_data/FlushMessagesTask.java | 118 -- .../java/com/akto/threat/detection/Main.java | 119 -- .../WindowBasedThresholdNotifierTest.java | 111 -- .../akto/hybrid_parsers/HttpCallParser.java | 1073 +++++++------- apps/pom.xml | 8 +- .../.gitignore | 0 .../Dockerfile | 0 .../README.md | 0 .../pom.xml | 16 +- .../java/com/akto/threat/detection/Main.java | 32 + .../detection/actor/ActorGenerator.java | 10 + .../actor/SourceIPActorGenerator.java | 24 + .../threat/detection/cache/CounterCache.java | 12 + .../detection/cache/LongValueCodec.java | 32 + .../cache/RedisBackedCounterCache.java | 106 ++ .../detection/config/kafka/KafkaConfig.java | 80 + .../config/kafka/KafkaConsumerConfig.java | 44 + .../config/kafka/KafkaProducerConfig.java | 44 + .../dto/MaliciousMessageEnvelope.java} | 14 +- .../threat/detection}/grpc/AuthToken.java | 7 +- .../window_based/Bin.java | 19 + .../window_based/Data.java | 49 + .../WindowBasedThresholdNotifier.java | 107 ++ .../detection/tasks/FlushSampleDataTask.java | 10 + .../tasks/MaliciousTrafficDetectorTask.java | 275 ++++ .../com/akto/threat/detection/tasks/Task.java | 6 + .../src/main/resources/version.txt | 0 .../v1/ConsumerServiceGrpc.java | 367 ----- .../v1/ConsumerServiceProto.java | 145 -- .../consumer_service/v1/MaliciousEvent.java | 1314 ----------------- .../v1/MaliciousEventOrBuilder.java | 95 -- .../v1/SaveMaliciousEventRequest.java | 719 --------- .../SaveMaliciousEventRequestOrBuilder.java | 35 - .../v1/SaveMaliciousEventResponse.java | 358 ----- .../SaveMaliciousEventResponseOrBuilder.java | 11 - .../v1/SaveSmartEventRequest.java | 558 ------- .../v1/SaveSmartEventRequestOrBuilder.java | 26 - .../v1/SaveSmartEventResponse.java | 358 ----- .../v1/SaveSmartEventResponseOrBuilder.java | 11 - .../consumer_service/v1/SmartEvent.java | 840 ----------- .../v1/SmartEventOrBuilder.java | 53 - 51 files changed, 1454 insertions(+), 6353 deletions(-) delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/LongValueCodec.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/KeyGenerator.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Bin.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Data.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java delete mode 100644 apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java delete mode 100644 apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java rename apps/{api-threat-detection => threat-detection}/.gitignore (100%) rename apps/{api-threat-detection => threat-detection}/Dockerfile (100%) rename apps/{api-threat-detection => threat-detection}/README.md (100%) rename apps/{api-threat-detection => threat-detection}/pom.xml (92%) create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/actor/ActorGenerator.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/actor/SourceIPActorGenerator.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/cache/CounterCache.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/cache/LongValueCodec.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConsumerConfig.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaProducerConfig.java rename apps/{api-threat-detection/src/main/java/com/akto/suspect_data/KafkaMessage.java => threat-detection/src/main/java/com/akto/threat/detection/dto/MaliciousMessageEnvelope.java} (70%) rename apps/{api-threat-detection/src/main/java/com/akto/auth => threat-detection/src/main/java/com/akto/threat/detection}/grpc/AuthToken.java (95%) create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/Bin.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/Data.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/Task.java rename apps/{api-threat-detection => threat-detection}/src/main/resources/version.txt (100%) delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceGrpc.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEvent.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEventOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponse.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponseOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponse.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponseOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEvent.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java diff --git a/.gitignore b/.gitignore index 685da2accc..b6e575853c 100644 --- a/.gitignore +++ b/.gitignore @@ -13,6 +13,7 @@ **/.settings **/dist **/gen +libs/protobuf/src/main/java/com/akto/proto libawesome.dylib temp_* *.templates-config.json diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java deleted file mode 100644 index cfc7266312..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/CounterCache.java +++ /dev/null @@ -1,12 +0,0 @@ -package com.akto.cache; - -public interface CounterCache { - - void incrementBy(String key, long val); - - void increment(String key); - - long get(String key); - - boolean exists(String key); -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/LongValueCodec.java b/apps/api-threat-detection/src/main/java/com/akto/cache/LongValueCodec.java deleted file mode 100644 index 2fce242dd5..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/LongValueCodec.java +++ /dev/null @@ -1,33 +0,0 @@ -package com.akto.cache; - -import io.lettuce.core.codec.RedisCodec; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; - -public class LongValueCodec implements RedisCodec { - - @Override - public String decodeKey(ByteBuffer bytes) { - return StandardCharsets.UTF_8.decode(bytes).toString(); - } - - @Override - public Long decodeValue(ByteBuffer bytes) { - if (!bytes.hasRemaining()) - return null; - return bytes.getLong(); - } - - @Override - public ByteBuffer encodeKey(String key) { - return StandardCharsets.UTF_8.encode(key); - } - - @Override - public ByteBuffer encodeValue(Long value) { - ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES); - buffer.putLong(value); - buffer.flip(); - return buffer; - } -} \ No newline at end of file diff --git a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java b/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java deleted file mode 100644 index 1f9a961f1b..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/cache/RedisBackedCounterCache.java +++ /dev/null @@ -1,108 +0,0 @@ -package com.akto.cache; - -import io.lettuce.core.ExpireArgs; -import io.lettuce.core.RedisClient; -import io.lettuce.core.api.StatefulRedisConnection; - -import java.util.Optional; -import java.util.concurrent.*; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; - -public class RedisBackedCounterCache implements CounterCache { - - class Op { - private final String key; - private final long value; - - public Op(String key, long value) { - this.key = key; - this.value = value; - } - - public String getKey() { - return key; - } - - public long getValue() { - return value; - } - } - - private final StatefulRedisConnection redis; - - private final Cache localCache; - - private final ConcurrentLinkedQueue pendingOps; - private final String prefix; - - public RedisBackedCounterCache(RedisClient redisClient, String prefix) { - this.prefix = prefix; - this.redis = redisClient.connect(new LongValueCodec()); - this.localCache = - Caffeine.newBuilder() - .maximumSize(100000) - .expireAfterWrite(3, TimeUnit.HOURS) - .build(); - - ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); - executor.scheduleAtFixedRate(this::syncToRedis, 60, 5, TimeUnit.SECONDS); - - this.pendingOps = new ConcurrentLinkedQueue<>(); - } - - private String getKey(String key) { - return prefix + "|" + key; - } - - @Override - public void increment(String key) { - incrementBy(key, 1); - } - - @Override - public void incrementBy(String key, long val) { - String _key = getKey(key); - localCache.asMap().merge(_key, val, Long::sum); - pendingOps.add(new Op(_key, val)); - - this.setExpiryIfNotSet(_key, 3 * 60 * 60); // added 3 hours expiry for now - } - - @Override - public long get(String key) { - return Optional.ofNullable(this.localCache.getIfPresent(getKey(key))).orElse(0L); - } - - @Override - public boolean exists(String key) { - return localCache.asMap().containsKey(getKey(key)); - } - - private void setExpiryIfNotSet(String key, long seconds) { - // We only set expiry for redis entry. For local cache we have lower expiry for - // all entries. - ExpireArgs args = ExpireArgs.Builder.nx(); - redis.async().expire(getKey(key), seconds, args); - } - - private void syncToRedis() { - while (!pendingOps.isEmpty()) { - Op op = pendingOps.poll(); - String key = op.getKey(); - long val = op.getValue(); - redis.async() - .incrby(key, val) - .whenComplete( - (result, ex) -> { - if (ex != null) { - ex.printStackTrace(); - } - - if (result != null) { - localCache.asMap().put(key, result); - } - }); - } - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java b/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java deleted file mode 100644 index 0d40950feb..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/HttpCallFilter.java +++ /dev/null @@ -1,230 +0,0 @@ -package com.akto.filters; - -import java.util.*; - -import com.akto.auth.grpc.AuthToken; -import com.akto.cache.RedisBackedCounterCache; -import com.akto.dao.context.Context; -import com.akto.dao.monitoring.FilterYamlTemplateDao; -import com.akto.data_actor.DataActor; -import com.akto.data_actor.DataActorFactory; -import com.akto.dto.ApiInfo.ApiInfoKey; -import com.akto.dto.api_protection_parse_layer.AggregationRules; -import com.akto.dto.api_protection_parse_layer.Condition; -import com.akto.dto.api_protection_parse_layer.Rule; -import com.akto.dto.HttpResponseParams; -import com.akto.dto.RawApi; -import com.akto.dto.monitoring.FilterConfig; -import com.akto.dto.test_editor.YamlTemplate; -import com.akto.dto.type.URLMethods.Method; -import com.akto.filters.aggregators.key_generator.SourceIPKeyGenerator; -import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier; -import com.akto.hybrid_parsers.HttpCallParser; -import com.akto.kafka.Kafka; -import com.akto.log.LoggerMaker; -import com.akto.log.LoggerMaker.LogDb; -import com.akto.proto.threat_protection.consumer_service.v1.*; -import com.akto.rules.TestPlugin; -import com.akto.suspect_data.KafkaMessage; -import com.akto.test_editor.execution.VariableResolver; -import com.akto.test_editor.filter.data_operands_impl.ValidationResult; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.util.JsonFormat; - -import io.grpc.Grpc; -import io.grpc.InsecureChannelCredentials; -import io.grpc.ManagedChannel; -import io.grpc.stub.StreamObserver; -import io.lettuce.core.RedisClient; - -public class HttpCallFilter { - private static final LoggerMaker loggerMaker = - new LoggerMaker(HttpCallFilter.class, LogDb.THREAT_DETECTION); - - private Map apiFilters; - private final HttpCallParser httpCallParser; - private final Kafka kafka; - - private static final int KAFKA_BATCH_SIZE = 1000; - private static final int KAFKA_BATCH_LINGER_MS = 1000; - private static final String KAFKA_MALICIOUS_TOPIC = "akto.malicious"; - - private static final int FILTER_REFRESH_INTERVAL = 10 * 60; - private int lastFilterFetch; - - private static final DataActor dataActor = DataActorFactory.fetchInstance(); - - private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; - - private final ConsumerServiceGrpc.ConsumerServiceStub consumerServiceStub; - - public HttpCallFilter( - RedisClient redisClient, int sync_threshold_count, int sync_threshold_time) { - this.apiFilters = new HashMap<>(); - this.lastFilterFetch = 0; - this.httpCallParser = new HttpCallParser(sync_threshold_count, sync_threshold_time); - - String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); - this.kafka = new Kafka(kafkaBrokerUrl, KAFKA_BATCH_LINGER_MS, KAFKA_BATCH_SIZE); - this.windowBasedThresholdNotifier = - new WindowBasedThresholdNotifier( - new RedisBackedCounterCache(redisClient, "wbt"), - new WindowBasedThresholdNotifier.Config(100, 10 * 60)); - - String target = "localhost:8980"; - ManagedChannel channel = - Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - this.consumerServiceStub = - ConsumerServiceGrpc.newStub(channel) - .withCallCredentials( - new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); - } - - public void filterFunction(List responseParams) { - - int now = Context.now(); - if ((lastFilterFetch + FILTER_REFRESH_INTERVAL) < now) { - // TODO: add support for only active templates. - List templates = dataActor.fetchFilterYamlTemplates(); - apiFilters = FilterYamlTemplateDao.fetchFilterConfig(false, templates, false); - lastFilterFetch = now; - } - - if (apiFilters == null || apiFilters.isEmpty()) { - return; - } - - List maliciousSamples = new ArrayList<>(); - for (HttpResponseParams responseParam : responseParams) { - for (FilterConfig apiFilter : apiFilters.values()) { - boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); - - // If a request passes any of the filter, then it's a malicious request, - // and so we push it to kafka - if (hasPassedFilter) { - // Later we will also add aggregation support - // Eg: 100 4xx requests in last 10 minutes. - // But regardless of whether request falls in aggregation or not, - // we still push malicious requests to kafka - - // todo: modify fetch yaml and read aggregate rules from it - List rules = new ArrayList<>(); - rules.add(new Rule("Lfi Rule 1", new Condition(100, 10))); - AggregationRules aggRules = new AggregationRules(); - aggRules.setRule(rules); - - SourceIPKeyGenerator.instance - .generate(responseParam) - .ifPresent( - actor -> { - String groupKey = apiFilter.getId(); - String aggKey = actor + "|" + groupKey; - - MaliciousEvent maliciousEvent = - MaliciousEvent.newBuilder() - .setActorId(actor) - .setFilterId(apiFilter.getId()) - .setUrl(responseParam.getRequestParams().getURL()) - .setMethod(responseParam.getRequestParams().getMethod()) - .setPayload(responseParam.getOrig()) - .setIp(actor) // For now using actor as IP - .setApiCollectionId( - responseParam.getRequestParams().getApiCollectionId()) - .setTimestamp(responseParam.getTime()) - .build(); - - try { - String data = JsonFormat.printer().print(maliciousEvent); - maliciousSamples.add(new KafkaMessage(responseParam.getAccountId(), data)); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); - return; - } - - for (Rule rule : aggRules.getRule()) { - WindowBasedThresholdNotifier.Result result = - this.windowBasedThresholdNotifier.shouldNotify( - aggKey, maliciousEvent, rule); - - if (result.shouldNotify()) { - SmartEvent smartEvent = - SmartEvent.newBuilder() - .setFilterId(apiFilter.getId()) - .setActorId(actor) - .setDetectedAt(responseParam.getTime()) - .setRuleId(rule.getName()) - .build(); - this.consumerServiceStub.saveSmartEvent( - SaveSmartEventRequest.newBuilder().setEvent(smartEvent).build(), - new StreamObserver() { - @Override - public void onNext(SaveSmartEventResponse value) { - // Do nothing - } - - @Override - public void onError(Throwable t) { - t.printStackTrace(); - } - - @Override - public void onCompleted() { - // Do nothing - } - }); - } - } - }); - } - } - } - - // Should we push all the messages in one go - // or call kafka.send for each HttpRequestParams - try { - maliciousSamples.forEach( - sample -> { - sample - .marshal() - .ifPresent( - data -> { - kafka.send(data, KAFKA_MALICIOUS_TOPIC); - }); - }); - } catch (Exception e) { - e.printStackTrace(); - } - } - - private boolean validateFilterForRequest( - HttpResponseParams responseParam, FilterConfig apiFilter) { - try { - String message = responseParam.getOrig(); - RawApi rawApi = RawApi.buildFromMessage(message); - int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); - responseParam.requestParams.setApiCollectionId(apiCollectionId); - String url = responseParam.getRequestParams().getURL(); - Method method = Method.fromString(responseParam.getRequestParams().getMethod()); - ApiInfoKey apiInfoKey = new ApiInfoKey(apiCollectionId, url, method); - Map varMap = apiFilter.resolveVarMap(); - VariableResolver.resolveWordList( - varMap, - new HashMap>() { - { - put(apiInfoKey, Collections.singletonList(message)); - } - }, - apiInfoKey); - String filterExecutionLogId = UUID.randomUUID().toString(); - ValidationResult res = - TestPlugin.validateFilter( - apiFilter.getFilter().getNode(), rawApi, apiInfoKey, varMap, filterExecutionLogId); - - return res.getIsValid(); - } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, String.format("Error in httpCallFilter %s", e.toString())); - } - - return false; - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/KeyGenerator.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/KeyGenerator.java deleted file mode 100644 index 2da78aa11b..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/KeyGenerator.java +++ /dev/null @@ -1,13 +0,0 @@ -package com.akto.filters.aggregators.key_generator; - -import com.akto.dto.HttpResponseParams; -import java.util.Optional; - -public interface KeyGenerator { - - /* - * Get the aggregation key. - * Key can be something like source IP - */ - Optional generate(HttpResponseParams responseParams); -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java deleted file mode 100644 index a70eed510a..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/key_generator/SourceIPKeyGenerator.java +++ /dev/null @@ -1,25 +0,0 @@ -package com.akto.filters.aggregators.key_generator; - -import com.akto.dto.HttpResponseParams; -import com.akto.runtime.policies.ApiAccessTypePolicy; - -import java.util.List; -import java.util.Optional; - -public class SourceIPKeyGenerator implements KeyGenerator { - - private SourceIPKeyGenerator() { - } - - public static SourceIPKeyGenerator instance = new SourceIPKeyGenerator(); - - @Override - public Optional generate(HttpResponseParams responseParams) { - List sourceIPs = ApiAccessTypePolicy.getSourceIps(responseParams); - if (sourceIPs.isEmpty()) { - return Optional.of(responseParams.getSourceIP()); - } - - return Optional.of(sourceIPs.get(0)); - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Bin.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Bin.java deleted file mode 100644 index ac7af5055f..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Bin.java +++ /dev/null @@ -1,19 +0,0 @@ -package com.akto.filters.aggregators.window_based; - -public class Bin { - int binId; - long count; - - public Bin(int binId, long count) { - this.binId = binId; - this.count = count; - } - - public int getBinId() { - return binId; - } - - public long getCount() { - return count; - } -} \ No newline at end of file diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Data.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Data.java deleted file mode 100644 index f93595edd5..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/Data.java +++ /dev/null @@ -1,52 +0,0 @@ -package com.akto.filters.aggregators.window_based; - -import java.util.ArrayList; -import java.util.List; - -import com.fasterxml.jackson.annotation.JsonProperty; - -public class Data { - @JsonProperty("ln") - public long lastNotifiedAt = 0; - - @JsonProperty("rq") - public List requests = new ArrayList<>(); - - public static class Request { - private long receivedAt; - - public Request() { - } - - public Request(long receivedAt) { - this.receivedAt = receivedAt; - } - - public long getReceivedAt() { - return receivedAt; - } - - public void setReceivedAt(long receivedAt) { - this.receivedAt = receivedAt; - } - } - - public Data() { - } - - public long getLastNotifiedAt() { - return lastNotifiedAt; - } - - public void setLastNotifiedAt(long lastNotifiedAt) { - this.lastNotifiedAt = lastNotifiedAt; - } - - public List getRequests() { - return requests; - } - - public void setRequests(List requests) { - this.requests = requests; - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java b/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java deleted file mode 100644 index 9fa8748d2f..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifier.java +++ /dev/null @@ -1,108 +0,0 @@ -package com.akto.filters.aggregators.window_based; - -import java.util.concurrent.ConcurrentMap; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; - -import com.akto.cache.CounterCache; -import com.akto.dto.api_protection_parse_layer.AggregationRules; -import com.akto.dto.api_protection_parse_layer.Rule; -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; - -public class WindowBasedThresholdNotifier { - - private final Config config; - - // We can use an in-memory cache for this, since we dont mind being notified - // more than once by multiple instances of the service. - // But on 1 instance, we should not notify more than once in the cooldown - // period. - // TODO: Move this to redis - private final ConcurrentMap notifiedMap; - - public static class Config { - private final int threshold; - private final int windowSizeInMinutes; - private int notificationCooldownInSeconds = 60 * 30; // 30 mins - - public Config(int threshold, int windowInSeconds) { - this.threshold = threshold; - this.windowSizeInMinutes = windowInSeconds; - } - - public int getThreshold() { - return threshold; - } - - public int getWindowSizeInMinutes() { - return windowSizeInMinutes; - } - - public int getNotificationCooldownInSeconds() { - return notificationCooldownInSeconds; - } - } - - public static class Result { - private final boolean shouldNotify; - - public Result(boolean shouldNotify) { - this.shouldNotify = shouldNotify; - } - - public boolean shouldNotify() { - return shouldNotify; - } - } - - public Config getConfig() { - return config; - } - - private final CounterCache cache; - - public WindowBasedThresholdNotifier(CounterCache cache, Config config) { - this.cache = cache; - this.config = config; - this.notifiedMap = new ConcurrentHashMap<>(); - } - - public Result shouldNotify(String aggKey, MaliciousEvent maliciousEvent, Rule rule) { - int binId = (int) maliciousEvent.getTimestamp() / 60; - String cacheKey = aggKey + "|" + binId; - this.cache.increment(cacheKey); - - long windowCount = 0L; - List bins = getBins(aggKey, binId - rule.getCondition().getWindowThreshold() + 1, binId); - for (Bin data : bins) { - windowCount += data.getCount(); - } - - boolean thresholdBreached = windowCount >= rule.getCondition().getMatchCount(); - - long now = System.currentTimeMillis() / 1000L; - long lastNotified = this.notifiedMap.getOrDefault(aggKey, 0L); - - boolean cooldownBreached = (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); - - if (thresholdBreached && cooldownBreached) { - this.notifiedMap.put(aggKey, now); - return new Result(true); - } - - return new Result(false); - } - - public List getBins(String aggKey, int binStart, int binEnd) { - List binData = new ArrayList<>(); - for (int i = binStart; i <= binEnd; i++) { - String key = aggKey + "|" + i; - if (!this.cache.exists(key)) { - continue; - } - binData.add(new Bin(i, this.cache.get(key))); - } - return binData; - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java b/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java deleted file mode 100644 index ade6e44c02..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/FlushMessagesTask.java +++ /dev/null @@ -1,118 +0,0 @@ -package com.akto.suspect_data; - -import java.time.Duration; -import java.util.*; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -import com.akto.auth.grpc.AuthToken; -import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; -import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc.ConsumerServiceStub; -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; -import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest; -import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; - -import com.akto.runtime.utils.Utils; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.util.JsonFormat; - -import io.grpc.Grpc; -import io.grpc.InsecureChannelCredentials; -import io.grpc.ManagedChannel; -import io.grpc.stub.StreamObserver; - -public class FlushMessagesTask { - - private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); - private final Consumer consumer; - private final ConsumerServiceStub asyncStub; - - private FlushMessagesTask() { - String kafkaBrokerUrl = System.getenv("AKTO_KAFKA_BROKER_URL"); - String groupId = "akto-flush-malicious-messages"; - - Properties properties = Utils.configProperties(kafkaBrokerUrl, groupId, 100); - this.consumer = new KafkaConsumer<>(properties); - - String target = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); - ManagedChannel channel = - Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - - this.asyncStub = - ConsumerServiceGrpc.newStub(channel) - .withCallCredentials( - new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); - } - - public static FlushMessagesTask instance = new FlushMessagesTask(); - - public void init() { - consumer.subscribe(Collections.singletonList("akto.malicious")); - pollingExecutor.execute( - new Runnable() { - @Override - public void run() { - while (true) { - try { - ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); - processRecords(records); - } catch (Exception e) { - e.printStackTrace(); - } - } - } - }); - } - - public void processRecords(ConsumerRecords records) { - Map> accWiseMessages = new HashMap<>(); - for (ConsumerRecord record : records) { - try { - MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); - KafkaMessage m = KafkaMessage.unmarshal(record.value()).orElse(null); - if (m == null) { - continue; - } - - JsonFormat.parser().merge(m.getData(), builder); - MaliciousEvent event = builder.build(); - accWiseMessages.computeIfAbsent(m.getAccountId(), k -> new ArrayList<>()).add(event); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); - } - } - - for (Map.Entry> entry : accWiseMessages.entrySet()) { - int accountId = Integer.parseInt(entry.getKey()); - List events = entry.getValue(); - - this.asyncStub.saveMaliciousEvent( - SaveMaliciousEventRequest.newBuilder().addAllEvents(events).build(), - new StreamObserver() { - @Override - public void onNext(SaveMaliciousEventResponse value) { - // Do nothing - } - - @Override - public void onError(Throwable t) { - t.printStackTrace(); - } - - @Override - public void onCompleted() { - // Do nothing - System.out.println( - String.format( - "Saved malicious events for account: %d. Saved event counts: %d", - accountId, events.size())); - } - }); - } - } -} diff --git a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java deleted file mode 100644 index 84dda8e8ed..0000000000 --- a/apps/api-threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ /dev/null @@ -1,119 +0,0 @@ -package com.akto.threat.detection; - -import java.util.*; - -import com.akto.suspect_data.FlushMessagesTask; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.akto.dao.context.Context; -import com.akto.dto.HttpResponseParams; -import com.akto.log.LoggerMaker; -import com.akto.log.LoggerMaker.LogDb; -import com.akto.metrics.AllMetrics; -import com.akto.runtime.utils.Utils; -import com.akto.traffic.KafkaRunner; -import io.lettuce.core.RedisClient; - -import com.akto.filters.HttpCallFilter; -import com.akto.parsers.HttpCallParser; - -public class Main { - private static final LogDb module = LogDb.THREAT_DETECTION; - private static final LoggerMaker loggerMaker = new LoggerMaker(Main.class, module); - private static final Logger logger = LoggerFactory.getLogger(Main.class); - private static final int sync_threshold_time = 120; - private static final int sync_threshold_count = 1000; - private static long lastSyncOffset = 0; - - private static final Map httpCallFilterMap = new HashMap<>(); - - private static final RedisClient redisClient = createRedisClient(); - - public static void main(String[] args) { - // Flush Messages task - FlushMessagesTask.instance.init(); - - String topicName = System.getenv("AKTO_KAFKA_TOPIC_NAME"); - if (topicName == null) { - String defaultTopic = "akto.api.protection"; - loggerMaker.infoAndAddToDb( - String.format( - "Kafka topic is not defined, using default topic : %s", defaultTopic)); - topicName = defaultTopic; - } - - KafkaRunner.consume( - module, - Collections.singletonList(topicName), - records -> { - processRecords(records); - return null; - }); - } - - public static void processRecords(ConsumerRecords records) { - long start = System.currentTimeMillis(); - - // TODO: what happens if exception - Map> responseParamsToAccountMap = new HashMap<>(); - for (ConsumerRecord r : records) { - HttpResponseParams httpResponseParams; - try { - Utils.printL(r.value()); - AllMetrics.instance.setRuntimeKafkaRecordCount(1); - AllMetrics.instance.setRuntimeKafkaRecordSize(r.value().length()); - lastSyncOffset++; - if (lastSyncOffset % 100 == 0) { - logger.info("Committing offset at position: " + lastSyncOffset); - } - httpResponseParams = HttpCallParser.parseKafkaMessage(r.value()); - } catch (Exception e) { - loggerMaker.errorAndAddToDb( - e, "Error while parsing kafka message " + e, LogDb.RUNTIME); - continue; - } - String accountId = httpResponseParams.getAccountId(); - if (!responseParamsToAccountMap.containsKey(accountId)) { - responseParamsToAccountMap.put(accountId, new ArrayList<>()); - } - responseParamsToAccountMap.get(accountId).add(httpResponseParams); - } - - for (String accountId : responseParamsToAccountMap.keySet()) { - int accountIdInt; - try { - accountIdInt = Integer.parseInt(accountId); - } catch (Exception ignored) { - loggerMaker.errorAndAddToDb("Account id not string", LogDb.RUNTIME); - continue; - } - - Context.accountId.set(accountIdInt); - - if (!httpCallFilterMap.containsKey(accountId)) { - HttpCallFilter filter = new HttpCallFilter(redisClient, sync_threshold_count, sync_threshold_time); - httpCallFilterMap.put(accountId, filter); - loggerMaker.infoAndAddToDb("New filter created for account: " + accountId); - } - - HttpCallFilter filter = httpCallFilterMap.get(accountId); - List accWiseResponse = responseParamsToAccountMap.get(accountId); - filter.filterFunction(accWiseResponse); - } - - AllMetrics.instance.setRuntimeProcessLatency(System.currentTimeMillis() - start); - } - - public static RedisClient createRedisClient() { - String host = System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_HOST", "localhost"); - int port = Integer.parseInt( - System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_PORT", "6379")); - int database = Integer.parseInt( - System.getenv().getOrDefault("AKTO_THREAT_DETECTION_REDIS_DB", "0")); - - return RedisClient.create("redis://" + host + ":" + port + "/" + database); - } -} diff --git a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java b/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java deleted file mode 100644 index 6294b37d1f..0000000000 --- a/apps/api-threat-detection/src/test/java/com/akto/filters/aggregators/window_based/WindowBasedThresholdNotifierTest.java +++ /dev/null @@ -1,111 +0,0 @@ -package com.akto.filters.aggregators.window_based; - -import static org.junit.Assert.assertEquals; - -import java.util.HashMap; -import java.util.Map; - -import com.akto.dto.HttpRequestParams; -import com.akto.dto.HttpResponseParams; -import com.akto.dto.api_protection_parse_layer.Condition; -import com.akto.dto.api_protection_parse_layer.Rule; -import com.akto.dto.monitoring.FilterConfig; -import com.akto.filters.aggregators.window_based.WindowBasedThresholdNotifier.Result; -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; - -import java.util.concurrent.ConcurrentHashMap; - -import org.junit.Test; - -import com.akto.cache.CounterCache; - -class MemCache implements CounterCache { - - private final ConcurrentHashMap cache; - - public MemCache() { - this.cache = new ConcurrentHashMap<>(); - } - - @Override - public void incrementBy(String key, long val) { - cache.put(key, cache.getOrDefault(key, 0L) + val); - } - - @Override - public void increment(String key) { - incrementBy(key, 1); - } - - @Override - public long get(String key) { - return cache.getOrDefault(key, 0L); - } - - @Override - public boolean exists(String key) { - return cache.containsKey(key); - } - - public Map internalCache() { - return cache; - } -} - -public class WindowBasedThresholdNotifierTest { - private static HttpResponseParams generateResponseParamsForStatusCode(int statusCode) { - return new HttpResponseParams( - "HTTP/1.1", - statusCode, - "Bad Request", - new HashMap<>(), - "{'error': 'Bad Request'}", - new HttpRequestParams( - "POST", "/api/v1/endpoint", "HTTP/1.1", new HashMap<>(), "{'error': 'Bad Request'}", 1), - (int) (System.currentTimeMillis() / 1000L), - "100000", - false, - HttpResponseParams.Source.OTHER, - "", - "192.168.0.1"); - } - - @Test - public void testShouldNotify() throws InterruptedException { - - MemCache cache = new MemCache(); - WindowBasedThresholdNotifier notifier = - new WindowBasedThresholdNotifier(cache, new WindowBasedThresholdNotifier.Config(10, 1)); - - boolean shouldNotify = false; - String ip = "192.168.0.1"; - - FilterConfig filterConfig = new FilterConfig(); - filterConfig.setId("4XX_FILTER"); - - for (int i = 0; i < 1000; i++) { - HttpResponseParams responseParams = generateResponseParamsForStatusCode(400); - Result res = - notifier.shouldNotify( - ip + "|" + "4XX_FILTER", - MaliciousEvent.newBuilder() - .setActorId(ip) - .setIp(ip) - .setTimestamp(responseParams.getTime()) - .setApiCollectionId(responseParams.getRequestParams().getApiCollectionId()) - .setMethod(responseParams.getRequestParams().getMethod()) - .setUrl(responseParams.getRequestParams().getURL()) - .setPayload(responseParams.getOrig()) - .build(), - new Rule("4XX_FILTER", new Condition(10, 1))); - shouldNotify = shouldNotify || res.shouldNotify(); - } - - long count = 0; - for (Map.Entry entry : cache.internalCache().entrySet()) { - count += entry.getValue(); - } - - assertEquals(1000, count); - } -} diff --git a/apps/mini-runtime/src/main/java/com/akto/hybrid_parsers/HttpCallParser.java b/apps/mini-runtime/src/main/java/com/akto/hybrid_parsers/HttpCallParser.java index 3ec1d7ea94..ddf0fce03d 100644 --- a/apps/mini-runtime/src/main/java/com/akto/hybrid_parsers/HttpCallParser.java +++ b/apps/mini-runtime/src/main/java/com/akto/hybrid_parsers/HttpCallParser.java @@ -44,556 +44,639 @@ import static com.akto.runtime.RuntimeUtil.matchesDefaultPayload; public class HttpCallParser { - private final int sync_threshold_count; - private final int sync_threshold_time; - private int sync_count = 0; - private int last_synced; - private static final LoggerMaker loggerMaker = new LoggerMaker(HttpCallParser.class); - public APICatalogSync apiCatalogSync; - public DependencyAnalyser dependencyAnalyser; - private Map hostNameToIdMap = new HashMap<>(); - private Map trafficMetricsMap = new HashMap<>(); - public static final ScheduledExecutorService trafficMetricsExecutor = Executors.newScheduledThreadPool(1); - private static final String trafficMetricsUrl = "https://logs.akto.io/traffic-metrics"; - private static final OkHttpClient client = CoreHTTPClient.client.newBuilder() - .writeTimeout(1, TimeUnit.SECONDS) - .readTimeout(1, TimeUnit.SECONDS) - .callTimeout(1, TimeUnit.SECONDS) - .build(); - - private static final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - private DataActor dataActor = DataActorFactory.fetchInstance(); - - public static void init() { - trafficMetricsExecutor.scheduleAtFixedRate(new Runnable() { - @Override - public void run() { - while(!queue.isEmpty()) { - BasicDBObject metrics = queue.poll(); - try { - sendTrafficMetricsToTelemetry(metrics); - } catch (Exception e) { - loggerMaker.errorAndAddToDb("Error while sending traffic_metrics data to prometheus", LogDb.RUNTIME); - } - } + private final int sync_threshold_count; + private final int sync_threshold_time; + private int sync_count = 0; + private int last_synced; + private static final LoggerMaker loggerMaker = new LoggerMaker(HttpCallParser.class); + public APICatalogSync apiCatalogSync; + public DependencyAnalyser dependencyAnalyser; + private Map hostNameToIdMap = new HashMap<>(); + private Map trafficMetricsMap = new HashMap<>(); + public static final ScheduledExecutorService trafficMetricsExecutor = + Executors.newScheduledThreadPool(1); + private static final String trafficMetricsUrl = "https://logs.akto.io/traffic-metrics"; + private static final OkHttpClient client = + CoreHTTPClient.client + .newBuilder() + .writeTimeout(1, TimeUnit.SECONDS) + .readTimeout(1, TimeUnit.SECONDS) + .callTimeout(1, TimeUnit.SECONDS) + .build(); + + private static final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private DataActor dataActor = DataActorFactory.fetchInstance(); + + public static void init() { + trafficMetricsExecutor.scheduleAtFixedRate( + new Runnable() { + @Override + public void run() { + while (!queue.isEmpty()) { + BasicDBObject metrics = queue.poll(); + try { + sendTrafficMetricsToTelemetry(metrics); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + "Error while sending traffic_metrics data to prometheus", LogDb.RUNTIME); + } } - },0,5,TimeUnit.MINUTES); - } - public HttpCallParser(String userIdentifier, int thresh, int sync_threshold_count, int sync_threshold_time, boolean fetchAllSTI) { - last_synced = 0; - this.sync_threshold_count = sync_threshold_count; - this.sync_threshold_time = sync_threshold_time; - apiCatalogSync = new APICatalogSync(userIdentifier, thresh, fetchAllSTI); - apiCatalogSync.buildFromDB(false, fetchAllSTI); - this.dependencyAnalyser = new DependencyAnalyser(apiCatalogSync.dbState, !Main.isOnprem); - } - - public HttpCallParser(int sync_threshold_count, int sync_threshold_time) { - this.sync_threshold_count = sync_threshold_count; - this.sync_threshold_time = sync_threshold_time; - } - - public static HttpResponseParams parseKafkaMessage(String message) throws Exception { - - //convert java object to JSON format - JSONObject jsonObject = JSON.parseObject(message); - - String method = jsonObject.getString("method"); - String url = jsonObject.getString("path"); - String type = jsonObject.getString("type"); - Map> requestHeaders = OriginalHttpRequest.buildHeadersMap(jsonObject, "requestHeaders"); - - String rawRequestPayload = jsonObject.getString("requestPayload"); - String requestPayload = HttpRequestResponseUtils.rawToJsonString(rawRequestPayload,requestHeaders); - - - - String apiCollectionIdStr = jsonObject.getOrDefault("akto_vxlan_id", "0").toString(); - int apiCollectionId = 0; - if (NumberUtils.isDigits(apiCollectionIdStr)) { - apiCollectionId = NumberUtils.toInt(apiCollectionIdStr, 0); - } - - HttpRequestParams requestParams = new HttpRequestParams( - method,url,type, requestHeaders, requestPayload, apiCollectionId - ); - - int statusCode = jsonObject.getInteger("statusCode"); - String status = jsonObject.getString("status"); - Map> responseHeaders = OriginalHttpRequest.buildHeadersMap(jsonObject, "responseHeaders"); - String payload = jsonObject.getString("responsePayload"); - payload = HttpRequestResponseUtils.rawToJsonString(payload, responseHeaders); - payload = JSONUtils.parseIfJsonP(payload); - int time = jsonObject.getInteger("time"); - String accountId = jsonObject.getString("akto_account_id"); - String sourceIP = jsonObject.getString("ip"); - String destIP = jsonObject.getString("destIp"); - String direction = jsonObject.getString("direction"); - - String isPendingStr = (String) jsonObject.getOrDefault("is_pending", "false"); - boolean isPending = !isPendingStr.toLowerCase().equals("false"); - String sourceStr = (String) jsonObject.getOrDefault("source", HttpResponseParams.Source.OTHER.name()); - HttpResponseParams.Source source = HttpResponseParams.Source.valueOf(sourceStr); - - return new HttpResponseParams( - type,statusCode, status, responseHeaders, payload, requestParams, time, accountId, isPending, source, message, sourceIP, destIP, direction - ); + } + }, + 0, + 5, + TimeUnit.MINUTES); + } + + public HttpCallParser( + String userIdentifier, + int thresh, + int sync_threshold_count, + int sync_threshold_time, + boolean fetchAllSTI) { + last_synced = 0; + this.sync_threshold_count = sync_threshold_count; + this.sync_threshold_time = sync_threshold_time; + apiCatalogSync = new APICatalogSync(userIdentifier, thresh, fetchAllSTI); + apiCatalogSync.buildFromDB(false, fetchAllSTI); + this.dependencyAnalyser = new DependencyAnalyser(apiCatalogSync.dbState, !Main.isOnprem); + } + + public HttpCallParser(int sync_threshold_count, int sync_threshold_time) { + this.sync_threshold_count = sync_threshold_count; + this.sync_threshold_time = sync_threshold_time; + } + + public static HttpResponseParams parseKafkaMessage(String message) { + + // convert java object to JSON format + JSONObject jsonObject = JSON.parseObject(message); + + String method = jsonObject.getString("method"); + String url = jsonObject.getString("path"); + String type = jsonObject.getString("type"); + Map> requestHeaders = + OriginalHttpRequest.buildHeadersMap(jsonObject, "requestHeaders"); + + String rawRequestPayload = jsonObject.getString("requestPayload"); + String requestPayload = + HttpRequestResponseUtils.rawToJsonString(rawRequestPayload, requestHeaders); + + String apiCollectionIdStr = jsonObject.getOrDefault("akto_vxlan_id", "0").toString(); + int apiCollectionId = 0; + if (NumberUtils.isDigits(apiCollectionIdStr)) { + apiCollectionId = NumberUtils.toInt(apiCollectionIdStr, 0); } - private static final Gson gson = new Gson(); - - public static String getHeaderValue(Map> headers, String headerKey) { - if (headers == null) return null; - for (String k: headers.keySet()) { - if (k.equalsIgnoreCase(headerKey)) { - List hosts = headers.getOrDefault(k, new ArrayList<>()); - if (hosts.size() > 0) return hosts.get(0); - return null; - } - } + HttpRequestParams requestParams = + new HttpRequestParams(method, url, type, requestHeaders, requestPayload, apiCollectionId); + + int statusCode = jsonObject.getInteger("statusCode"); + String status = jsonObject.getString("status"); + Map> responseHeaders = + OriginalHttpRequest.buildHeadersMap(jsonObject, "responseHeaders"); + String payload = jsonObject.getString("responsePayload"); + payload = HttpRequestResponseUtils.rawToJsonString(payload, responseHeaders); + payload = JSONUtils.parseIfJsonP(payload); + int time = jsonObject.getInteger("time"); + String accountId = jsonObject.getString("akto_account_id"); + String sourceIP = jsonObject.getString("ip"); + String destIP = jsonObject.getString("destIp"); + String direction = jsonObject.getString("direction"); + + String isPendingStr = (String) jsonObject.getOrDefault("is_pending", "false"); + boolean isPending = !isPendingStr.toLowerCase().equals("false"); + String sourceStr = + (String) jsonObject.getOrDefault("source", HttpResponseParams.Source.OTHER.name()); + HttpResponseParams.Source source = HttpResponseParams.Source.valueOf(sourceStr); + + return new HttpResponseParams( + type, + statusCode, + status, + responseHeaders, + payload, + requestParams, + time, + accountId, + isPending, + source, + message, + sourceIP, + destIP, + direction); + } + + private static final Gson gson = new Gson(); + + public static String getHeaderValue(Map> headers, String headerKey) { + if (headers == null) return null; + for (String k : headers.keySet()) { + if (k.equalsIgnoreCase(headerKey)) { + List hosts = headers.getOrDefault(k, new ArrayList<>()); + if (hosts.size() > 0) return hosts.get(0); return null; + } } - - public int createCollectionSimple(int vxlanId) { - dataActor.createCollectionSimple(vxlanId); - return vxlanId; + return null; + } + + public int createCollectionSimple(int vxlanId) { + dataActor.createCollectionSimple(vxlanId); + return vxlanId; + } + + public int createCollectionBasedOnHostName(int id, String host) throws Exception { + FindOneAndUpdateOptions updateOptions = new FindOneAndUpdateOptions(); + updateOptions.upsert(true); + // 3 cases + // 1. If 2 threads are trying to insert same host simultaneously then both will succeed with + // upsert true + // 2. If we are trying to insert different host but same id (hashCode collision) then it will + // fail, + // so we loop 20 times till we succeed + boolean flag = false; + for (int i = 0; i < 100; i++) { + id += i; + try { + dataActor.createCollectionForHost(host, id); + flag = true; + break; + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + "Error while inserting apiCollection, trying again " + i + " " + e.getMessage(), + LogDb.RUNTIME); + } } - - - public int createCollectionBasedOnHostName(int id, String host) throws Exception { - FindOneAndUpdateOptions updateOptions = new FindOneAndUpdateOptions(); - updateOptions.upsert(true); - // 3 cases - // 1. If 2 threads are trying to insert same host simultaneously then both will succeed with upsert true - // 2. If we are trying to insert different host but same id (hashCode collision) then it will fail, - // so we loop 20 times till we succeed - boolean flag = false; - for (int i=0;i < 100; i++) { - id += i; - try { - dataActor.createCollectionForHost(host, id); - flag = true; - break; - } catch (Exception e) { - loggerMaker.errorAndAddToDb("Error while inserting apiCollection, trying again " + i + " " + e.getMessage(), LogDb.RUNTIME); - } - } - if (flag) { // flag tells if we were successfully able to insert collection - loggerMaker.infoAndAddToDb("Using collectionId=" + id + " for " + host, LogDb.RUNTIME); - return id; - } else { - throw new Exception("Not able to insert"); - } + if (flag) { // flag tells if we were successfully able to insert collection + loggerMaker.infoAndAddToDb("Using collectionId=" + id + " for " + host, LogDb.RUNTIME); + return id; + } else { + throw new Exception("Not able to insert"); } - - - int numberOfSyncs = 0; - - public void syncFunction(List responseParams, boolean syncImmediately, boolean fetchAllSTI, AccountSettings accountSettings) { - // USE ONLY filteredResponseParams and not responseParams - List filteredResponseParams = responseParams; - if (accountSettings != null && accountSettings.getDefaultPayloads() != null) { - filteredResponseParams = filterDefaultPayloads(filteredResponseParams, accountSettings.getDefaultPayloads()); - } - filteredResponseParams = filterHttpResponseParams(filteredResponseParams, accountSettings); - boolean isHarOrPcap = aggregate(filteredResponseParams, aggregatorMap); - - for (int apiCollectionId: aggregatorMap.keySet()) { - URLAggregator aggregator = aggregatorMap.get(apiCollectionId); - apiCatalogSync.computeDelta(aggregator, false, apiCollectionId); - } - - // for (HttpResponseParams responseParam: filteredResponseParams) { - // dependencyAnalyser.analyse(responseParam.getOrig(), responseParam.requestParams.getApiCollectionId()); - // } - - this.sync_count += filteredResponseParams.size(); - int syncThresh = numberOfSyncs < 10 ? 10000 : sync_threshold_count; - if (syncImmediately || this.sync_count >= syncThresh || (Context.now() - this.last_synced) > this.sync_threshold_time || isHarOrPcap) { - numberOfSyncs++; - apiCatalogSync.syncWithDB(syncImmediately, fetchAllSTI); - // dependencyAnalyser.dbState = apiCatalogSync.dbState; - // dependencyAnalyser.syncWithDb(); - syncTrafficMetricsWithDB(); - this.last_synced = Context.now(); - this.sync_count = 0; - } - + } + + int numberOfSyncs = 0; + + public void syncFunction( + List responseParams, + boolean syncImmediately, + boolean fetchAllSTI, + AccountSettings accountSettings) { + // USE ONLY filteredResponseParams and not responseParams + List filteredResponseParams = responseParams; + if (accountSettings != null && accountSettings.getDefaultPayloads() != null) { + filteredResponseParams = + filterDefaultPayloads(filteredResponseParams, accountSettings.getDefaultPayloads()); } + filteredResponseParams = filterHttpResponseParams(filteredResponseParams, accountSettings); + boolean isHarOrPcap = aggregate(filteredResponseParams, aggregatorMap); - private List filterDefaultPayloads(List filteredResponseParams, Map defaultPayloadMap) { - List ret = new ArrayList<>(); - for(HttpResponseParams httpResponseParams: filteredResponseParams) { - if (matchesDefaultPayload(httpResponseParams, defaultPayloadMap)) continue; - - ret.add(httpResponseParams); - } - - return ret; + for (int apiCollectionId : aggregatorMap.keySet()) { + URLAggregator aggregator = aggregatorMap.get(apiCollectionId); + apiCatalogSync.computeDelta(aggregator, false, apiCollectionId); } - public void syncTrafficMetricsWithDB() { - loggerMaker.infoAndAddToDb("Starting syncing traffic metrics", LogDb.RUNTIME); - try { - syncTrafficMetricsWithDBHelper(); - } catch (Exception e) { - loggerMaker.errorAndAddToDb("Error while updating traffic metrics: " + e.getMessage(), LogDb.RUNTIME); - } finally { - trafficMetricsMap = new HashMap<>(); - } - loggerMaker.infoAndAddToDb("Finished syncing traffic metrics", LogDb.RUNTIME); + // for (HttpResponseParams responseParam: filteredResponseParams) { + // dependencyAnalyser.analyse(responseParam.getOrig(), + // responseParam.requestParams.getApiCollectionId()); + // } + + this.sync_count += filteredResponseParams.size(); + int syncThresh = numberOfSyncs < 10 ? 10000 : sync_threshold_count; + if (syncImmediately + || this.sync_count >= syncThresh + || (Context.now() - this.last_synced) > this.sync_threshold_time + || isHarOrPcap) { + numberOfSyncs++; + apiCatalogSync.syncWithDB(syncImmediately, fetchAllSTI); + // dependencyAnalyser.dbState = apiCatalogSync.dbState; + // dependencyAnalyser.syncWithDb(); + syncTrafficMetricsWithDB(); + this.last_synced = Context.now(); + this.sync_count = 0; } + } - public void syncTrafficMetricsWithDBHelper() { - List bulkUpdates = new ArrayList<>(); - BasicDBObject metricsData = new BasicDBObject(); - int accountId = Context.accountId.get(); - Organization organization = dataActor.fetchOrganization(accountId); - for (TrafficMetrics trafficMetrics: trafficMetricsMap.values()) { - TrafficMetrics.Key key = trafficMetrics.getId(); - Map countMap = trafficMetrics.getCountMap(); - - if (countMap == null || countMap.isEmpty()) continue; - - Map filterMap = TrafficMetricsDao.getFiltersMap(key); - - int count = 0; - ArrayList individualUpdates = new ArrayList<>(); - for (String ts: countMap.keySet()) { - UpdatePayload updatePayload = new UpdatePayload("countMap." + ts, countMap.get(ts), "inc"); - individualUpdates.add(updatePayload.toString()); - count += countMap.get(ts); - } - - if (organization != null && Main.isOnprem) { - String metricKey = key.getName().name() + "|" + key.getIp() + "|" + key.getHost() + "|" + key.getVxlanID() + "|" + organization.getId() + "|" + accountId; - count += (int) metricsData.getOrDefault(metricKey, 0); - metricsData.put(metricKey, count); - } + private List filterDefaultPayloads( + List filteredResponseParams, + Map defaultPayloadMap) { + List ret = new ArrayList<>(); + for (HttpResponseParams httpResponseParams : filteredResponseParams) { + if (matchesDefaultPayload(httpResponseParams, defaultPayloadMap)) continue; - bulkUpdates.add(new BulkUpdates(filterMap, individualUpdates)); - - } - - if (bulkUpdates.size() > 0) { - if (metricsData.size() != 0 && Main.isOnprem) { - queue.add(metricsData); - } - List writesForTraffic = new ArrayList<>(); - writesForTraffic.addAll(bulkUpdates); - dataActor.bulkWriteTrafficMetrics(writesForTraffic); - } + ret.add(httpResponseParams); } - private static void sendTrafficMetricsToTelemetry(BasicDBObject metricsData) { - MediaType mediaType = MediaType.parse("application/json"); - RequestBody body = RequestBody.create(new BasicDBObject("data", metricsData).toJson(), mediaType); - Request request = new Request.Builder() - .url(trafficMetricsUrl) - .method("POST", body) - .addHeader("Content-Type", "application/json") - .build(); - Response response = null; - try { - response = client.newCall(request).execute(); - } catch (IOException e) { - loggerMaker.errorAndAddToDb("Error while executing request " + request.url() + ": " + e.getMessage(), LogDb.RUNTIME); - } finally { - if (response != null) { - response.close(); - } - } - if (response!= null && response.isSuccessful()) { - loggerMaker.infoAndAddToDb("Updated traffic_metrics", LogDb.RUNTIME); - } else { - loggerMaker.infoAndAddToDb("Traffic_metrics not sent", LogDb.RUNTIME); - } + return ret; + } + + public void syncTrafficMetricsWithDB() { + loggerMaker.infoAndAddToDb("Starting syncing traffic metrics", LogDb.RUNTIME); + try { + syncTrafficMetricsWithDBHelper(); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + "Error while updating traffic metrics: " + e.getMessage(), LogDb.RUNTIME); + } finally { + trafficMetricsMap = new HashMap<>(); } - - public static boolean useHostCondition(String hostName, HttpResponseParams.Source source) { - List whiteListSource = Arrays.asList(HttpResponseParams.Source.MIRRORING); - boolean hostNameCondition; - if (hostName == null) { - hostNameCondition = false; - } else { - hostNameCondition = ! ( hostName.toLowerCase().equals(hostName.toUpperCase()) ); - } - return whiteListSource.contains(source) && hostNameCondition && ApiCollection.useHost; + loggerMaker.infoAndAddToDb("Finished syncing traffic metrics", LogDb.RUNTIME); + } + + public void syncTrafficMetricsWithDBHelper() { + List bulkUpdates = new ArrayList<>(); + BasicDBObject metricsData = new BasicDBObject(); + int accountId = Context.accountId.get(); + Organization organization = dataActor.fetchOrganization(accountId); + for (TrafficMetrics trafficMetrics : trafficMetricsMap.values()) { + TrafficMetrics.Key key = trafficMetrics.getId(); + Map countMap = trafficMetrics.getCountMap(); + + if (countMap == null || countMap.isEmpty()) continue; + + Map filterMap = TrafficMetricsDao.getFiltersMap(key); + + int count = 0; + ArrayList individualUpdates = new ArrayList<>(); + for (String ts : countMap.keySet()) { + UpdatePayload updatePayload = new UpdatePayload("countMap." + ts, countMap.get(ts), "inc"); + individualUpdates.add(updatePayload.toString()); + count += countMap.get(ts); + } + + if (organization != null && Main.isOnprem) { + String metricKey = + key.getName().name() + + "|" + + key.getIp() + + "|" + + key.getHost() + + "|" + + key.getVxlanID() + + "|" + + organization.getId() + + "|" + + accountId; + count += (int) metricsData.getOrDefault(metricKey, 0); + metricsData.put(metricKey, count); + } + + bulkUpdates.add(new BulkUpdates(filterMap, individualUpdates)); } - public static int getBucketStartEpoch() { - return Context.now()/(3600*24); + if (bulkUpdates.size() > 0) { + if (metricsData.size() != 0 && Main.isOnprem) { + queue.add(metricsData); + } + List writesForTraffic = new ArrayList<>(); + writesForTraffic.addAll(bulkUpdates); + dataActor.bulkWriteTrafficMetrics(writesForTraffic); } - - public static int getBucketEndEpoch() { - return Context.now()/(3600*24) + 1; + } + + private static void sendTrafficMetricsToTelemetry(BasicDBObject metricsData) { + MediaType mediaType = MediaType.parse("application/json"); + RequestBody body = + RequestBody.create(new BasicDBObject("data", metricsData).toJson(), mediaType); + Request request = + new Request.Builder() + .url(trafficMetricsUrl) + .method("POST", body) + .addHeader("Content-Type", "application/json") + .build(); + Response response = null; + try { + response = client.newCall(request).execute(); + } catch (IOException e) { + loggerMaker.errorAndAddToDb( + "Error while executing request " + request.url() + ": " + e.getMessage(), LogDb.RUNTIME); + } finally { + if (response != null) { + response.close(); + } } - - public static TrafficMetrics.Key getTrafficMetricsKey(HttpResponseParams httpResponseParam, TrafficMetrics.Name name) { - int bucketStartEpoch = getBucketStartEpoch(); - int bucketEndEpoch = getBucketEndEpoch(); - - String hostName = getHeaderValue(httpResponseParam.getRequestParams().getHeaders(), "host"); - - if (hostName != null && hostName.toLowerCase().equals(hostName.toUpperCase()) ) { - hostName = "ip-host"; - } - - return new TrafficMetrics.Key( - httpResponseParam.getSourceIP(), hostName, httpResponseParam.requestParams.getApiCollectionId(), - name, bucketStartEpoch, bucketEndEpoch - ); + if (response != null && response.isSuccessful()) { + loggerMaker.infoAndAddToDb("Updated traffic_metrics", LogDb.RUNTIME); + } else { + loggerMaker.infoAndAddToDb("Traffic_metrics not sent", LogDb.RUNTIME); } - - public void incTrafficMetrics(TrafficMetrics.Key key, int value) { - if (trafficMetricsMap == null) trafficMetricsMap = new HashMap<>(); - TrafficMetrics trafficMetrics = trafficMetricsMap.get(key); - if (trafficMetrics == null) { - trafficMetrics = new TrafficMetrics(key, new HashMap<>()); - trafficMetricsMap.put(key, trafficMetrics); - } - - trafficMetrics.inc(value); + } + + public static boolean useHostCondition(String hostName, HttpResponseParams.Source source) { + List whiteListSource = + Arrays.asList(HttpResponseParams.Source.MIRRORING); + boolean hostNameCondition; + if (hostName == null) { + hostNameCondition = false; + } else { + hostNameCondition = !(hostName.toLowerCase().equals(hostName.toUpperCase())); } + return whiteListSource.contains(source) && hostNameCondition && ApiCollection.useHost; + } - public int createApiCollectionId(HttpResponseParams httpResponseParam){ - int apiCollectionId; - String hostName = getHeaderValue(httpResponseParam.getRequestParams().getHeaders(), "host"); - - if (hostName != null && !hostNameToIdMap.containsKey(hostName) && RuntimeUtil.hasSpecialCharacters(hostName)) { - hostName = "Special_Char_Host"; - } - - int vxlanId = httpResponseParam.requestParams.getApiCollectionId(); - - if (useHostCondition(hostName, httpResponseParam.getSource())) { - hostName = hostName.toLowerCase(); - hostName = hostName.trim(); - - String key = hostName; - - if (hostNameToIdMap.containsKey(key)) { - apiCollectionId = hostNameToIdMap.get(key); + public static int getBucketStartEpoch() { + return Context.now() / (3600 * 24); + } - } else { - int id = hostName.hashCode(); - try { + public static int getBucketEndEpoch() { + return Context.now() / (3600 * 24) + 1; + } - apiCollectionId = createCollectionBasedOnHostName(id, hostName); + public static TrafficMetrics.Key getTrafficMetricsKey( + HttpResponseParams httpResponseParam, TrafficMetrics.Name name) { + int bucketStartEpoch = getBucketStartEpoch(); + int bucketEndEpoch = getBucketEndEpoch(); - hostNameToIdMap.put(key, apiCollectionId); - } catch (Exception e) { - loggerMaker.errorAndAddToDb("Failed to create collection for host : " + hostName, LogDb.RUNTIME); - createCollectionSimple(vxlanId); - hostNameToIdMap.put("null " + vxlanId, vxlanId); - apiCollectionId = httpResponseParam.requestParams.getApiCollectionId(); - } - } - - } else { - String key = "null" + " " + vxlanId; - if (!hostNameToIdMap.containsKey(key)) { - createCollectionSimple(vxlanId); - hostNameToIdMap.put(key, vxlanId); - } + String hostName = getHeaderValue(httpResponseParam.getRequestParams().getHeaders(), "host"); - apiCollectionId = vxlanId; - } - return apiCollectionId; + if (hostName != null && hostName.toLowerCase().equals(hostName.toUpperCase())) { + hostName = "ip-host"; } - public static final String CONTENT_TYPE = "CONTENT-TYPE"; - - public boolean isRedundantEndpoint(String url, List discardedUrlList){ - StringJoiner joiner = new StringJoiner("|", ".*\\.(", ")(\\?.*)?"); - for (String extension : discardedUrlList) { - if(extension.startsWith(CONTENT_TYPE)){ - continue; - } - joiner.add(extension); - } - String regex = joiner.toString(); - - Pattern pattern = Pattern.compile(regex); - Matcher matcher = pattern.matcher(url); - return matcher.matches(); + return new TrafficMetrics.Key( + httpResponseParam.getSourceIP(), + hostName, + httpResponseParam.requestParams.getApiCollectionId(), + name, + bucketStartEpoch, + bucketEndEpoch); + } + + public void incTrafficMetrics(TrafficMetrics.Key key, int value) { + if (trafficMetricsMap == null) trafficMetricsMap = new HashMap<>(); + TrafficMetrics trafficMetrics = trafficMetricsMap.get(key); + if (trafficMetrics == null) { + trafficMetrics = new TrafficMetrics(key, new HashMap<>()); + trafficMetricsMap.put(key, trafficMetrics); } - private boolean isInvalidContentType(String contentType){ - boolean res = false; - if(contentType == null || contentType.length() == 0) return res; + trafficMetrics.inc(value); + } - res = contentType.contains("javascript") || contentType.contains("png"); - return res; - } + public int createApiCollectionId(HttpResponseParams httpResponseParam) { + int apiCollectionId; + String hostName = getHeaderValue(httpResponseParam.getRequestParams().getHeaders(), "host"); - private boolean isBlankResponseBodyForGET(String method, String contentType, String matchContentType, - String responseBody) { - boolean res = true; - if (contentType == null || contentType.length() == 0) - return false; - res &= contentType.contains(matchContentType); - res &= "GET".equals(method.toUpperCase()); - - /* - * To be sure that the content type - * header matches the actual payload. - * - * We will need to add more type validation as needed. - */ - if (matchContentType.contains("html")) { - res &= responseBody.startsWith("<") && responseBody.endsWith(">"); - } else { - res &= false; - } - return res; + if (hostName != null + && !hostNameToIdMap.containsKey(hostName) + && RuntimeUtil.hasSpecialCharacters(hostName)) { + hostName = "Special_Char_Host"; } - public List filterHttpResponseParams(List httpResponseParamsList, AccountSettings accountSettings) { - List filteredResponseParams = new ArrayList<>(); - int originalSize = httpResponseParamsList.size(); - for (HttpResponseParams httpResponseParam: httpResponseParamsList) { + int vxlanId = httpResponseParam.requestParams.getApiCollectionId(); - if (httpResponseParam.getSource().equals(HttpResponseParams.Source.MIRRORING)) { - TrafficMetrics.Key totalRequestsKey = getTrafficMetricsKey(httpResponseParam, TrafficMetrics.Name.TOTAL_REQUESTS_RUNTIME); - incTrafficMetrics(totalRequestsKey,1); - } + if (useHostCondition(hostName, httpResponseParam.getSource())) { + hostName = hostName.toLowerCase(); + hostName = hostName.trim(); - boolean cond = HttpResponseParams.validHttpResponseCode(httpResponseParam.getStatusCode()); - if (httpResponseParam.getSource().equals(HttpResponseParams.Source.POSTMAN) && httpResponseParam.getStatusCode() <= 0) { - cond = true; - } - - if (!cond) continue; - - String ignoreAktoFlag = getHeaderValue(httpResponseParam.getRequestParams().getHeaders(),Constants.AKTO_IGNORE_FLAG); - if (ignoreAktoFlag != null) continue; - - // check for garbage points here - if(accountSettings != null && accountSettings.getAllowRedundantEndpointsList() != null){ - if(isRedundantEndpoint(httpResponseParam.getRequestParams().getURL(), accountSettings.getAllowRedundantEndpointsList())){ - continue; - } - List contentTypeList = (List) httpResponseParam.getRequestParams().getHeaders().getOrDefault("content-type", new ArrayList<>()); - String contentType = null; - if(!contentTypeList.isEmpty()){ - contentType = contentTypeList.get(0); - } - if(isInvalidContentType(contentType)){ - continue; - } - - try { - List responseContentTypeList = (List) httpResponseParam.getHeaders().getOrDefault("content-type", new ArrayList<>()); - String allContentTypes = responseContentTypeList.toString(); - String method = httpResponseParam.getRequestParams().getMethod(); - String responseBody = httpResponseParam.getPayload(); - boolean ignore = false; - for (String extension : accountSettings.getAllowRedundantEndpointsList()) { - if(extension.startsWith(CONTENT_TYPE)){ - String matchContentType = extension.split(" ")[1]; - if(isBlankResponseBodyForGET(method, allContentTypes, matchContentType, responseBody)){ - ignore = true; - break; - } - } - } - if(ignore){ - continue; - } - - } catch(Exception e){ - loggerMaker.errorAndAddToDb(e, "Error while ignoring content-type redundant samples " + e.toString(), LogDb.RUNTIME); - } + String key = hostName; - } + if (hostNameToIdMap.containsKey(key)) { + apiCollectionId = hostNameToIdMap.get(key); - int apiCollectionId = createApiCollectionId(httpResponseParam); - - httpResponseParam.requestParams.setApiCollectionId(apiCollectionId); + } else { + int id = hostName.hashCode(); + try { - List responseParamsList = GraphQLUtils.getUtils().parseGraphqlResponseParam(httpResponseParam); - if (responseParamsList.isEmpty()) { - filteredResponseParams.add(httpResponseParam); - } else { - filteredResponseParams.addAll(responseParamsList); - loggerMaker.infoAndAddToDb("Adding " + responseParamsList.size() + "new graphql endpoints in invetory",LogDb.RUNTIME); - } - - if (httpResponseParam.getSource().equals(HttpResponseParams.Source.MIRRORING)) { - TrafficMetrics.Key processedRequestsKey = getTrafficMetricsKey(httpResponseParam, TrafficMetrics.Name.FILTERED_REQUESTS_RUNTIME); - incTrafficMetrics(processedRequestsKey,1); - } + apiCollectionId = createCollectionBasedOnHostName(id, hostName); + hostNameToIdMap.put(key, apiCollectionId); + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + "Failed to create collection for host : " + hostName, LogDb.RUNTIME); + createCollectionSimple(vxlanId); + hostNameToIdMap.put("null " + vxlanId, vxlanId); + apiCollectionId = httpResponseParam.requestParams.getApiCollectionId(); } - int filteredSize = filteredResponseParams.size(); - loggerMaker.debugInfoAddToDb("Filtered " + (originalSize - filteredSize) + " responses", LogDb.RUNTIME); - return filteredResponseParams; - } + } - private Map aggregatorMap = new HashMap<>(); + } else { + String key = "null" + " " + vxlanId; + if (!hostNameToIdMap.containsKey(key)) { + createCollectionSimple(vxlanId); + hostNameToIdMap.put(key, vxlanId); + } - public void setAggregatorMap(Map aggregatorMap){ - this.aggregatorMap=aggregatorMap; + apiCollectionId = vxlanId; } - - public Map getAggregatorMap(){ - return this.aggregatorMap; + return apiCollectionId; + } + + public static final String CONTENT_TYPE = "CONTENT-TYPE"; + + public boolean isRedundantEndpoint(String url, List discardedUrlList) { + StringJoiner joiner = new StringJoiner("|", ".*\\.(", ")(\\?.*)?"); + for (String extension : discardedUrlList) { + if (extension.startsWith(CONTENT_TYPE)) { + continue; + } + joiner.add(extension); } + String regex = joiner.toString(); + + Pattern pattern = Pattern.compile(regex); + Matcher matcher = pattern.matcher(url); + return matcher.matches(); + } + + private boolean isInvalidContentType(String contentType) { + boolean res = false; + if (contentType == null || contentType.length() == 0) return res; + + res = contentType.contains("javascript") || contentType.contains("png"); + return res; + } + + private boolean isBlankResponseBodyForGET( + String method, String contentType, String matchContentType, String responseBody) { + boolean res = true; + if (contentType == null || contentType.length() == 0) return false; + res &= contentType.contains(matchContentType); + res &= "GET".equals(method.toUpperCase()); + + /* + * To be sure that the content type + * header matches the actual payload. + * + * We will need to add more type validation as needed. + */ + if (matchContentType.contains("html")) { + res &= responseBody.startsWith("<") && responseBody.endsWith(">"); + } else { + res &= false; + } + return res; + } + + public List filterHttpResponseParams( + List httpResponseParamsList, AccountSettings accountSettings) { + List filteredResponseParams = new ArrayList<>(); + int originalSize = httpResponseParamsList.size(); + for (HttpResponseParams httpResponseParam : httpResponseParamsList) { + + if (httpResponseParam.getSource().equals(HttpResponseParams.Source.MIRRORING)) { + TrafficMetrics.Key totalRequestsKey = + getTrafficMetricsKey(httpResponseParam, TrafficMetrics.Name.TOTAL_REQUESTS_RUNTIME); + incTrafficMetrics(totalRequestsKey, 1); + } + + boolean cond = HttpResponseParams.validHttpResponseCode(httpResponseParam.getStatusCode()); + if (httpResponseParam.getSource().equals(HttpResponseParams.Source.POSTMAN) + && httpResponseParam.getStatusCode() <= 0) { + cond = true; + } + + if (!cond) continue; + + String ignoreAktoFlag = + getHeaderValue( + httpResponseParam.getRequestParams().getHeaders(), Constants.AKTO_IGNORE_FLAG); + if (ignoreAktoFlag != null) continue; + + // check for garbage points here + if (accountSettings != null && accountSettings.getAllowRedundantEndpointsList() != null) { + if (isRedundantEndpoint( + httpResponseParam.getRequestParams().getURL(), + accountSettings.getAllowRedundantEndpointsList())) { + continue; + } + List contentTypeList = + (List) + httpResponseParam + .getRequestParams() + .getHeaders() + .getOrDefault("content-type", new ArrayList<>()); + String contentType = null; + if (!contentTypeList.isEmpty()) { + contentType = contentTypeList.get(0); + } + if (isInvalidContentType(contentType)) { + continue; + } - public static boolean aggregate(List responses, Map aggregatorMap) { - int count = 0; - boolean ret = false; - Set urlSet= new HashSet<>(); - for (HttpResponseParams responseParams: responses) { - if (responseParams.getSource() == HttpResponseParams.Source.HAR || responseParams.getSource() == HttpResponseParams.Source.PCAP) { - ret = true; - } - - HttpRequestParams requestParams = responseParams.requestParams; - if (requestParams != null) { - String path = requestParams.getMethod() + " " + requestParams.url; - if (urlSet.size() < 50) { - urlSet.add(path); - } + try { + List responseContentTypeList = + (List) + httpResponseParam.getHeaders().getOrDefault("content-type", new ArrayList<>()); + String allContentTypes = responseContentTypeList.toString(); + String method = httpResponseParam.getRequestParams().getMethod(); + String responseBody = httpResponseParam.getPayload(); + boolean ignore = false; + for (String extension : accountSettings.getAllowRedundantEndpointsList()) { + if (extension.startsWith(CONTENT_TYPE)) { + String matchContentType = extension.split(" ")[1]; + if (isBlankResponseBodyForGET( + method, allContentTypes, matchContentType, responseBody)) { + ignore = true; + break; + } } + } + if (ignore) { + continue; + } - try { - int collId = responseParams.getRequestParams().getApiCollectionId(); - URLAggregator aggregator = aggregatorMap.get(collId); - if (aggregator == null) { - aggregator = new URLAggregator(); - aggregatorMap.put(collId, aggregator); - } - - aggregator.addURL(responseParams); - count++; - } catch (Exception e) { - } + } catch (Exception e) { + loggerMaker.errorAndAddToDb( + e, + "Error while ignoring content-type redundant samples " + e.toString(), + LogDb.RUNTIME); } - - loggerMaker.debugInfoAddToDb("URLs: " + urlSet.toString(), LogDb.RUNTIME); - loggerMaker.infoAndAddToDb("added " + count + " urls", LogDb.RUNTIME); - return ret; + } + + int apiCollectionId = createApiCollectionId(httpResponseParam); + + httpResponseParam.requestParams.setApiCollectionId(apiCollectionId); + + List responseParamsList = + GraphQLUtils.getUtils().parseGraphqlResponseParam(httpResponseParam); + if (responseParamsList.isEmpty()) { + filteredResponseParams.add(httpResponseParam); + } else { + filteredResponseParams.addAll(responseParamsList); + loggerMaker.infoAndAddToDb( + "Adding " + responseParamsList.size() + "new graphql endpoints in invetory", + LogDb.RUNTIME); + } + + if (httpResponseParam.getSource().equals(HttpResponseParams.Source.MIRRORING)) { + TrafficMetrics.Key processedRequestsKey = + getTrafficMetricsKey(httpResponseParam, TrafficMetrics.Name.FILTERED_REQUESTS_RUNTIME); + incTrafficMetrics(processedRequestsKey, 1); + } } + int filteredSize = filteredResponseParams.size(); + loggerMaker.debugInfoAddToDb( + "Filtered " + (originalSize - filteredSize) + " responses", LogDb.RUNTIME); + return filteredResponseParams; + } + + private Map aggregatorMap = new HashMap<>(); + + public void setAggregatorMap(Map aggregatorMap) { + this.aggregatorMap = aggregatorMap; + } + + public Map getAggregatorMap() { + return this.aggregatorMap; + } + + public static boolean aggregate( + List responses, Map aggregatorMap) { + int count = 0; + boolean ret = false; + Set urlSet = new HashSet<>(); + for (HttpResponseParams responseParams : responses) { + if (responseParams.getSource() == HttpResponseParams.Source.HAR + || responseParams.getSource() == HttpResponseParams.Source.PCAP) { + ret = true; + } + + HttpRequestParams requestParams = responseParams.requestParams; + if (requestParams != null) { + String path = requestParams.getMethod() + " " + requestParams.url; + if (urlSet.size() < 50) { + urlSet.add(path); + } + } + + try { + int collId = responseParams.getRequestParams().getApiCollectionId(); + URLAggregator aggregator = aggregatorMap.get(collId); + if (aggregator == null) { + aggregator = new URLAggregator(); + aggregatorMap.put(collId, aggregator); + } - public int getLastSyncTime() { - return this.last_synced; + aggregator.addURL(responseParams); + count++; + } catch (Exception e) { + } } - public int getSyncCount() { - return this.sync_count; - } + loggerMaker.debugInfoAddToDb("URLs: " + urlSet.toString(), LogDb.RUNTIME); + loggerMaker.infoAndAddToDb("added " + count + " urls", LogDb.RUNTIME); + return ret; + } - public Map getHostNameToIdMap() { - return hostNameToIdMap; - } + public int getLastSyncTime() { + return this.last_synced; + } - public void setHostNameToIdMap(Map hostNameToIdMap) { - this.hostNameToIdMap = hostNameToIdMap; - } + public int getSyncCount() { + return this.sync_count; + } - public void setTrafficMetricsMap(Map trafficMetricsMap) { - this.trafficMetricsMap = trafficMetricsMap; - } -} \ No newline at end of file + public Map getHostNameToIdMap() { + return hostNameToIdMap; + } + + public void setHostNameToIdMap(Map hostNameToIdMap) { + this.hostNameToIdMap = hostNameToIdMap; + } + + public void setTrafficMetricsMap(Map trafficMetricsMap) { + this.trafficMetricsMap = trafficMetricsMap; + } +} diff --git a/apps/pom.xml b/apps/pom.xml index 52ff06ff91..0dcd5138f0 100644 --- a/apps/pom.xml +++ b/apps/pom.xml @@ -49,15 +49,15 @@ - api-threat-detection + threat-detection - api-threat-detection/pom.xml + threat-detection/pom.xml - api-threat-detection - + threat-detection + source-code-analyser diff --git a/apps/api-threat-detection/.gitignore b/apps/threat-detection/.gitignore similarity index 100% rename from apps/api-threat-detection/.gitignore rename to apps/threat-detection/.gitignore diff --git a/apps/api-threat-detection/Dockerfile b/apps/threat-detection/Dockerfile similarity index 100% rename from apps/api-threat-detection/Dockerfile rename to apps/threat-detection/Dockerfile diff --git a/apps/api-threat-detection/README.md b/apps/threat-detection/README.md similarity index 100% rename from apps/api-threat-detection/README.md rename to apps/threat-detection/README.md diff --git a/apps/api-threat-detection/pom.xml b/apps/threat-detection/pom.xml similarity index 92% rename from apps/api-threat-detection/pom.xml rename to apps/threat-detection/pom.xml index 5c1a9d2fbb..053bfc7862 100644 --- a/apps/api-threat-detection/pom.xml +++ b/apps/threat-detection/pom.xml @@ -1,7 +1,7 @@ + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 @@ -10,8 +10,8 @@ ${revision} - com.akto.apps.api-threat-detection - api-threat-detection + com.akto.apps.threat-detection + threat-detection jar @@ -88,6 +88,12 @@ caffeine 2.9.3 + + com.fasterxml.jackson.core + jackson-databind + 2.16.1 + compile + @@ -175,7 +181,7 @@ org.apache.maven.plugins maven-jar-plugin - api-threat-detection-1.0-SNAPSHOT-jar-with-dependencies + threat-detection-1.0-SNAPSHOT-jar-with-dependencies true diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java new file mode 100644 index 0000000000..045558ccf2 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -0,0 +1,32 @@ +package com.akto.threat.detection; + +import com.akto.threat.detection.config.kafka.KafkaConfig; +import com.akto.threat.detection.config.kafka.KafkaConsumerConfig; +import com.akto.threat.detection.config.kafka.KafkaProducerConfig; +import com.akto.threat.detection.tasks.MaliciousTrafficDetectorTask; +import io.lettuce.core.RedisClient; + +public class Main { + + public static void main(String[] args) { + new MaliciousTrafficDetectorTask( + KafkaConfig.newBuilder() + .setTopic("akto.api.logs") + .setGroupId("akto.threat.detection") + .setBootstrapServers("localhost:29092") + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) + .build(), + createRedisClient()) + .run(); + } + + public static RedisClient createRedisClient() { + return RedisClient.create(System.getenv("AKTO_THREAT_DETECTION_REDIS_URI")); + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/actor/ActorGenerator.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/actor/ActorGenerator.java new file mode 100644 index 0000000000..9f27be49bc --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/actor/ActorGenerator.java @@ -0,0 +1,10 @@ +package com.akto.threat.detection.actor; + +import com.akto.dto.HttpResponseParams; + +import java.util.Optional; + +public interface ActorGenerator { + + Optional generate(HttpResponseParams responseParams); +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/actor/SourceIPActorGenerator.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/actor/SourceIPActorGenerator.java new file mode 100644 index 0000000000..ebe444a2f1 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/actor/SourceIPActorGenerator.java @@ -0,0 +1,24 @@ +package com.akto.threat.detection.actor; + +import com.akto.dto.HttpResponseParams; +import com.akto.runtime.policies.ApiAccessTypePolicy; + +import java.util.List; +import java.util.Optional; + +public class SourceIPActorGenerator implements ActorGenerator { + + private SourceIPActorGenerator() {} + + public static SourceIPActorGenerator instance = new SourceIPActorGenerator(); + + @Override + public Optional generate(HttpResponseParams responseParams) { + List sourceIPs = ApiAccessTypePolicy.getSourceIps(responseParams); + if (sourceIPs.isEmpty()) { + return Optional.of(responseParams.getSourceIP()); + } + + return Optional.of(sourceIPs.get(0)); + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/CounterCache.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/CounterCache.java new file mode 100644 index 0000000000..49a59419e8 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/CounterCache.java @@ -0,0 +1,12 @@ +package com.akto.threat.detection.cache; + +public interface CounterCache { + + void incrementBy(String key, long val); + + void increment(String key); + + long get(String key); + + boolean exists(String key); +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/LongValueCodec.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/LongValueCodec.java new file mode 100644 index 0000000000..54879be184 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/LongValueCodec.java @@ -0,0 +1,32 @@ +package com.akto.threat.detection.cache; + +import io.lettuce.core.codec.RedisCodec; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +public class LongValueCodec implements RedisCodec { + + @Override + public String decodeKey(ByteBuffer bytes) { + return StandardCharsets.UTF_8.decode(bytes).toString(); + } + + @Override + public Long decodeValue(ByteBuffer bytes) { + if (!bytes.hasRemaining()) return null; + return bytes.getLong(); + } + + @Override + public ByteBuffer encodeKey(String key) { + return StandardCharsets.UTF_8.encode(key); + } + + @Override + public ByteBuffer encodeValue(Long value) { + ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES); + buffer.putLong(value); + buffer.flip(); + return buffer; + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java new file mode 100644 index 0000000000..55ac19424e --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java @@ -0,0 +1,106 @@ +package com.akto.threat.detection.cache; + +import io.lettuce.core.ExpireArgs; +import io.lettuce.core.RedisClient; +import io.lettuce.core.api.StatefulRedisConnection; + +import java.util.Optional; +import java.util.concurrent.*; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; + +public class RedisBackedCounterCache implements CounterCache { + + static class Op { + private final String key; + private final long value; + + public Op(String key, long value) { + this.key = key; + this.value = value; + } + + public String getKey() { + return key; + } + + public long getValue() { + return value; + } + } + + private final StatefulRedisConnection redis; + + private final Cache localCache; + + private final ConcurrentLinkedQueue pendingOps; + private final String prefix; + + public RedisBackedCounterCache(RedisClient redisClient, String prefix) { + this.prefix = prefix; + this.redis = redisClient.connect(new LongValueCodec()); + this.localCache = + Caffeine.newBuilder().maximumSize(100000).expireAfterWrite(3, TimeUnit.HOURS).build(); + + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + executor.scheduleAtFixedRate(this::syncToRedis, 60, 5, TimeUnit.SECONDS); + + this.pendingOps = new ConcurrentLinkedQueue<>(); + } + + private String getKey(String key) { + return prefix + "|" + key; + } + + @Override + public void increment(String key) { + incrementBy(key, 1); + } + + @Override + public void incrementBy(String key, long val) { + String _key = getKey(key); + localCache.asMap().merge(_key, val, Long::sum); + pendingOps.add(new Op(_key, val)); + + this.setExpiryIfNotSet(_key, 3 * 60 * 60); // added 3 hours expiry for now + } + + @Override + public long get(String key) { + return Optional.ofNullable(this.localCache.getIfPresent(getKey(key))).orElse(0L); + } + + @Override + public boolean exists(String key) { + return localCache.asMap().containsKey(getKey(key)); + } + + private void setExpiryIfNotSet(String key, long seconds) { + // We only set expiry for redis entry. For local cache we have lower expiry for + // all entries. + ExpireArgs args = ExpireArgs.Builder.nx(); + redis.async().expire(getKey(key), seconds, args); + } + + private void syncToRedis() { + while (!pendingOps.isEmpty()) { + Op op = pendingOps.poll(); + String key = op.getKey(); + long val = op.getValue(); + redis + .async() + .incrby(key, val) + .whenComplete( + (result, ex) -> { + if (ex != null) { + ex.printStackTrace(); + } + + if (result != null) { + localCache.asMap().put(key, result); + } + }); + } + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java new file mode 100644 index 0000000000..dd1210ad1f --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java @@ -0,0 +1,80 @@ +package com.akto.threat.detection.config.kafka; + +public class KafkaConfig { + private final String topic; + private final String bootstrapServers; + private final String groupId; + private final KafkaConsumerConfig consumerConfig; + private final KafkaProducerConfig producerConfig; + + public static class Builder { + private String topic; + private String bootstrapServers; + private String groupId; + private KafkaConsumerConfig consumerConfig; + private KafkaProducerConfig producerConfig; + + private Builder() {} + + public Builder setTopic(String topic) { + this.topic = topic; + return this; + } + + public Builder setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + return this; + } + + public Builder setGroupId(String groupId) { + this.groupId = groupId; + return this; + } + + public Builder setConsumerConfig(KafkaConsumerConfig consumerConfig) { + this.consumerConfig = consumerConfig; + return this; + } + + public Builder setProducerConfig(KafkaProducerConfig producerConfig) { + this.producerConfig = producerConfig; + return this; + } + + public KafkaConfig build() { + return new KafkaConfig(this); + } + } + + private KafkaConfig(Builder builder) { + this.topic = builder.topic; + this.bootstrapServers = builder.bootstrapServers; + this.groupId = builder.groupId; + this.consumerConfig = builder.consumerConfig; + this.producerConfig = builder.producerConfig; + } + + public String getTopic() { + return topic; + } + + public String getBootstrapServers() { + return bootstrapServers; + } + + public String getGroupId() { + return groupId; + } + + public KafkaConsumerConfig getConsumerConfig() { + return consumerConfig; + } + + public KafkaProducerConfig getProducerConfig() { + return producerConfig; + } + + public static Builder newBuilder() { + return new Builder(); + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConsumerConfig.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConsumerConfig.java new file mode 100644 index 0000000000..969eb96b50 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConsumerConfig.java @@ -0,0 +1,44 @@ +package com.akto.threat.detection.config.kafka; + +public class KafkaConsumerConfig { + private final int maxPollRecords; + private final int pollDurationMilli; + + public static class Builder { + private int maxPollRecords; + private int pollDurationMilli; + + private Builder() {} + + public Builder setMaxPollRecords(int maxPollRecords) { + this.maxPollRecords = maxPollRecords; + return this; + } + + public Builder setPollDurationMilli(int pollDurationMilli) { + this.pollDurationMilli = pollDurationMilli; + return this; + } + + public KafkaConsumerConfig build() { + return new KafkaConsumerConfig(this); + } + } + + public KafkaConsumerConfig(Builder builder) { + this.maxPollRecords = builder.maxPollRecords; + this.pollDurationMilli = builder.pollDurationMilli; + } + + public int getMaxPollRecords() { + return maxPollRecords; + } + + public int getPollDurationMilli() { + return pollDurationMilli; + } + + public static Builder newBuilder() { + return new Builder(); + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaProducerConfig.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaProducerConfig.java new file mode 100644 index 0000000000..a2d8fd6c70 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaProducerConfig.java @@ -0,0 +1,44 @@ +package com.akto.threat.detection.config.kafka; + +public class KafkaProducerConfig { + private final int lingerMs; + private final int batchSize; + + public static class Builder { + private int lingerMs; + private int batchSize; + + public Builder() {} + + public Builder setLingerMs(int lingerMs) { + this.lingerMs = lingerMs; + return this; + } + + public Builder setBatchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + public KafkaProducerConfig build() { + return new KafkaProducerConfig(this); + } + } + + public static Builder newBuilder() { + return new Builder(); + } + + public int getLingerMs() { + return lingerMs; + } + + public int getBatchSize() { + return batchSize; + } + + public KafkaProducerConfig(Builder builder) { + this.lingerMs = builder.lingerMs; + this.batchSize = builder.batchSize; + } +} diff --git a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/KafkaMessage.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MaliciousMessageEnvelope.java similarity index 70% rename from apps/api-threat-detection/src/main/java/com/akto/suspect_data/KafkaMessage.java rename to apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MaliciousMessageEnvelope.java index 5b371c2495..ba51bb2bcd 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/suspect_data/KafkaMessage.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MaliciousMessageEnvelope.java @@ -1,19 +1,19 @@ -package com.akto.suspect_data; +package com.akto.threat.detection.dto; import com.fasterxml.jackson.databind.ObjectMapper; import java.util.Optional; // Kafka Message Wrapper for suspect data -public class KafkaMessage { +public class MaliciousMessageEnvelope { private String accountId; private String data; - private static ObjectMapper objectMapper = new ObjectMapper(); + private static final ObjectMapper objectMapper = new ObjectMapper(); - public KafkaMessage() {} + public MaliciousMessageEnvelope() {} - public KafkaMessage(String accountId, String data) { + public MaliciousMessageEnvelope(String accountId, String data) { this.accountId = accountId; this.data = data; } @@ -44,9 +44,9 @@ public Optional marshal() { return Optional.empty(); } - public static Optional unmarshal(String message) { + public static Optional unmarshal(String message) { try { - return Optional.ofNullable(objectMapper.readValue(message, KafkaMessage.class)); + return Optional.ofNullable(objectMapper.readValue(message, MaliciousMessageEnvelope.class)); } catch (Exception e) { e.printStackTrace(); } diff --git a/apps/api-threat-detection/src/main/java/com/akto/auth/grpc/AuthToken.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/grpc/AuthToken.java similarity index 95% rename from apps/api-threat-detection/src/main/java/com/akto/auth/grpc/AuthToken.java rename to apps/threat-detection/src/main/java/com/akto/threat/detection/grpc/AuthToken.java index 8fbe579cfd..17bc9cbc38 100644 --- a/apps/api-threat-detection/src/main/java/com/akto/auth/grpc/AuthToken.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/grpc/AuthToken.java @@ -1,13 +1,12 @@ -package com.akto.auth.grpc; +package com.akto.threat.detection.grpc; + +import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; import io.grpc.CallCredentials; import io.grpc.Metadata; import io.grpc.Status; - import java.util.concurrent.Executor; -import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; - public class AuthToken extends CallCredentials { private final String token; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/Bin.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/Bin.java new file mode 100644 index 0000000000..fa1202f2dd --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/Bin.java @@ -0,0 +1,19 @@ +package com.akto.threat.detection.smart_event_detector.window_based; + +public class Bin { + int binId; + long count; + + public Bin(int binId, long count) { + this.binId = binId; + this.count = count; + } + + public int getBinId() { + return binId; + } + + public long getCount() { + return count; + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/Data.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/Data.java new file mode 100644 index 0000000000..43f928a6ab --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/Data.java @@ -0,0 +1,49 @@ +package com.akto.threat.detection.smart_event_detector.window_based; + +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.ArrayList; +import java.util.List; + +public class Data { + @JsonProperty("ln") + public long lastNotifiedAt = 0; + + @JsonProperty("rq") + public List requests = new ArrayList<>(); + + public static class Request { + private long receivedAt; + + public Request() {} + + public Request(long receivedAt) { + this.receivedAt = receivedAt; + } + + public long getReceivedAt() { + return receivedAt; + } + + public void setReceivedAt(long receivedAt) { + this.receivedAt = receivedAt; + } + } + + public Data() {} + + public long getLastNotifiedAt() { + return lastNotifiedAt; + } + + public void setLastNotifiedAt(long lastNotifiedAt) { + this.lastNotifiedAt = lastNotifiedAt; + } + + public List getRequests() { + return requests; + } + + public void setRequests(List requests) { + this.requests = requests; + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java new file mode 100644 index 0000000000..2d4bf3a076 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java @@ -0,0 +1,107 @@ +package com.akto.threat.detection.smart_event_detector.window_based; + +import com.akto.threat.detection.cache.CounterCache; +import com.akto.dto.api_protection_parse_layer.Rule; +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class WindowBasedThresholdNotifier { + + private final Config config; + + // We can use an in-memory cache for this, since we dont mind being notified + // more than once by multiple instances of the service. + // But on 1 instance, we should not notify more than once in the cooldown + // period. + // TODO: Move this to redis + private final ConcurrentMap notifiedMap; + + public static class Config { + private final int threshold; + private final int windowSizeInMinutes; + private int notificationCooldownInSeconds = 60 * 30; // 30 mins + + public Config(int threshold, int windowInSeconds) { + this.threshold = threshold; + this.windowSizeInMinutes = windowInSeconds; + } + + public int getThreshold() { + return threshold; + } + + public int getWindowSizeInMinutes() { + return windowSizeInMinutes; + } + + public int getNotificationCooldownInSeconds() { + return notificationCooldownInSeconds; + } + } + + public static class Result { + private final boolean shouldNotify; + + public Result(boolean shouldNotify) { + this.shouldNotify = shouldNotify; + } + + public boolean shouldNotify() { + return shouldNotify; + } + } + + public Config getConfig() { + return config; + } + + private final CounterCache cache; + + public WindowBasedThresholdNotifier(CounterCache cache, Config config) { + this.cache = cache; + this.config = config; + this.notifiedMap = new ConcurrentHashMap<>(); + } + + public Result shouldNotify(String aggKey, MaliciousEvent maliciousEvent, Rule rule) { + int binId = (int) maliciousEvent.getTimestamp() / 60; + String cacheKey = aggKey + "|" + binId; + this.cache.increment(cacheKey); + + long windowCount = 0L; + List bins = getBins(aggKey, binId - rule.getCondition().getWindowThreshold() + 1, binId); + for (Bin data : bins) { + windowCount += data.getCount(); + } + + boolean thresholdBreached = windowCount >= rule.getCondition().getMatchCount(); + + long now = System.currentTimeMillis() / 1000L; + long lastNotified = this.notifiedMap.getOrDefault(aggKey, 0L); + + boolean cooldownBreached = + (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); + + if (thresholdBreached && cooldownBreached) { + this.notifiedMap.put(aggKey, now); + return new Result(true); + } + + return new Result(false); + } + + public List getBins(String aggKey, int binStart, int binEnd) { + List binData = new ArrayList<>(); + for (int i = binStart; i <= binEnd; i++) { + String key = aggKey + "|" + i; + if (!this.cache.exists(key)) { + continue; + } + binData.add(new Bin(i, this.cache.get(key))); + } + return binData; + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java new file mode 100644 index 0000000000..4d79780fe9 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -0,0 +1,10 @@ +package com.akto.threat.detection.tasks; + +/* +This will read sample malicious data from kafka topic and save it to DB. + */ +public class FlushSampleDataTask implements Task { + + @Override + public void run() {} +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java new file mode 100644 index 0000000000..73a612a521 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -0,0 +1,275 @@ +package com.akto.threat.detection.tasks; + +import com.akto.threat.detection.actor.SourceIPActorGenerator; +import com.akto.threat.detection.cache.RedisBackedCounterCache; +import com.akto.threat.detection.config.kafka.KafkaConfig; +import com.akto.dao.monitoring.FilterYamlTemplateDao; +import com.akto.data_actor.DataActor; +import com.akto.data_actor.DataActorFactory; +import com.akto.dto.ApiInfo; +import com.akto.dto.HttpResponseParams; +import com.akto.threat.detection.dto.MaliciousMessageEnvelope; +import com.akto.dto.RawApi; +import com.akto.dto.api_protection_parse_layer.AggregationRules; +import com.akto.dto.api_protection_parse_layer.Condition; +import com.akto.dto.api_protection_parse_layer.Rule; +import com.akto.dto.monitoring.FilterConfig; +import com.akto.dto.test_editor.YamlTemplate; +import com.akto.dto.type.URLMethods; +import com.akto.threat.detection.grpc.AuthToken; +import com.akto.hybrid_parsers.HttpCallParser; +import com.akto.kafka.Kafka; +import com.akto.proto.threat_protection.consumer_service.v1.*; +import com.akto.rules.TestPlugin; +import com.akto.runtime.utils.Utils; +import com.akto.threat.detection.smart_event_detector.window_based.WindowBasedThresholdNotifier; +import com.akto.test_editor.execution.VariableResolver; +import com.akto.test_editor.filter.data_operands_impl.ValidationResult; +import com.google.protobuf.InvalidProtocolBufferException; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import io.grpc.stub.StreamObserver; +import io.lettuce.core.RedisClient; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import com.google.protobuf.util.JsonFormat; + +import java.time.Duration; +import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/* +Class is responsible for consuming traffic data from the Kafka topic. +Pass data through filters and identify malicious traffic. + */ +public class MaliciousTrafficDetectorTask { + + private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); + private final Consumer kafkaConsumer; + private final KafkaConfig kafkaConfig; + private final HttpCallParser httpCallParser; + private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; + + private Map apiFilters; + private int filterLastUpdatedAt = 0; + + private final Kafka kafka; + + private static final String KAFKA_MALICIOUS_TOPIC = "akto.malicious"; + + private static final DataActor dataActor = DataActorFactory.fetchInstance(); + private final ConsumerServiceGrpc.ConsumerServiceStub consumerServiceStub; + + public MaliciousTrafficDetectorTask(KafkaConfig trafficConfig, RedisClient redisClient) { + this.kafkaConfig = trafficConfig; + + String kafkaBrokerUrl = trafficConfig.getBootstrapServers(); + String groupId = trafficConfig.getGroupId(); + + Properties properties = + Utils.configProperties( + kafkaBrokerUrl, groupId, trafficConfig.getConsumerConfig().getMaxPollRecords()); + this.kafkaConsumer = new KafkaConsumer<>(properties); + + this.httpCallParser = new HttpCallParser(120, 1000); + + this.windowBasedThresholdNotifier = + new WindowBasedThresholdNotifier( + new RedisBackedCounterCache(redisClient, "wbt"), + new WindowBasedThresholdNotifier.Config(100, 10 * 60)); + + String target = "localhost:8980"; + ManagedChannel channel = + Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); + this.consumerServiceStub = + ConsumerServiceGrpc.newStub(channel) + .withCallCredentials( + new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); + + this.kafka = + new Kafka( + kafkaBrokerUrl, + trafficConfig.getProducerConfig().getLingerMs(), + trafficConfig.getProducerConfig().getBatchSize()); + } + + public void run() { + this.kafkaConsumer.subscribe(Collections.singletonList(this.kafkaConfig.getTopic())); + pollingExecutor.execute( + new Runnable() { + @Override + public void run() { + // Poll data from Kafka topic + while (true) { + ConsumerRecords records = + kafkaConsumer.poll( + Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); + for (ConsumerRecord record : records) { + processRecord(record); + } + } + } + }); + } + + private Map getFilters() { + int now = (int) (System.currentTimeMillis() / 1000); + if (now - filterLastUpdatedAt < 60) { + return apiFilters; + } + + List templates = dataActor.fetchFilterYamlTemplates(); + apiFilters = FilterYamlTemplateDao.fetchFilterConfig(false, templates, false); + this.filterLastUpdatedAt = now; + return apiFilters; + } + + private MaliciousMessageEnvelope generateKafkaMessage( + String accountId, MaliciousEvent maliciousEvent) throws InvalidProtocolBufferException { + String data = JsonFormat.printer().print(maliciousEvent); + return new MaliciousMessageEnvelope(accountId, data); + } + + private boolean validateFilterForRequest( + HttpResponseParams responseParam, FilterConfig apiFilter) { + try { + String message = responseParam.getOrig(); + RawApi rawApi = RawApi.buildFromMessage(message); + int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); + responseParam.requestParams.setApiCollectionId(apiCollectionId); + String url = responseParam.getRequestParams().getURL(); + URLMethods.Method method = + URLMethods.Method.fromString(responseParam.getRequestParams().getMethod()); + ApiInfo.ApiInfoKey apiInfoKey = new ApiInfo.ApiInfoKey(apiCollectionId, url, method); + Map varMap = apiFilter.resolveVarMap(); + VariableResolver.resolveWordList( + varMap, + new HashMap>() { + { + put(apiInfoKey, Collections.singletonList(message)); + } + }, + apiInfoKey); + String filterExecutionLogId = UUID.randomUUID().toString(); + ValidationResult res = + TestPlugin.validateFilter( + apiFilter.getFilter().getNode(), rawApi, apiInfoKey, varMap, filterExecutionLogId); + + return res.getIsValid(); + } catch (Exception e) { + e.printStackTrace(); + } + + return false; + } + + private void processRecord(ConsumerRecord record) { + Map filters = this.getFilters(); + if (filters.isEmpty()) { + return; + } + + HttpResponseParams responseParam = HttpCallParser.parseKafkaMessage(record.value()); + List maliciousMessages = new ArrayList<>(); + + for (FilterConfig apiFilter : apiFilters.values()) { + boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); + + // If a request passes any of the filter, then it's a malicious request, + // and so we push it to kafka + if (hasPassedFilter) { + // Later we will also add aggregation support + // Eg: 100 4xx requests in last 10 minutes. + // But regardless of whether request falls in aggregation or not, + // we still push malicious requests to kafka + + // todo: modify fetch yaml and read aggregate rules from it + List rules = new ArrayList<>(); + rules.add(new Rule("Lfi Rule 1", new Condition(100, 10))); + AggregationRules aggRules = new AggregationRules(); + aggRules.setRule(rules); + + SourceIPActorGenerator.instance + .generate(responseParam) + .ifPresent( + actor -> { + String groupKey = apiFilter.getId(); + String aggKey = actor + "|" + groupKey; + + MaliciousEvent maliciousEvent = + MaliciousEvent.newBuilder() + .setActorId(actor) + .setFilterId(apiFilter.getId()) + .setUrl(responseParam.getRequestParams().getURL()) + .setMethod(responseParam.getRequestParams().getMethod()) + .setPayload(responseParam.getOrig()) + .setIp(actor) // For now using actor as IP + .setApiCollectionId(responseParam.getRequestParams().getApiCollectionId()) + .setTimestamp(responseParam.getTime()) + .build(); + + try { + maliciousMessages.add( + generateKafkaMessage(responseParam.getAccountId(), maliciousEvent)); + } catch (InvalidProtocolBufferException e) { + return; + } + + for (Rule rule : aggRules.getRule()) { + WindowBasedThresholdNotifier.Result result = + this.windowBasedThresholdNotifier.shouldNotify( + aggKey, maliciousEvent, rule); + + if (result.shouldNotify()) { + SmartEvent smartEvent = + SmartEvent.newBuilder() + .setFilterId(apiFilter.getId()) + .setActorId(actor) + .setDetectedAt(responseParam.getTime()) + .setRuleId(rule.getName()) + .build(); + this.consumerServiceStub.saveSmartEvent( + SaveSmartEventRequest.newBuilder().setEvent(smartEvent).build(), + new StreamObserver() { + @Override + public void onNext(SaveSmartEventResponse value) { + // Do nothing + } + + @Override + public void onError(Throwable t) { + t.printStackTrace(); + } + + @Override + public void onCompleted() { + // Do nothing + } + }); + } + } + }); + } + } + + // Should we push all the messages in one go + // or call kafka.send for each HttpRequestParams + try { + maliciousMessages.forEach( + sample -> { + sample + .marshal() + .ifPresent( + data -> { + kafka.send(data, KAFKA_MALICIOUS_TOPIC); + }); + }); + } catch (Exception e) { + e.printStackTrace(); + } + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/Task.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/Task.java new file mode 100644 index 0000000000..1156ccbd8b --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/Task.java @@ -0,0 +1,6 @@ +package com.akto.threat.detection.tasks; + +public interface Task { + + void run(); +} diff --git a/apps/api-threat-detection/src/main/resources/version.txt b/apps/threat-detection/src/main/resources/version.txt similarity index 100% rename from apps/api-threat-detection/src/main/resources/version.txt rename to apps/threat-detection/src/main/resources/version.txt diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceGrpc.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceGrpc.java deleted file mode 100644 index f927cfd0d2..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceGrpc.java +++ /dev/null @@ -1,367 +0,0 @@ -package com.akto.proto.threat_protection.consumer_service.v1; - -import static io.grpc.MethodDescriptor.generateFullMethodName; - -/** - */ -@javax.annotation.Generated( - value = "by gRPC proto compiler (version 1.68.1)", - comments = "Source: threat_protection/consumer_service/v1/consumer_service.proto") -@io.grpc.stub.annotations.GrpcGenerated -public final class ConsumerServiceGrpc { - - private ConsumerServiceGrpc() {} - - public static final java.lang.String SERVICE_NAME = "threat_protection.consumer_service.v1.ConsumerService"; - - // Static method descriptors that strictly reflect the proto. - private static volatile io.grpc.MethodDescriptor getSaveMaliciousEventMethod; - - @io.grpc.stub.annotations.RpcMethod( - fullMethodName = SERVICE_NAME + '/' + "SaveMaliciousEvent", - requestType = com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.class, - responseType = com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.class, - methodType = io.grpc.MethodDescriptor.MethodType.UNARY) - public static io.grpc.MethodDescriptor getSaveMaliciousEventMethod() { - io.grpc.MethodDescriptor getSaveMaliciousEventMethod; - if ((getSaveMaliciousEventMethod = ConsumerServiceGrpc.getSaveMaliciousEventMethod) == null) { - synchronized (ConsumerServiceGrpc.class) { - if ((getSaveMaliciousEventMethod = ConsumerServiceGrpc.getSaveMaliciousEventMethod) == null) { - ConsumerServiceGrpc.getSaveMaliciousEventMethod = getSaveMaliciousEventMethod = - io.grpc.MethodDescriptor.newBuilder() - .setType(io.grpc.MethodDescriptor.MethodType.UNARY) - .setFullMethodName(generateFullMethodName(SERVICE_NAME, "SaveMaliciousEvent")) - .setSampledToLocalTracing(true) - .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.getDefaultInstance())) - .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.getDefaultInstance())) - .setSchemaDescriptor(new ConsumerServiceMethodDescriptorSupplier("SaveMaliciousEvent")) - .build(); - } - } - } - return getSaveMaliciousEventMethod; - } - - private static volatile io.grpc.MethodDescriptor getSaveSmartEventMethod; - - @io.grpc.stub.annotations.RpcMethod( - fullMethodName = SERVICE_NAME + '/' + "SaveSmartEvent", - requestType = com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.class, - responseType = com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.class, - methodType = io.grpc.MethodDescriptor.MethodType.UNARY) - public static io.grpc.MethodDescriptor getSaveSmartEventMethod() { - io.grpc.MethodDescriptor getSaveSmartEventMethod; - if ((getSaveSmartEventMethod = ConsumerServiceGrpc.getSaveSmartEventMethod) == null) { - synchronized (ConsumerServiceGrpc.class) { - if ((getSaveSmartEventMethod = ConsumerServiceGrpc.getSaveSmartEventMethod) == null) { - ConsumerServiceGrpc.getSaveSmartEventMethod = getSaveSmartEventMethod = - io.grpc.MethodDescriptor.newBuilder() - .setType(io.grpc.MethodDescriptor.MethodType.UNARY) - .setFullMethodName(generateFullMethodName(SERVICE_NAME, "SaveSmartEvent")) - .setSampledToLocalTracing(true) - .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.getDefaultInstance())) - .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.getDefaultInstance())) - .setSchemaDescriptor(new ConsumerServiceMethodDescriptorSupplier("SaveSmartEvent")) - .build(); - } - } - } - return getSaveSmartEventMethod; - } - - /** - * Creates a new async stub that supports all call types for the service - */ - public static ConsumerServiceStub newStub(io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public ConsumerServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceStub(channel, callOptions); - } - }; - return ConsumerServiceStub.newStub(factory, channel); - } - - /** - * Creates a new blocking-style stub that supports unary and streaming output calls on the service - */ - public static ConsumerServiceBlockingStub newBlockingStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public ConsumerServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceBlockingStub(channel, callOptions); - } - }; - return ConsumerServiceBlockingStub.newStub(factory, channel); - } - - /** - * Creates a new ListenableFuture-style stub that supports unary calls on the service - */ - public static ConsumerServiceFutureStub newFutureStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public ConsumerServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceFutureStub(channel, callOptions); - } - }; - return ConsumerServiceFutureStub.newStub(factory, channel); - } - - /** - */ - public interface AsyncService { - - /** - */ - default void saveMaliciousEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getSaveMaliciousEventMethod(), responseObserver); - } - - /** - */ - default void saveSmartEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getSaveSmartEventMethod(), responseObserver); - } - } - - /** - * Base class for the server implementation of the service ConsumerService. - */ - public static abstract class ConsumerServiceImplBase - implements io.grpc.BindableService, AsyncService { - - @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { - return ConsumerServiceGrpc.bindService(this); - } - } - - /** - * A stub to allow clients to do asynchronous rpc calls to service ConsumerService. - */ - public static final class ConsumerServiceStub - extends io.grpc.stub.AbstractAsyncStub { - private ConsumerServiceStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected ConsumerServiceStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceStub(channel, callOptions); - } - - /** - */ - public void saveMaliciousEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ClientCalls.asyncUnaryCall( - getChannel().newCall(getSaveMaliciousEventMethod(), getCallOptions()), request, responseObserver); - } - - /** - */ - public void saveSmartEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ClientCalls.asyncUnaryCall( - getChannel().newCall(getSaveSmartEventMethod(), getCallOptions()), request, responseObserver); - } - } - - /** - * A stub to allow clients to do synchronous rpc calls to service ConsumerService. - */ - public static final class ConsumerServiceBlockingStub - extends io.grpc.stub.AbstractBlockingStub { - private ConsumerServiceBlockingStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected ConsumerServiceBlockingStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceBlockingStub(channel, callOptions); - } - - /** - */ - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse saveMaliciousEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest request) { - return io.grpc.stub.ClientCalls.blockingUnaryCall( - getChannel(), getSaveMaliciousEventMethod(), getCallOptions(), request); - } - - /** - */ - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse saveSmartEvent(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest request) { - return io.grpc.stub.ClientCalls.blockingUnaryCall( - getChannel(), getSaveSmartEventMethod(), getCallOptions(), request); - } - } - - /** - * A stub to allow clients to do ListenableFuture-style rpc calls to service ConsumerService. - */ - public static final class ConsumerServiceFutureStub - extends io.grpc.stub.AbstractFutureStub { - private ConsumerServiceFutureStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected ConsumerServiceFutureStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceFutureStub(channel, callOptions); - } - - /** - */ - public com.google.common.util.concurrent.ListenableFuture saveMaliciousEvent( - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest request) { - return io.grpc.stub.ClientCalls.futureUnaryCall( - getChannel().newCall(getSaveMaliciousEventMethod(), getCallOptions()), request); - } - - /** - */ - public com.google.common.util.concurrent.ListenableFuture saveSmartEvent( - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest request) { - return io.grpc.stub.ClientCalls.futureUnaryCall( - getChannel().newCall(getSaveSmartEventMethod(), getCallOptions()), request); - } - } - - private static final int METHODID_SAVE_MALICIOUS_EVENT = 0; - private static final int METHODID_SAVE_SMART_EVENT = 1; - - private static final class MethodHandlers implements - io.grpc.stub.ServerCalls.UnaryMethod, - io.grpc.stub.ServerCalls.ServerStreamingMethod, - io.grpc.stub.ServerCalls.ClientStreamingMethod, - io.grpc.stub.ServerCalls.BidiStreamingMethod { - private final AsyncService serviceImpl; - private final int methodId; - - MethodHandlers(AsyncService serviceImpl, int methodId) { - this.serviceImpl = serviceImpl; - this.methodId = methodId; - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - case METHODID_SAVE_MALICIOUS_EVENT: - serviceImpl.saveMaliciousEvent((com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest) request, - (io.grpc.stub.StreamObserver) responseObserver); - break; - case METHODID_SAVE_SMART_EVENT: - serviceImpl.saveSmartEvent((com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest) request, - (io.grpc.stub.StreamObserver) responseObserver); - break; - default: - throw new AssertionError(); - } - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public io.grpc.stub.StreamObserver invoke( - io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - default: - throw new AssertionError(); - } - } - } - - public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { - return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) - .addMethod( - getSaveMaliciousEventMethod(), - io.grpc.stub.ServerCalls.asyncUnaryCall( - new MethodHandlers< - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest, - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse>( - service, METHODID_SAVE_MALICIOUS_EVENT))) - .addMethod( - getSaveSmartEventMethod(), - io.grpc.stub.ServerCalls.asyncUnaryCall( - new MethodHandlers< - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest, - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse>( - service, METHODID_SAVE_SMART_EVENT))) - .build(); - } - - private static abstract class ConsumerServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { - ConsumerServiceBaseDescriptorSupplier() {} - - @java.lang.Override - public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.getDescriptor(); - } - - @java.lang.Override - public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { - return getFileDescriptor().findServiceByName("ConsumerService"); - } - } - - private static final class ConsumerServiceFileDescriptorSupplier - extends ConsumerServiceBaseDescriptorSupplier { - ConsumerServiceFileDescriptorSupplier() {} - } - - private static final class ConsumerServiceMethodDescriptorSupplier - extends ConsumerServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { - private final java.lang.String methodName; - - ConsumerServiceMethodDescriptorSupplier(java.lang.String methodName) { - this.methodName = methodName; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { - return getServiceDescriptor().findMethodByName(methodName); - } - } - - private static volatile io.grpc.ServiceDescriptor serviceDescriptor; - - public static io.grpc.ServiceDescriptor getServiceDescriptor() { - io.grpc.ServiceDescriptor result = serviceDescriptor; - if (result == null) { - synchronized (ConsumerServiceGrpc.class) { - result = serviceDescriptor; - if (result == null) { - serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) - .setSchemaDescriptor(new ConsumerServiceFileDescriptorSupplier()) - .addMethod(getSaveMaliciousEventMethod()) - .addMethod(getSaveSmartEventMethod()) - .build(); - } - } - } - return result; - } -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java deleted file mode 100644 index 79e87f5a06..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/ConsumerServiceProto.java +++ /dev/null @@ -1,145 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -public final class ConsumerServiceProto { - private ConsumerServiceProto() {} - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - ConsumerServiceProto.class.getName()); - } - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { - } - - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); - } - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_consumer_service_v1_SmartEvent_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_consumer_service_v1_SmartEvent_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_consumer_service_v1_MaliciousEvent_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_consumer_service_v1_MaliciousEvent_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n builder) { - super(builder); - } - private MaliciousEvent() { - actorId_ = ""; - filterId_ = ""; - ip_ = ""; - url_ = ""; - method_ = ""; - payload_ = ""; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.class, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder.class); - } - - public static final int ACTOR_ID_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object actorId_ = ""; - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The actorId. - */ - @java.lang.Override - public java.lang.String getActorId() { - java.lang.Object ref = actorId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actorId_ = s; - return s; - } - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The bytes for actorId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getActorIdBytes() { - java.lang.Object ref = actorId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actorId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int IP_FIELD_NUMBER = 3; - @SuppressWarnings("serial") - private volatile java.lang.Object ip_ = ""; - /** - * string ip = 3 [json_name = "ip"]; - * @return The ip. - */ - @java.lang.Override - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } - } - /** - * string ip = 3 [json_name = "ip"]; - * @return The bytes for ip. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TIMESTAMP_FIELD_NUMBER = 4; - private long timestamp_ = 0L; - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - - public static final int URL_FIELD_NUMBER = 5; - @SuppressWarnings("serial") - private volatile java.lang.Object url_ = ""; - /** - * string url = 5 [json_name = "url"]; - * @return The url. - */ - @java.lang.Override - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } - } - /** - * string url = 5 [json_name = "url"]; - * @return The bytes for url. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int METHOD_FIELD_NUMBER = 6; - @SuppressWarnings("serial") - private volatile java.lang.Object method_ = ""; - /** - * string method = 6 [json_name = "method"]; - * @return The method. - */ - @java.lang.Override - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } - } - /** - * string method = 6 [json_name = "method"]; - * @return The bytes for method. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int API_COLLECTION_ID_FIELD_NUMBER = 7; - private int apiCollectionId_ = 0; - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - - public static final int PAYLOAD_FIELD_NUMBER = 8; - @SuppressWarnings("serial") - private volatile java.lang.Object payload_ = ""; - /** - * string payload = 8 [json_name = "payload"]; - * @return The payload. - */ - @java.lang.Override - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } - } - /** - * string payload = 8 [json_name = "payload"]; - * @return The bytes for payload. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actorId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, actorId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 3, ip_); - } - if (timestamp_ != 0L) { - output.writeInt64(4, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 5, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 6, method_); - } - if (apiCollectionId_ != 0) { - output.writeInt32(7, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 8, payload_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actorId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actorId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(3, ip_); - } - if (timestamp_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(4, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(5, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(6, method_); - } - if (apiCollectionId_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(7, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(8, payload_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent other = (com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent) obj; - - if (!getActorId() - .equals(other.getActorId())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (!getIp() - .equals(other.getIp())) return false; - if (getTimestamp() - != other.getTimestamp()) return false; - if (!getUrl() - .equals(other.getUrl())) return false; - if (!getMethod() - .equals(other.getMethod())) return false; - if (getApiCollectionId() - != other.getApiCollectionId()) return false; - if (!getPayload() - .equals(other.getPayload())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACTOR_ID_FIELD_NUMBER; - hash = (53 * hash) + getActorId().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (37 * hash) + IP_FIELD_NUMBER; - hash = (53 * hash) + getIp().hashCode(); - hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getTimestamp()); - hash = (37 * hash) + URL_FIELD_NUMBER; - hash = (53 * hash) + getUrl().hashCode(); - hash = (37 * hash) + METHOD_FIELD_NUMBER; - hash = (53 * hash) + getMethod().hashCode(); - hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; - hash = (53 * hash) + getApiCollectionId(); - hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; - hash = (53 * hash) + getPayload().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.consumer_service.v1.MaliciousEvent} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.MaliciousEvent) - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.class, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - actorId_ = ""; - filterId_ = ""; - ip_ = ""; - timestamp_ = 0L; - url_ = ""; - method_ = ""; - apiCollectionId_ = 0; - payload_ = ""; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getDefaultInstanceForType() { - return com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent build() { - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent buildPartial() { - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent result = new com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.actorId_ = actorId_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.filterId_ = filterId_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.ip_ = ip_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.timestamp_ = timestamp_; - } - if (((from_bitField0_ & 0x00000010) != 0)) { - result.url_ = url_; - } - if (((from_bitField0_ & 0x00000020) != 0)) { - result.method_ = method_; - } - if (((from_bitField0_ & 0x00000040) != 0)) { - result.apiCollectionId_ = apiCollectionId_; - } - if (((from_bitField0_ & 0x00000080) != 0)) { - result.payload_ = payload_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent) { - return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent other) { - if (other == com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.getDefaultInstance()) return this; - if (!other.getActorId().isEmpty()) { - actorId_ = other.actorId_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000002; - onChanged(); - } - if (!other.getIp().isEmpty()) { - ip_ = other.ip_; - bitField0_ |= 0x00000004; - onChanged(); - } - if (other.getTimestamp() != 0L) { - setTimestamp(other.getTimestamp()); - } - if (!other.getUrl().isEmpty()) { - url_ = other.url_; - bitField0_ |= 0x00000010; - onChanged(); - } - if (!other.getMethod().isEmpty()) { - method_ = other.method_; - bitField0_ |= 0x00000020; - onChanged(); - } - if (other.getApiCollectionId() != 0) { - setApiCollectionId(other.getApiCollectionId()); - } - if (!other.getPayload().isEmpty()) { - payload_ = other.payload_; - bitField0_ |= 0x00000080; - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - actorId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000002; - break; - } // case 18 - case 26: { - ip_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000004; - break; - } // case 26 - case 32: { - timestamp_ = input.readInt64(); - bitField0_ |= 0x00000008; - break; - } // case 32 - case 42: { - url_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000010; - break; - } // case 42 - case 50: { - method_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000020; - break; - } // case 50 - case 56: { - apiCollectionId_ = input.readInt32(); - bitField0_ |= 0x00000040; - break; - } // case 56 - case 66: { - payload_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000080; - break; - } // case 66 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object actorId_ = ""; - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The actorId. - */ - public java.lang.String getActorId() { - java.lang.Object ref = actorId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actorId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The bytes for actorId. - */ - public com.google.protobuf.ByteString - getActorIdBytes() { - java.lang.Object ref = actorId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actorId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @param value The actorId to set. - * @return This builder for chaining. - */ - public Builder setActorId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - actorId_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return This builder for chaining. - */ - public Builder clearActorId() { - actorId_ = getDefaultInstance().getActorId(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @param value The bytes for actorId to set. - * @return This builder for chaining. - */ - public Builder setActorIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - actorId_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - private java.lang.Object ip_ = ""; - /** - * string ip = 3 [json_name = "ip"]; - * @return The ip. - */ - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string ip = 3 [json_name = "ip"]; - * @return The bytes for ip. - */ - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string ip = 3 [json_name = "ip"]; - * @param value The ip to set. - * @return This builder for chaining. - */ - public Builder setIp( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ip_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * string ip = 3 [json_name = "ip"]; - * @return This builder for chaining. - */ - public Builder clearIp() { - ip_ = getDefaultInstance().getIp(); - bitField0_ = (bitField0_ & ~0x00000004); - onChanged(); - return this; - } - /** - * string ip = 3 [json_name = "ip"]; - * @param value The bytes for ip to set. - * @return This builder for chaining. - */ - public Builder setIpBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ip_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - - private long timestamp_ ; - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @param value The timestamp to set. - * @return This builder for chaining. - */ - public Builder setTimestamp(long value) { - - timestamp_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @return This builder for chaining. - */ - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000008); - timestamp_ = 0L; - onChanged(); - return this; - } - - private java.lang.Object url_ = ""; - /** - * string url = 5 [json_name = "url"]; - * @return The url. - */ - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string url = 5 [json_name = "url"]; - * @return The bytes for url. - */ - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string url = 5 [json_name = "url"]; - * @param value The url to set. - * @return This builder for chaining. - */ - public Builder setUrl( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - url_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - /** - * string url = 5 [json_name = "url"]; - * @return This builder for chaining. - */ - public Builder clearUrl() { - url_ = getDefaultInstance().getUrl(); - bitField0_ = (bitField0_ & ~0x00000010); - onChanged(); - return this; - } - /** - * string url = 5 [json_name = "url"]; - * @param value The bytes for url to set. - * @return This builder for chaining. - */ - public Builder setUrlBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - url_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - - private java.lang.Object method_ = ""; - /** - * string method = 6 [json_name = "method"]; - * @return The method. - */ - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string method = 6 [json_name = "method"]; - * @return The bytes for method. - */ - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string method = 6 [json_name = "method"]; - * @param value The method to set. - * @return This builder for chaining. - */ - public Builder setMethod( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - method_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - /** - * string method = 6 [json_name = "method"]; - * @return This builder for chaining. - */ - public Builder clearMethod() { - method_ = getDefaultInstance().getMethod(); - bitField0_ = (bitField0_ & ~0x00000020); - onChanged(); - return this; - } - /** - * string method = 6 [json_name = "method"]; - * @param value The bytes for method to set. - * @return This builder for chaining. - */ - public Builder setMethodBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - method_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - - private int apiCollectionId_ ; - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @param value The apiCollectionId to set. - * @return This builder for chaining. - */ - public Builder setApiCollectionId(int value) { - - apiCollectionId_ = value; - bitField0_ |= 0x00000040; - onChanged(); - return this; - } - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @return This builder for chaining. - */ - public Builder clearApiCollectionId() { - bitField0_ = (bitField0_ & ~0x00000040); - apiCollectionId_ = 0; - onChanged(); - return this; - } - - private java.lang.Object payload_ = ""; - /** - * string payload = 8 [json_name = "payload"]; - * @return The payload. - */ - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string payload = 8 [json_name = "payload"]; - * @return The bytes for payload. - */ - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string payload = 8 [json_name = "payload"]; - * @param value The payload to set. - * @return This builder for chaining. - */ - public Builder setPayload( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - payload_ = value; - bitField0_ |= 0x00000080; - onChanged(); - return this; - } - /** - * string payload = 8 [json_name = "payload"]; - * @return This builder for chaining. - */ - public Builder clearPayload() { - payload_ = getDefaultInstance().getPayload(); - bitField0_ = (bitField0_ & ~0x00000080); - onChanged(); - return this; - } - /** - * string payload = 8 [json_name = "payload"]; - * @param value The bytes for payload to set. - * @return This builder for chaining. - */ - public Builder setPayloadBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - payload_ = value; - bitField0_ |= 0x00000080; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.MaliciousEvent) - } - - // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.MaliciousEvent) - private static final com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent(); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public MaliciousEvent parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEventOrBuilder.java deleted file mode 100644 index 5abab768f8..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/MaliciousEventOrBuilder.java +++ /dev/null @@ -1,95 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -public interface MaliciousEventOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.MaliciousEvent) - com.google.protobuf.MessageOrBuilder { - - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The actorId. - */ - java.lang.String getActorId(); - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The bytes for actorId. - */ - com.google.protobuf.ByteString - getActorIdBytes(); - - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - java.lang.String getFilterId(); - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - com.google.protobuf.ByteString - getFilterIdBytes(); - - /** - * string ip = 3 [json_name = "ip"]; - * @return The ip. - */ - java.lang.String getIp(); - /** - * string ip = 3 [json_name = "ip"]; - * @return The bytes for ip. - */ - com.google.protobuf.ByteString - getIpBytes(); - - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @return The timestamp. - */ - long getTimestamp(); - - /** - * string url = 5 [json_name = "url"]; - * @return The url. - */ - java.lang.String getUrl(); - /** - * string url = 5 [json_name = "url"]; - * @return The bytes for url. - */ - com.google.protobuf.ByteString - getUrlBytes(); - - /** - * string method = 6 [json_name = "method"]; - * @return The method. - */ - java.lang.String getMethod(); - /** - * string method = 6 [json_name = "method"]; - * @return The bytes for method. - */ - com.google.protobuf.ByteString - getMethodBytes(); - - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - int getApiCollectionId(); - - /** - * string payload = 8 [json_name = "payload"]; - * @return The payload. - */ - java.lang.String getPayload(); - /** - * string payload = 8 [json_name = "payload"]; - * @return The bytes for payload. - */ - com.google.protobuf.ByteString - getPayloadBytes(); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java deleted file mode 100644 index cd6a177be9..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequest.java +++ /dev/null @@ -1,719 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -/** - * Protobuf type {@code threat_protection.consumer_service.v1.SaveMaliciousEventRequest} - */ -public final class SaveMaliciousEventRequest extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) - SaveMaliciousEventRequestOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SaveMaliciousEventRequest.class.getName()); - } - // Use SaveMaliciousEventRequest.newBuilder() to construct. - private SaveMaliciousEventRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SaveMaliciousEventRequest() { - events_ = java.util.Collections.emptyList(); - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.Builder.class); - } - - public static final int EVENTS_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private java.util.List events_; - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - @java.lang.Override - public java.util.List getEventsList() { - return events_; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - @java.lang.Override - public java.util.List - getEventsOrBuilderList() { - return events_; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - @java.lang.Override - public int getEventsCount() { - return events_.size(); - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getEvents(int index) { - return events_.get(index); - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder getEventsOrBuilder( - int index) { - return events_.get(index); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - for (int i = 0; i < events_.size(); i++) { - output.writeMessage(2, events_.get(i)); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - for (int i = 0; i < events_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, events_.get(i)); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest other = (com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest) obj; - - if (!getEventsList() - .equals(other.getEventsList())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - if (getEventsCount() > 0) { - hash = (37 * hash) + EVENTS_FIELD_NUMBER; - hash = (53 * hash) + getEventsList().hashCode(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.consumer_service.v1.SaveMaliciousEventRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - if (eventsBuilder_ == null) { - events_ = java.util.Collections.emptyList(); - } else { - events_ = null; - eventsBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000001); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest build() { - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest buildPartial() { - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result = new com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest(this); - buildPartialRepeatedFields(result); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartialRepeatedFields(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result) { - if (eventsBuilder_ == null) { - if (((bitField0_ & 0x00000001) != 0)) { - events_ = java.util.Collections.unmodifiableList(events_); - bitField0_ = (bitField0_ & ~0x00000001); - } - result.events_ = events_; - } else { - result.events_ = eventsBuilder_.build(); - } - } - - private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest result) { - int from_bitField0_ = bitField0_; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest) { - return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest other) { - if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest.getDefaultInstance()) return this; - if (eventsBuilder_ == null) { - if (!other.events_.isEmpty()) { - if (events_.isEmpty()) { - events_ = other.events_; - bitField0_ = (bitField0_ & ~0x00000001); - } else { - ensureEventsIsMutable(); - events_.addAll(other.events_); - } - onChanged(); - } - } else { - if (!other.events_.isEmpty()) { - if (eventsBuilder_.isEmpty()) { - eventsBuilder_.dispose(); - eventsBuilder_ = null; - events_ = other.events_; - bitField0_ = (bitField0_ & ~0x00000001); - eventsBuilder_ = - com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getEventsFieldBuilder() : null; - } else { - eventsBuilder_.addAllMessages(other.events_); - } - } - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 18: { - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent m = - input.readMessage( - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.parser(), - extensionRegistry); - if (eventsBuilder_ == null) { - ensureEventsIsMutable(); - events_.add(m); - } else { - eventsBuilder_.addMessage(m); - } - break; - } // case 18 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.util.List events_ = - java.util.Collections.emptyList(); - private void ensureEventsIsMutable() { - if (!((bitField0_ & 0x00000001) != 0)) { - events_ = new java.util.ArrayList(events_); - bitField0_ |= 0x00000001; - } - } - - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder> eventsBuilder_; - - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public java.util.List getEventsList() { - if (eventsBuilder_ == null) { - return java.util.Collections.unmodifiableList(events_); - } else { - return eventsBuilder_.getMessageList(); - } - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public int getEventsCount() { - if (eventsBuilder_ == null) { - return events_.size(); - } else { - return eventsBuilder_.getCount(); - } - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getEvents(int index) { - if (eventsBuilder_ == null) { - return events_.get(index); - } else { - return eventsBuilder_.getMessage(index); - } - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder setEvents( - int index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent value) { - if (eventsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureEventsIsMutable(); - events_.set(index, value); - onChanged(); - } else { - eventsBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder setEvents( - int index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder builderForValue) { - if (eventsBuilder_ == null) { - ensureEventsIsMutable(); - events_.set(index, builderForValue.build()); - onChanged(); - } else { - eventsBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder addEvents(com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent value) { - if (eventsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureEventsIsMutable(); - events_.add(value); - onChanged(); - } else { - eventsBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder addEvents( - int index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent value) { - if (eventsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureEventsIsMutable(); - events_.add(index, value); - onChanged(); - } else { - eventsBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder addEvents( - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder builderForValue) { - if (eventsBuilder_ == null) { - ensureEventsIsMutable(); - events_.add(builderForValue.build()); - onChanged(); - } else { - eventsBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder addEvents( - int index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder builderForValue) { - if (eventsBuilder_ == null) { - ensureEventsIsMutable(); - events_.add(index, builderForValue.build()); - onChanged(); - } else { - eventsBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder addAllEvents( - java.lang.Iterable values) { - if (eventsBuilder_ == null) { - ensureEventsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, events_); - onChanged(); - } else { - eventsBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder clearEvents() { - if (eventsBuilder_ == null) { - events_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - } else { - eventsBuilder_.clear(); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public Builder removeEvents(int index) { - if (eventsBuilder_ == null) { - ensureEventsIsMutable(); - events_.remove(index); - onChanged(); - } else { - eventsBuilder_.remove(index); - } - return this; - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder getEventsBuilder( - int index) { - return getEventsFieldBuilder().getBuilder(index); - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder getEventsOrBuilder( - int index) { - if (eventsBuilder_ == null) { - return events_.get(index); } else { - return eventsBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public java.util.List - getEventsOrBuilderList() { - if (eventsBuilder_ != null) { - return eventsBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(events_); - } - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder addEventsBuilder() { - return getEventsFieldBuilder().addBuilder( - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.getDefaultInstance()); - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder addEventsBuilder( - int index) { - return getEventsFieldBuilder().addBuilder( - index, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.getDefaultInstance()); - } - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - public java.util.List - getEventsBuilderList() { - return getEventsFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder> - getEventsFieldBuilder() { - if (eventsBuilder_ == null) { - eventsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder>( - events_, - ((bitField0_ & 0x00000001) != 0), - getParentForChildren(), - isClean()); - events_ = null; - } - return eventsBuilder_; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) - private static final com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest(); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SaveMaliciousEventRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java deleted file mode 100644 index 2cbb8407ea..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventRequestOrBuilder.java +++ /dev/null @@ -1,35 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -public interface SaveMaliciousEventRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveMaliciousEventRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - java.util.List - getEventsList(); - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent getEvents(int index); - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - int getEventsCount(); - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - java.util.List - getEventsOrBuilderList(); - /** - * repeated .threat_protection.consumer_service.v1.MaliciousEvent events = 2 [json_name = "events"]; - */ - com.akto.proto.threat_protection.consumer_service.v1.MaliciousEventOrBuilder getEventsOrBuilder( - int index); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponse.java deleted file mode 100644 index c4c28a8bba..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponse.java +++ /dev/null @@ -1,358 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -/** - * Protobuf type {@code threat_protection.consumer_service.v1.SaveMaliciousEventResponse} - */ -public final class SaveMaliciousEventResponse extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) - SaveMaliciousEventResponseOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SaveMaliciousEventResponse.class.getName()); - } - // Use SaveMaliciousEventResponse.newBuilder() to construct. - private SaveMaliciousEventResponse(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SaveMaliciousEventResponse() { - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.Builder.class); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse other = (com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse) obj; - - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.consumer_service.v1.SaveMaliciousEventResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.class, com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveMaliciousEventResponse_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse getDefaultInstanceForType() { - return com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse build() { - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse buildPartial() { - com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse result = new com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse(this); - onBuilt(); - return result; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse) { - return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse other) { - if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse.getDefaultInstance()) return this; - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) - } - - // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) - private static final com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse(); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SaveMaliciousEventResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponseOrBuilder.java deleted file mode 100644 index 9c481b6591..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveMaliciousEventResponseOrBuilder.java +++ /dev/null @@ -1,11 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -public interface SaveMaliciousEventResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveMaliciousEventResponse) - com.google.protobuf.MessageOrBuilder { -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java deleted file mode 100644 index bb4f037719..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequest.java +++ /dev/null @@ -1,558 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -/** - * Protobuf type {@code threat_protection.consumer_service.v1.SaveSmartEventRequest} - */ -public final class SaveSmartEventRequest extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SaveSmartEventRequest) - SaveSmartEventRequestOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SaveSmartEventRequest.class.getName()); - } - // Use SaveSmartEventRequest.newBuilder() to construct. - private SaveSmartEventRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SaveSmartEventRequest() { - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.Builder.class); - } - - private int bitField0_; - public static final int EVENT_FIELD_NUMBER = 2; - private com.akto.proto.threat_protection.consumer_service.v1.SmartEvent event_; - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - * @return Whether the event field is set. - */ - @java.lang.Override - public boolean hasEvent() { - return ((bitField0_ & 0x00000001) != 0); - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - * @return The event. - */ - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getEvent() { - return event_ == null ? com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance() : event_; - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder getEventOrBuilder() { - return event_ == null ? com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance() : event_; - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (((bitField0_ & 0x00000001) != 0)) { - output.writeMessage(2, getEvent()); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (((bitField0_ & 0x00000001) != 0)) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getEvent()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest other = (com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest) obj; - - if (hasEvent() != other.hasEvent()) return false; - if (hasEvent()) { - if (!getEvent() - .equals(other.getEvent())) return false; - } - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - if (hasEvent()) { - hash = (37 * hash) + EVENT_FIELD_NUMBER; - hash = (53 * hash) + getEvent().hashCode(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.consumer_service.v1.SaveSmartEventRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SaveSmartEventRequest) - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.class, com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage - .alwaysUseFieldBuilders) { - getEventFieldBuilder(); - } - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - event_ = null; - if (eventBuilder_ != null) { - eventBuilder_.dispose(); - eventBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest build() { - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest buildPartial() { - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest result = new com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest result) { - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.event_ = eventBuilder_ == null - ? event_ - : eventBuilder_.build(); - to_bitField0_ |= 0x00000001; - } - result.bitField0_ |= to_bitField0_; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest) { - return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest other) { - if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest.getDefaultInstance()) return this; - if (other.hasEvent()) { - mergeEvent(other.getEvent()); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 18: { - input.readMessage( - getEventFieldBuilder().getBuilder(), - extensionRegistry); - bitField0_ |= 0x00000001; - break; - } // case 18 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private com.akto.proto.threat_protection.consumer_service.v1.SmartEvent event_; - private com.google.protobuf.SingleFieldBuilder< - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder> eventBuilder_; - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - * @return Whether the event field is set. - */ - public boolean hasEvent() { - return ((bitField0_ & 0x00000001) != 0); - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - * @return The event. - */ - public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getEvent() { - if (eventBuilder_ == null) { - return event_ == null ? com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance() : event_; - } else { - return eventBuilder_.getMessage(); - } - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - public Builder setEvent(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent value) { - if (eventBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - event_ = value; - } else { - eventBuilder_.setMessage(value); - } - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - public Builder setEvent( - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder builderForValue) { - if (eventBuilder_ == null) { - event_ = builderForValue.build(); - } else { - eventBuilder_.setMessage(builderForValue.build()); - } - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - public Builder mergeEvent(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent value) { - if (eventBuilder_ == null) { - if (((bitField0_ & 0x00000001) != 0) && - event_ != null && - event_ != com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance()) { - getEventBuilder().mergeFrom(value); - } else { - event_ = value; - } - } else { - eventBuilder_.mergeFrom(value); - } - if (event_ != null) { - bitField0_ |= 0x00000001; - onChanged(); - } - return this; - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - public Builder clearEvent() { - bitField0_ = (bitField0_ & ~0x00000001); - event_ = null; - if (eventBuilder_ != null) { - eventBuilder_.dispose(); - eventBuilder_ = null; - } - onChanged(); - return this; - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder getEventBuilder() { - bitField0_ |= 0x00000001; - onChanged(); - return getEventFieldBuilder().getBuilder(); - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - public com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder getEventOrBuilder() { - if (eventBuilder_ != null) { - return eventBuilder_.getMessageOrBuilder(); - } else { - return event_ == null ? - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance() : event_; - } - } - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - private com.google.protobuf.SingleFieldBuilder< - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder> - getEventFieldBuilder() { - if (eventBuilder_ == null) { - eventBuilder_ = new com.google.protobuf.SingleFieldBuilder< - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder, com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder>( - getEvent(), - getParentForChildren(), - isClean()); - event_ = null; - } - return eventBuilder_; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SaveSmartEventRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SaveSmartEventRequest) - private static final com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest(); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SaveSmartEventRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java deleted file mode 100644 index 5fcd871a80..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventRequestOrBuilder.java +++ /dev/null @@ -1,26 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -public interface SaveSmartEventRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveSmartEventRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - * @return Whether the event field is set. - */ - boolean hasEvent(); - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - * @return The event. - */ - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getEvent(); - /** - * .threat_protection.consumer_service.v1.SmartEvent event = 2 [json_name = "event"]; - */ - com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder getEventOrBuilder(); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponse.java deleted file mode 100644 index 809931e36a..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponse.java +++ /dev/null @@ -1,358 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -/** - * Protobuf type {@code threat_protection.consumer_service.v1.SaveSmartEventResponse} - */ -public final class SaveSmartEventResponse extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SaveSmartEventResponse) - SaveSmartEventResponseOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SaveSmartEventResponse.class.getName()); - } - // Use SaveSmartEventResponse.newBuilder() to construct. - private SaveSmartEventResponse(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SaveSmartEventResponse() { - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.class, com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.Builder.class); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse other = (com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse) obj; - - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.consumer_service.v1.SaveSmartEventResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SaveSmartEventResponse) - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.class, com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SaveSmartEventResponse_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse getDefaultInstanceForType() { - return com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse build() { - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse buildPartial() { - com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse result = new com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse(this); - onBuilt(); - return result; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse) { - return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse other) { - if (other == com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse.getDefaultInstance()) return this; - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SaveSmartEventResponse) - } - - // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SaveSmartEventResponse) - private static final com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse(); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SaveSmartEventResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponseOrBuilder.java deleted file mode 100644 index a551db5627..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SaveSmartEventResponseOrBuilder.java +++ /dev/null @@ -1,11 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -public interface SaveSmartEventResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SaveSmartEventResponse) - com.google.protobuf.MessageOrBuilder { -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEvent.java deleted file mode 100644 index 60d42cb17d..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEvent.java +++ /dev/null @@ -1,840 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -/** - * Protobuf type {@code threat_protection.consumer_service.v1.SmartEvent} - */ -public final class SmartEvent extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.consumer_service.v1.SmartEvent) - SmartEventOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SmartEvent.class.getName()); - } - // Use SmartEvent.newBuilder() to construct. - private SmartEvent(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SmartEvent() { - actorId_ = ""; - filterId_ = ""; - ruleId_ = ""; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.class, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder.class); - } - - public static final int ACTOR_ID_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object actorId_ = ""; - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The actorId. - */ - @java.lang.Override - public java.lang.String getActorId() { - java.lang.Object ref = actorId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actorId_ = s; - return s; - } - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The bytes for actorId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getActorIdBytes() { - java.lang.Object ref = actorId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actorId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int DETECTED_AT_FIELD_NUMBER = 3; - private long detectedAt_ = 0L; - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - - public static final int RULE_ID_FIELD_NUMBER = 4; - @SuppressWarnings("serial") - private volatile java.lang.Object ruleId_ = ""; - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The ruleId. - */ - @java.lang.Override - public java.lang.String getRuleId() { - java.lang.Object ref = ruleId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ruleId_ = s; - return s; - } - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The bytes for ruleId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getRuleIdBytes() { - java.lang.Object ref = ruleId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ruleId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actorId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, actorId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); - } - if (detectedAt_ != 0L) { - output.writeInt64(3, detectedAt_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ruleId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 4, ruleId_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actorId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actorId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); - } - if (detectedAt_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(3, detectedAt_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ruleId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(4, ruleId_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.consumer_service.v1.SmartEvent)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent other = (com.akto.proto.threat_protection.consumer_service.v1.SmartEvent) obj; - - if (!getActorId() - .equals(other.getActorId())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (getDetectedAt() - != other.getDetectedAt()) return false; - if (!getRuleId() - .equals(other.getRuleId())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACTOR_ID_FIELD_NUMBER; - hash = (53 * hash) + getActorId().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getDetectedAt()); - hash = (37 * hash) + RULE_ID_FIELD_NUMBER; - hash = (53 * hash) + getRuleId().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.consumer_service.v1.SmartEvent} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.consumer_service.v1.SmartEvent) - com.akto.proto.threat_protection.consumer_service.v1.SmartEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.class, com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - actorId_ = ""; - filterId_ = ""; - detectedAt_ = 0L; - ruleId_ = ""; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_consumer_service_v1_SmartEvent_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getDefaultInstanceForType() { - return com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent build() { - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent buildPartial() { - com.akto.proto.threat_protection.consumer_service.v1.SmartEvent result = new com.akto.proto.threat_protection.consumer_service.v1.SmartEvent(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.actorId_ = actorId_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.filterId_ = filterId_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.detectedAt_ = detectedAt_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.ruleId_ = ruleId_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.consumer_service.v1.SmartEvent) { - return mergeFrom((com.akto.proto.threat_protection.consumer_service.v1.SmartEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.consumer_service.v1.SmartEvent other) { - if (other == com.akto.proto.threat_protection.consumer_service.v1.SmartEvent.getDefaultInstance()) return this; - if (!other.getActorId().isEmpty()) { - actorId_ = other.actorId_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000002; - onChanged(); - } - if (other.getDetectedAt() != 0L) { - setDetectedAt(other.getDetectedAt()); - } - if (!other.getRuleId().isEmpty()) { - ruleId_ = other.ruleId_; - bitField0_ |= 0x00000008; - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - actorId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000002; - break; - } // case 18 - case 24: { - detectedAt_ = input.readInt64(); - bitField0_ |= 0x00000004; - break; - } // case 24 - case 34: { - ruleId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000008; - break; - } // case 34 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object actorId_ = ""; - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The actorId. - */ - public java.lang.String getActorId() { - java.lang.Object ref = actorId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actorId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The bytes for actorId. - */ - public com.google.protobuf.ByteString - getActorIdBytes() { - java.lang.Object ref = actorId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actorId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @param value The actorId to set. - * @return This builder for chaining. - */ - public Builder setActorId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - actorId_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return This builder for chaining. - */ - public Builder clearActorId() { - actorId_ = getDefaultInstance().getActorId(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string actor_id = 1 [json_name = "actorId"]; - * @param value The bytes for actorId to set. - * @return This builder for chaining. - */ - public Builder setActorIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - actorId_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - private long detectedAt_ ; - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @param value The detectedAt to set. - * @return This builder for chaining. - */ - public Builder setDetectedAt(long value) { - - detectedAt_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return This builder for chaining. - */ - public Builder clearDetectedAt() { - bitField0_ = (bitField0_ & ~0x00000004); - detectedAt_ = 0L; - onChanged(); - return this; - } - - private java.lang.Object ruleId_ = ""; - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The ruleId. - */ - public java.lang.String getRuleId() { - java.lang.Object ref = ruleId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ruleId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The bytes for ruleId. - */ - public com.google.protobuf.ByteString - getRuleIdBytes() { - java.lang.Object ref = ruleId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ruleId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @param value The ruleId to set. - * @return This builder for chaining. - */ - public Builder setRuleId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ruleId_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return This builder for chaining. - */ - public Builder clearRuleId() { - ruleId_ = getDefaultInstance().getRuleId(); - bitField0_ = (bitField0_ & ~0x00000008); - onChanged(); - return this; - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @param value The bytes for ruleId to set. - * @return This builder for chaining. - */ - public Builder setRuleIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ruleId_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.consumer_service.v1.SmartEvent) - } - - // @@protoc_insertion_point(class_scope:threat_protection.consumer_service.v1.SmartEvent) - private static final com.akto.proto.threat_protection.consumer_service.v1.SmartEvent DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.consumer_service.v1.SmartEvent(); - } - - public static com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SmartEvent parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.consumer_service.v1.SmartEvent getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java deleted file mode 100644 index fc243b8af8..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/consumer_service/v1/SmartEventOrBuilder.java +++ /dev/null @@ -1,53 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.consumer_service.v1; - -public interface SmartEventOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.consumer_service.v1.SmartEvent) - com.google.protobuf.MessageOrBuilder { - - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The actorId. - */ - java.lang.String getActorId(); - /** - * string actor_id = 1 [json_name = "actorId"]; - * @return The bytes for actorId. - */ - com.google.protobuf.ByteString - getActorIdBytes(); - - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - java.lang.String getFilterId(); - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - com.google.protobuf.ByteString - getFilterIdBytes(); - - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - long getDetectedAt(); - - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The ruleId. - */ - java.lang.String getRuleId(); - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The bytes for ruleId. - */ - com.google.protobuf.ByteString - getRuleIdBytes(); -} From 68a4e24be3f1c143116ccb8ecce6ac2e0be8d257 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Mon, 25 Nov 2024 16:43:00 +0530 Subject: [PATCH 27/73] saving malicious sample data to postgres db --- apps/threat-detection/pom.xml | 6 + .../java/com/akto/threat/detection/Main.java | 47 +++-- .../detection/config/kafka/KafkaConfig.java | 12 -- .../db/malicious_event/MaliciousEventDao.java | 74 ++++++++ .../malicious_event/MaliciousEventModel.java | 163 ++++++++++++++++++ .../detection/tasks/FlushSampleDataTask.java | 106 +++++++++++- .../tasks/MaliciousTrafficDetectorTask.java | 6 +- 7 files changed, 383 insertions(+), 31 deletions(-) create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java diff --git a/apps/threat-detection/pom.xml b/apps/threat-detection/pom.xml index 053bfc7862..c239e1e8e7 100644 --- a/apps/threat-detection/pom.xml +++ b/apps/threat-detection/pom.xml @@ -95,6 +95,12 @@ compile + + org.postgresql + postgresql + 42.7.4 + + diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index 045558ccf2..6d8bdb27f6 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -1,32 +1,47 @@ package com.akto.threat.detection; +import com.akto.DaoInit; import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.config.kafka.KafkaConsumerConfig; import com.akto.threat.detection.config.kafka.KafkaProducerConfig; +import com.akto.threat.detection.tasks.FlushSampleDataTask; import com.akto.threat.detection.tasks.MaliciousTrafficDetectorTask; +import com.mongodb.ConnectionString; import io.lettuce.core.RedisClient; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; + public class Main { - public static void main(String[] args) { - new MaliciousTrafficDetectorTask( - KafkaConfig.newBuilder() - .setTopic("akto.api.logs") - .setGroupId("akto.threat.detection") - .setBootstrapServers("localhost:29092") - .setConsumerConfig( - KafkaConsumerConfig.newBuilder() - .setMaxPollRecords(100) - .setPollDurationMilli(100) - .build()) - .setProducerConfig( - KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) - .build(), - createRedisClient()) - .run(); + public static void main(String[] args) throws Exception { + DaoInit.init(new ConnectionString(System.getenv("AKTO_MONGO_CONN"))); + KafkaConfig trafficKafka = + KafkaConfig.newBuilder() + .setGroupId("akto.threat.detection") + .setBootstrapServers("localhost:29092") + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) + .build(); + + Connection postgres = createPostgresConnection(); + + new MaliciousTrafficDetectorTask(trafficKafka, createRedisClient()).run(); + new FlushSampleDataTask(postgres, trafficKafka).run(); } public static RedisClient createRedisClient() { return RedisClient.create(System.getenv("AKTO_THREAT_DETECTION_REDIS_URI")); } + + public static Connection createPostgresConnection() throws SQLException { + String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); + return DriverManager.getConnection(url); + } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java index dd1210ad1f..c50d78314e 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java @@ -1,14 +1,12 @@ package com.akto.threat.detection.config.kafka; public class KafkaConfig { - private final String topic; private final String bootstrapServers; private final String groupId; private final KafkaConsumerConfig consumerConfig; private final KafkaProducerConfig producerConfig; public static class Builder { - private String topic; private String bootstrapServers; private String groupId; private KafkaConsumerConfig consumerConfig; @@ -16,11 +14,6 @@ public static class Builder { private Builder() {} - public Builder setTopic(String topic) { - this.topic = topic; - return this; - } - public Builder setBootstrapServers(String bootstrapServers) { this.bootstrapServers = bootstrapServers; return this; @@ -47,17 +40,12 @@ public KafkaConfig build() { } private KafkaConfig(Builder builder) { - this.topic = builder.topic; this.bootstrapServers = builder.bootstrapServers; this.groupId = builder.groupId; this.consumerConfig = builder.consumerConfig; this.producerConfig = builder.producerConfig; } - public String getTopic() { - return topic; - } - public String getBootstrapServers() { return bootstrapServers; } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java new file mode 100644 index 0000000000..5e563cca44 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java @@ -0,0 +1,74 @@ +package com.akto.threat.detection.db.malicious_event; + +import com.akto.dto.type.URLMethods; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.List; +import java.util.Optional; + +public class MaliciousEventDao { + + private final Connection conn; + private static final int BATCH_SIZE = 50; + + public MaliciousEventDao(Connection conn) { + this.conn = conn; + } + + public void batchInsert(List events) throws SQLException { + String sql = + "INSERT INTO threat_detection.malicious_event (id, actor_id, filter_id, url, method, timestamp, data, ip, country) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)"; + conn.setAutoCommit(false); + for (int i = 0; i < events.size(); i++) { + MaliciousEventModel event = events.get(i); + PreparedStatement stmt = this.conn.prepareStatement(sql); + stmt.setString(1, event.getId()); + stmt.setString(2, event.getActorId()); + stmt.setString(3, event.getFilterId()); + stmt.setString(4, event.getUrl()); + stmt.setString(5, event.getMethod().name()); + stmt.setLong(6, event.getTimestamp()); + stmt.setString(7, event.getOrig()); + stmt.setString(8, event.getIp()); + stmt.setString(9, event.getCountry()); + + stmt.addBatch(); + + if (i % BATCH_SIZE == 0 || i == events.size() - 1) { + stmt.executeBatch(); + stmt.clearBatch(); + } + } + + conn.commit(); + } + + public Optional findOne(String id) throws SQLException { + String sql = "SELECT * FROM threat_detection.malicious_event WHERE id = ?"; + PreparedStatement stmt = this.conn.prepareStatement(sql); + try (ResultSet rs = stmt.executeQuery()) { + if (rs.next()) { + MaliciousEventModel model = + MaliciousEventModel.newBuilder() + .setId(rs.getString("id")) + .setActorId(rs.getString("actor_id")) + .setFilterId(rs.getString("filter_id")) + .setUrl(rs.getString("url")) + .setMethod(URLMethods.Method.fromString(rs.getString("method"))) + .setTimestamp(rs.getLong("timestamp")) + .setOrig(rs.getString("data")) + .setIp(rs.getString("ip")) + .setCountry(rs.getString("country")) + .build(); + + return Optional.of(model); + } + } catch (SQLException e) { + e.printStackTrace(); + } + return Optional.empty(); + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java new file mode 100644 index 0000000000..87d7e2fd00 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java @@ -0,0 +1,163 @@ +package com.akto.threat.detection.db.malicious_event; + +import com.akto.dto.type.URLMethods; + +import java.util.UUID; + +public class MaliciousEventModel { + + private String id; + private String actorId; + private String filterId; + private String url; + private URLMethods.Method method; + private long timestamp; + private String orig; + + // Geo location data + private String ip; + private String country; + + public MaliciousEventModel() {} + + public MaliciousEventModel(Builder builder) { + this.id = builder.id == null ? UUID.randomUUID().toString() : builder.id; + this.actorId = builder.actorId; + this.filterId = builder.filterId; + this.url = builder.url; + this.method = builder.method; + this.timestamp = builder.timestamp; + this.orig = builder.orig; + this.ip = builder.ip; + this.country = builder.country; + } + + public static class Builder { + private String id; + private String actorId; + private String filterId; + private String url; + private URLMethods.Method method; + private long timestamp; + private String orig; + private String ip; + private String country; + + public Builder setId(String id) { + this.id = id; + return this; + } + + public Builder setActorId(String actorId) { + this.actorId = actorId; + return this; + } + + public Builder setFilterId(String filterId) { + this.filterId = filterId; + return this; + } + + public Builder setUrl(String url) { + this.url = url; + return this; + } + + public Builder setMethod(URLMethods.Method method) { + this.method = method; + return this; + } + + public Builder setTimestamp(long timestamp) { + this.timestamp = timestamp; + return this; + } + + public Builder setOrig(String orig) { + this.orig = orig; + return this; + } + + public Builder setIp(String ip) { + this.ip = ip; + return this; + } + + public Builder setCountry(String country) { + this.country = country; + return this; + } + + public MaliciousEventModel build() { + return new MaliciousEventModel(this); + } + } + + public static Builder newBuilder() { + return new Builder(); + } + + public String getId() { + return id; + } + + public String getActorId() { + return actorId; + } + + public String getFilterId() { + return filterId; + } + + public String getUrl() { + return url; + } + + public URLMethods.Method getMethod() { + return method; + } + + public long getTimestamp() { + return timestamp; + } + + public String getOrig() { + return orig; + } + + public String getIp() { + return ip; + } + + public String getCountry() { + return country; + } + + @Override + public String toString() { + return "MaliciousEventModel{" + + "ip='" + + ip + + '\'' + + ", id='" + + id + + '\'' + + ", actorId='" + + actorId + + '\'' + + ", filterId='" + + filterId + + '\'' + + ", url='" + + url + + '\'' + + ", method=" + + method + + ", timestamp=" + + timestamp + + ", country='" + + country + + '\'' + + '}'; + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 4d79780fe9..b16c017754 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -1,10 +1,114 @@ package com.akto.threat.detection.tasks; +import com.akto.dto.type.URLMethods; +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; +import com.akto.runtime.utils.Utils; +import com.akto.threat.detection.config.kafka.KafkaConfig; +import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; +import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; +import com.akto.threat.detection.dto.MaliciousMessageEnvelope; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.util.JsonFormat; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; + +import java.sql.Connection; +import java.sql.SQLException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + /* This will read sample malicious data from kafka topic and save it to DB. */ public class FlushSampleDataTask implements Task { + private final Connection conn; + private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); + private final Consumer kafkaConsumer; + private final KafkaConfig kafkaConfig; + private final MaliciousEventDao maliciousEventDao; + + public FlushSampleDataTask(Connection conn, KafkaConfig trafficConfig) { + this.conn = conn; + this.kafkaConfig = trafficConfig; + + String kafkaBrokerUrl = trafficConfig.getBootstrapServers(); + String groupId = trafficConfig.getGroupId(); + + Properties properties = + Utils.configProperties( + kafkaBrokerUrl, groupId, trafficConfig.getConsumerConfig().getMaxPollRecords()); + this.kafkaConsumer = new KafkaConsumer<>(properties); + + this.maliciousEventDao = new MaliciousEventDao(conn); + } + @Override - public void run() {} + public void run() { + this.kafkaConsumer.subscribe(Collections.singletonList("akto.malicious")); + + pollingExecutor.execute( + new Runnable() { + @Override + public void run() { + // Poll data from Kafka topic + while (true) { + ConsumerRecords records = + kafkaConsumer.poll( + Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); + if (records.isEmpty()) { + continue; + } + + processRecords(records); + } + } + }); + } + + private void processRecords(ConsumerRecords records) { + List events = new ArrayList<>(); + records.forEach( + r -> { + String message = r.value(); + MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); + MaliciousMessageEnvelope m = MaliciousMessageEnvelope.unmarshal(message).orElse(null); + if (m == null) { + return; + } + + try { + JsonFormat.parser().merge(m.getData(), builder); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + return; + } + + MaliciousEvent evt = builder.build(); + + events.add( + MaliciousEventModel.newBuilder() + .setActorId(m.getAccountId()) + .setFilterId(evt.getFilterId()) + .setUrl(evt.getUrl()) + .setMethod(URLMethods.Method.fromString(evt.getMethod())) + .setTimestamp(evt.getTimestamp()) + .setOrig(evt.getPayload()) + .setIp(evt.getIp()) + .setCountry("US") // TODO: Call maxmind to get country code for IP + .build()); + }); + + try { + this.maliciousEventDao.batchInsert(events); + } catch (SQLException e) { + e.printStackTrace(); + } + } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 73a612a521..61616b9e6c 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -1,5 +1,6 @@ package com.akto.threat.detection.tasks; +import com.akto.dao.context.Context; import com.akto.threat.detection.actor.SourceIPActorGenerator; import com.akto.threat.detection.cache.RedisBackedCounterCache; import com.akto.threat.detection.config.kafka.KafkaConfig; @@ -98,7 +99,7 @@ public MaliciousTrafficDetectorTask(KafkaConfig trafficConfig, RedisClient redis } public void run() { - this.kafkaConsumer.subscribe(Collections.singletonList(this.kafkaConfig.getTopic())); + this.kafkaConsumer.subscribe(Collections.singletonList("akto.api.logs")); pollingExecutor.execute( new Runnable() { @Override @@ -168,12 +169,13 @@ private boolean validateFilterForRequest( } private void processRecord(ConsumerRecord record) { + HttpResponseParams responseParam = HttpCallParser.parseKafkaMessage(record.value()); + Context.accountId.set(Integer.parseInt(responseParam.getAccountId())); Map filters = this.getFilters(); if (filters.isEmpty()) { return; } - HttpResponseParams responseParam = HttpCallParser.parseKafkaMessage(record.value()); List maliciousMessages = new ArrayList<>(); for (FilterConfig apiFilter : apiFilters.values()) { From f7cd9957b79baf4c08330982ac87cc5f627f5691 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 26 Nov 2024 13:01:16 +0530 Subject: [PATCH 28/73] abstracted out common kafka polling consumer task --- .../java/com/akto/threat/detection/Main.java | 2 +- ...sageEnvelope.java => MessageEnvelope.java} | 10 +-- .../tasks/AbstractKafkaConsumerTask.java | 56 ++++++++++++++ .../detection/tasks/FlushSampleDataTask.java | 54 ++----------- .../tasks/MaliciousTrafficDetectorTask.java | 10 +-- .../detection/tasks/SendAlertsToBackend.java | 75 +++++++++++++++++++ 6 files changed, 150 insertions(+), 57 deletions(-) rename apps/threat-detection/src/main/java/com/akto/threat/detection/dto/{MaliciousMessageEnvelope.java => MessageEnvelope.java} (79%) create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index 6d8bdb27f6..0e9840c165 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -33,7 +33,7 @@ public static void main(String[] args) throws Exception { Connection postgres = createPostgresConnection(); new MaliciousTrafficDetectorTask(trafficKafka, createRedisClient()).run(); - new FlushSampleDataTask(postgres, trafficKafka).run(); + new FlushSampleDataTask(postgres, trafficKafka, "akto,malicious").run(); } public static RedisClient createRedisClient() { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MaliciousMessageEnvelope.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java similarity index 79% rename from apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MaliciousMessageEnvelope.java rename to apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java index ba51bb2bcd..4960eb53f4 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MaliciousMessageEnvelope.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java @@ -5,15 +5,15 @@ import java.util.Optional; // Kafka Message Wrapper for suspect data -public class MaliciousMessageEnvelope { +public class MessageEnvelope { private String accountId; private String data; private static final ObjectMapper objectMapper = new ObjectMapper(); - public MaliciousMessageEnvelope() {} + public MessageEnvelope() {} - public MaliciousMessageEnvelope(String accountId, String data) { + public MessageEnvelope(String accountId, String data) { this.accountId = accountId; this.data = data; } @@ -44,9 +44,9 @@ public Optional marshal() { return Optional.empty(); } - public static Optional unmarshal(String message) { + public static Optional unmarshal(String message) { try { - return Optional.ofNullable(objectMapper.readValue(message, MaliciousMessageEnvelope.class)); + return Optional.ofNullable(objectMapper.readValue(message, MessageEnvelope.class)); } catch (Exception e) { e.printStackTrace(); } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java new file mode 100644 index 0000000000..9054786ba8 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java @@ -0,0 +1,56 @@ +package com.akto.threat.detection.tasks; + +import com.akto.runtime.utils.Utils; +import com.akto.threat.detection.config.kafka.KafkaConfig; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; + +import java.time.Duration; +import java.util.Collections; +import java.util.Properties; +import java.util.concurrent.ExecutorService; + +public abstract class AbstractKafkaConsumerTask implements Task { + + protected Consumer kafkaConsumer; + protected KafkaConfig kafkaConfig; + protected String kafkaTopic; + + public AbstractKafkaConsumerTask(KafkaConfig kafkaConfig, String kafkaTopic) { + this.kafkaTopic = kafkaTopic; + + String kafkaBrokerUrl = kafkaConfig.getBootstrapServers(); + String groupId = kafkaConfig.getGroupId(); + + Properties properties = + Utils.configProperties( + kafkaBrokerUrl, groupId, kafkaConfig.getConsumerConfig().getMaxPollRecords()); + this.kafkaConsumer = new KafkaConsumer<>(properties); + } + + abstract ExecutorService getPollingExecutor(); + + @Override + public void run() { + this.kafkaConsumer.subscribe(Collections.singletonList(this.kafkaTopic)); + + this.getPollingExecutor() + .execute( + () -> { + // Poll data from Kafka topic + while (true) { + ConsumerRecords records = + kafkaConsumer.poll( + Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); + if (records.isEmpty()) { + continue; + } + + processRecords(records); + } + }); + } + + abstract void processRecords(ConsumerRecords records); +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index b16c017754..c17edff5ca 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -2,83 +2,45 @@ import com.akto.dto.type.URLMethods; import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; -import com.akto.runtime.utils.Utils; import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; -import com.akto.threat.detection.dto.MaliciousMessageEnvelope; +import com.akto.threat.detection.dto.MessageEnvelope; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; -import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; import java.sql.Connection; import java.sql.SQLException; -import java.time.Duration; import java.util.ArrayList; -import java.util.Collections; import java.util.List; -import java.util.Properties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; /* This will read sample malicious data from kafka topic and save it to DB. */ -public class FlushSampleDataTask implements Task { +public class FlushSampleDataTask extends AbstractKafkaConsumerTask { - private final Connection conn; - private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); - private final Consumer kafkaConsumer; - private final KafkaConfig kafkaConfig; private final MaliciousEventDao maliciousEventDao; - public FlushSampleDataTask(Connection conn, KafkaConfig trafficConfig) { - this.conn = conn; - this.kafkaConfig = trafficConfig; - - String kafkaBrokerUrl = trafficConfig.getBootstrapServers(); - String groupId = trafficConfig.getGroupId(); - - Properties properties = - Utils.configProperties( - kafkaBrokerUrl, groupId, trafficConfig.getConsumerConfig().getMaxPollRecords()); - this.kafkaConsumer = new KafkaConsumer<>(properties); - + public FlushSampleDataTask(Connection conn, KafkaConfig trafficConfig, String topic) { + super(trafficConfig, topic); this.maliciousEventDao = new MaliciousEventDao(conn); } @Override - public void run() { - this.kafkaConsumer.subscribe(Collections.singletonList("akto.malicious")); - - pollingExecutor.execute( - new Runnable() { - @Override - public void run() { - // Poll data from Kafka topic - while (true) { - ConsumerRecords records = - kafkaConsumer.poll( - Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); - if (records.isEmpty()) { - continue; - } - - processRecords(records); - } - } - }); + ExecutorService getPollingExecutor() { + return Executors.newSingleThreadExecutor(); } - private void processRecords(ConsumerRecords records) { + protected void processRecords(ConsumerRecords records) { List events = new ArrayList<>(); records.forEach( r -> { String message = r.value(); MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); - MaliciousMessageEnvelope m = MaliciousMessageEnvelope.unmarshal(message).orElse(null); + MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); if (m == null) { return; } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 61616b9e6c..ed69c5497d 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -9,7 +9,7 @@ import com.akto.data_actor.DataActorFactory; import com.akto.dto.ApiInfo; import com.akto.dto.HttpResponseParams; -import com.akto.threat.detection.dto.MaliciousMessageEnvelope; +import com.akto.threat.detection.dto.MessageEnvelope; import com.akto.dto.RawApi; import com.akto.dto.api_protection_parse_layer.AggregationRules; import com.akto.dto.api_protection_parse_layer.Condition; @@ -129,10 +129,10 @@ private Map getFilters() { return apiFilters; } - private MaliciousMessageEnvelope generateKafkaMessage( - String accountId, MaliciousEvent maliciousEvent) throws InvalidProtocolBufferException { + private MessageEnvelope generateKafkaMessage(String accountId, MaliciousEvent maliciousEvent) + throws InvalidProtocolBufferException { String data = JsonFormat.printer().print(maliciousEvent); - return new MaliciousMessageEnvelope(accountId, data); + return new MessageEnvelope(accountId, data); } private boolean validateFilterForRequest( @@ -176,7 +176,7 @@ private void processRecord(ConsumerRecord record) { return; } - List maliciousMessages = new ArrayList<>(); + List maliciousMessages = new ArrayList<>(); for (FilterConfig apiFilter : apiFilters.values()) { boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java new file mode 100644 index 0000000000..3428b14070 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java @@ -0,0 +1,75 @@ +package com.akto.threat.detection.tasks; + +import com.akto.dto.type.URLMethods; +import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; +import com.akto.threat.detection.config.kafka.KafkaConfig; +import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; +import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; +import com.akto.threat.detection.dto.MessageEnvelope; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.util.JsonFormat; +import org.apache.kafka.clients.consumer.ConsumerRecords; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/* +This will read sample malicious data from kafka topic and save it to DB. + */ +public class SendAlertsToBackend extends AbstractKafkaConsumerTask { + + private final MaliciousEventDao maliciousEventDao; + + public SendAlertsToBackend(Connection conn, KafkaConfig trafficConfig, String topic) { + super(trafficConfig, topic); + this.maliciousEventDao = new MaliciousEventDao(conn); + } + + ExecutorService getPollingExecutor() { + return Executors.newSingleThreadExecutor(); + } + + protected void processRecords(ConsumerRecords records) { + List events = new ArrayList<>(); + records.forEach( + r -> { + String message = r.value(); + MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); + MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); + if (m == null) { + return; + } + + try { + JsonFormat.parser().merge(m.getData(), builder); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + return; + } + + MaliciousEvent evt = builder.build(); + + events.add( + MaliciousEventModel.newBuilder() + .setActorId(m.getAccountId()) + .setFilterId(evt.getFilterId()) + .setUrl(evt.getUrl()) + .setMethod(URLMethods.Method.fromString(evt.getMethod())) + .setTimestamp(evt.getTimestamp()) + .setOrig(evt.getPayload()) + .setIp(evt.getIp()) + .setCountry("US") // TODO: Call maxmind to get country code for IP + .build()); + }); + + try { + this.maliciousEventDao.batchInsert(events); + } catch (SQLException e) { + e.printStackTrace(); + } + } +} From ad5b13b9a9f88547b29d65a8091ec5a88718edbf Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Thu, 28 Nov 2024 15:39:53 +0530 Subject: [PATCH 29/73] added tasks for sending alerts to protection backend --- .../java/com/akto/threat/detection/Main.java | 19 +++- .../db/malicious_event/MaliciousEventDao.java | 36 +++++-- .../malicious_event/MaliciousEventModel.java | 24 ++--- .../threat/detection/dto/MessageEnvelope.java | 9 ++ .../detection/tasks/FlushSampleDataTask.java | 1 - .../tasks/MaliciousTrafficDetectorTask.java | 95 +++++++------------ .../detection/tasks/SendAlertsToBackend.java | 89 ++++++++++++----- .../v1/consumer_service.proto | 19 ++-- 8 files changed, 163 insertions(+), 129 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index 0e9840c165..729caa1a6c 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -6,6 +6,7 @@ import com.akto.threat.detection.config.kafka.KafkaProducerConfig; import com.akto.threat.detection.tasks.FlushSampleDataTask; import com.akto.threat.detection.tasks.MaliciousTrafficDetectorTask; +import com.akto.threat.detection.tasks.SendAlertsToBackend; import com.mongodb.ConnectionString; import io.lettuce.core.RedisClient; @@ -30,10 +31,24 @@ public static void main(String[] args) throws Exception { KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) .build(); + KafkaConfig internalKafka = + KafkaConfig.newBuilder() + .setGroupId("akto.threat.detection") + .setBootstrapServers("localhost:29092") + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) + .build(); + Connection postgres = createPostgresConnection(); - new MaliciousTrafficDetectorTask(trafficKafka, createRedisClient()).run(); - new FlushSampleDataTask(postgres, trafficKafka, "akto,malicious").run(); + new MaliciousTrafficDetectorTask(trafficKafka, internalKafka, createRedisClient()).run(); + new FlushSampleDataTask(postgres, internalKafka, "akto,malicious").run(); + new SendAlertsToBackend(postgres, internalKafka, "akto.smart_event").run(); } public static RedisClient createRedisClient() { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java index 5e563cca44..9b3e1736cc 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java @@ -6,8 +6,8 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; +import java.util.ArrayList; import java.util.List; -import java.util.Optional; public class MaliciousEventDao { @@ -33,7 +33,6 @@ public void batchInsert(List events) throws SQLException { stmt.setLong(6, event.getTimestamp()); stmt.setString(7, event.getOrig()); stmt.setString(8, event.getIp()); - stmt.setString(9, event.getCountry()); stmt.addBatch(); @@ -46,11 +45,17 @@ public void batchInsert(List events) throws SQLException { conn.commit(); } - public Optional findOne(String id) throws SQLException { - String sql = "SELECT * FROM threat_detection.malicious_event WHERE id = ?"; + public List findGivenActorIdAndFilterId( + String actor, String filterId, int limit) throws SQLException { + String sql = + "SELECT * FROM threat_detection.malicious_event WHERE actor_id = ? AND filter_id = ? LIMIT ?"; PreparedStatement stmt = this.conn.prepareStatement(sql); + stmt.setString(1, actor); + stmt.setString(2, filterId); + stmt.setInt(3, limit); try (ResultSet rs = stmt.executeQuery()) { - if (rs.next()) { + List models = new ArrayList<>(); + while (rs.next()) { MaliciousEventModel model = MaliciousEventModel.newBuilder() .setId(rs.getString("id")) @@ -61,14 +66,25 @@ public Optional findOne(String id) throws SQLException { .setTimestamp(rs.getLong("timestamp")) .setOrig(rs.getString("data")) .setIp(rs.getString("ip")) - .setCountry(rs.getString("country")) .build(); + models.add(model); + } + return models; + } + } - return Optional.of(model); + public int countTotalMaliciousEventGivenActorIdAndFilterId(String actor, String filterId) + throws SQLException { + String sql = + "SELECT COUNT(*) FROM threat_detection.malicious_event WHERE actor_id = ? AND filter_id = ?"; + PreparedStatement stmt = this.conn.prepareStatement(sql); + stmt.setString(1, actor); + stmt.setString(2, filterId); + try (ResultSet rs = stmt.executeQuery()) { + if (rs.next()) { + return rs.getInt(1); } - } catch (SQLException e) { - e.printStackTrace(); } - return Optional.empty(); + return 0; } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java index 87d7e2fd00..aa15837261 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java @@ -16,7 +16,6 @@ public class MaliciousEventModel { // Geo location data private String ip; - private String country; public MaliciousEventModel() {} @@ -29,7 +28,6 @@ public MaliciousEventModel(Builder builder) { this.timestamp = builder.timestamp; this.orig = builder.orig; this.ip = builder.ip; - this.country = builder.country; } public static class Builder { @@ -41,7 +39,6 @@ public static class Builder { private long timestamp; private String orig; private String ip; - private String country; public Builder setId(String id) { this.id = id; @@ -83,11 +80,6 @@ public Builder setIp(String ip) { return this; } - public Builder setCountry(String country) { - this.country = country; - return this; - } - public MaliciousEventModel build() { return new MaliciousEventModel(this); } @@ -129,17 +121,10 @@ public String getIp() { return ip; } - public String getCountry() { - return country; - } - @Override public String toString() { return "MaliciousEventModel{" - + "ip='" - + ip - + '\'' - + ", id='" + + "id='" + id + '\'' + ", actorId='" @@ -155,8 +140,11 @@ public String toString() { + method + ", timestamp=" + timestamp - + ", country='" - + country + + ", orig='" + + orig + + '\'' + + ", ip='" + + ip + '\'' + '}'; } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java index 4960eb53f4..2ac2535d9b 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java @@ -1,6 +1,9 @@ package com.akto.threat.detection.dto; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Message; +import com.google.protobuf.util.JsonFormat; import java.util.Optional; @@ -53,4 +56,10 @@ public static Optional unmarshal(String message) { return Optional.empty(); } + + public static MessageEnvelope generateEnvelope(String accountId, Message msg) + throws InvalidProtocolBufferException { + String data = JsonFormat.printer().print(msg); + return new MessageEnvelope(accountId, data); + } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index c17edff5ca..03f619a1d0 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -63,7 +63,6 @@ protected void processRecords(ConsumerRecords records) { .setTimestamp(evt.getTimestamp()) .setOrig(evt.getPayload()) .setIp(evt.getIp()) - .setCountry("US") // TODO: Call maxmind to get country code for IP .build()); }); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index ed69c5497d..4b4e8d597a 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -17,7 +17,6 @@ import com.akto.dto.monitoring.FilterConfig; import com.akto.dto.test_editor.YamlTemplate; import com.akto.dto.type.URLMethods; -import com.akto.threat.detection.grpc.AuthToken; import com.akto.hybrid_parsers.HttpCallParser; import com.akto.kafka.Kafka; import com.akto.proto.threat_protection.consumer_service.v1.*; @@ -27,16 +26,11 @@ import com.akto.test_editor.execution.VariableResolver; import com.akto.test_editor.filter.data_operands_impl.ValidationResult; import com.google.protobuf.InvalidProtocolBufferException; -import io.grpc.Grpc; -import io.grpc.InsecureChannelCredentials; -import io.grpc.ManagedChannel; -import io.grpc.stub.StreamObserver; import io.lettuce.core.RedisClient; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; -import com.google.protobuf.util.JsonFormat; import java.time.Duration; import java.util.*; @@ -58,23 +52,24 @@ public class MaliciousTrafficDetectorTask { private Map apiFilters; private int filterLastUpdatedAt = 0; - private final Kafka kafka; + private final Kafka internalKafka; private static final String KAFKA_MALICIOUS_TOPIC = "akto.malicious"; + private static final String KAFKA_SMART_EVENT_TOPIC = "akto.smart_event"; private static final DataActor dataActor = DataActorFactory.fetchInstance(); - private final ConsumerServiceGrpc.ConsumerServiceStub consumerServiceStub; - public MaliciousTrafficDetectorTask(KafkaConfig trafficConfig, RedisClient redisClient) { + public MaliciousTrafficDetectorTask( + KafkaConfig trafficConfig, KafkaConfig internalConfig, RedisClient redisClient) { this.kafkaConfig = trafficConfig; String kafkaBrokerUrl = trafficConfig.getBootstrapServers(); String groupId = trafficConfig.getGroupId(); - Properties properties = - Utils.configProperties( - kafkaBrokerUrl, groupId, trafficConfig.getConsumerConfig().getMaxPollRecords()); - this.kafkaConsumer = new KafkaConsumer<>(properties); + this.kafkaConsumer = + new KafkaConsumer<>( + Utils.configProperties( + kafkaBrokerUrl, groupId, trafficConfig.getConsumerConfig().getMaxPollRecords())); this.httpCallParser = new HttpCallParser(120, 1000); @@ -83,35 +78,24 @@ public MaliciousTrafficDetectorTask(KafkaConfig trafficConfig, RedisClient redis new RedisBackedCounterCache(redisClient, "wbt"), new WindowBasedThresholdNotifier.Config(100, 10 * 60)); - String target = "localhost:8980"; - ManagedChannel channel = - Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - this.consumerServiceStub = - ConsumerServiceGrpc.newStub(channel) - .withCallCredentials( - new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); - - this.kafka = + this.internalKafka = new Kafka( - kafkaBrokerUrl, - trafficConfig.getProducerConfig().getLingerMs(), - trafficConfig.getProducerConfig().getBatchSize()); + internalConfig.getBootstrapServers(), + internalConfig.getProducerConfig().getLingerMs(), + internalConfig.getProducerConfig().getBatchSize()); } public void run() { this.kafkaConsumer.subscribe(Collections.singletonList("akto.api.logs")); pollingExecutor.execute( - new Runnable() { - @Override - public void run() { - // Poll data from Kafka topic - while (true) { - ConsumerRecords records = - kafkaConsumer.poll( - Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); - for (ConsumerRecord record : records) { - processRecord(record); - } + () -> { + // Poll data from Kafka topic + while (true) { + ConsumerRecords records = + kafkaConsumer.poll( + Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); + for (ConsumerRecord record : records) { + processRecord(record); } } }); @@ -129,12 +113,6 @@ private Map getFilters() { return apiFilters; } - private MessageEnvelope generateKafkaMessage(String accountId, MaliciousEvent maliciousEvent) - throws InvalidProtocolBufferException { - String data = JsonFormat.printer().print(maliciousEvent); - return new MessageEnvelope(accountId, data); - } - private boolean validateFilterForRequest( HttpResponseParams responseParam, FilterConfig apiFilter) { try { @@ -216,7 +194,8 @@ private void processRecord(ConsumerRecord record) { try { maliciousMessages.add( - generateKafkaMessage(responseParam.getAccountId(), maliciousEvent)); + MessageEnvelope.generateEnvelope( + responseParam.getAccountId(), maliciousEvent)); } catch (InvalidProtocolBufferException e) { return; } @@ -234,24 +213,16 @@ private void processRecord(ConsumerRecord record) { .setDetectedAt(responseParam.getTime()) .setRuleId(rule.getName()) .build(); - this.consumerServiceStub.saveSmartEvent( - SaveSmartEventRequest.newBuilder().setEvent(smartEvent).build(), - new StreamObserver() { - @Override - public void onNext(SaveSmartEventResponse value) { - // Do nothing - } - - @Override - public void onError(Throwable t) { - t.printStackTrace(); - } - - @Override - public void onCompleted() { - // Do nothing - } - }); + try { + MessageEnvelope.generateEnvelope(responseParam.getAccountId(), smartEvent) + .marshal() + .ifPresent( + data -> { + internalKafka.send(data, KAFKA_SMART_EVENT_TOPIC); + }); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } } } }); @@ -267,7 +238,7 @@ public void onCompleted() { .marshal() .ifPresent( data -> { - kafka.send(data, KAFKA_MALICIOUS_TOPIC); + internalKafka.send(data, KAFKA_MALICIOUS_TOPIC); }); }); } catch (Exception e) { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java index 3428b14070..1357b9adc0 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java @@ -1,21 +1,25 @@ package com.akto.threat.detection.tasks; -import com.akto.dto.type.URLMethods; -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; +import com.akto.proto.threat_protection.consumer_service.v1.*; import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; import com.akto.threat.detection.dto.MessageEnvelope; +import com.akto.threat.detection.grpc.AuthToken; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import io.grpc.stub.StreamObserver; import org.apache.kafka.clients.consumer.ConsumerRecords; import java.sql.Connection; -import java.sql.SQLException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.stream.Collectors; /* This will read sample malicious data from kafka topic and save it to DB. @@ -24,9 +28,19 @@ public class SendAlertsToBackend extends AbstractKafkaConsumerTask { private final MaliciousEventDao maliciousEventDao; + private final ConsumerServiceGrpc.ConsumerServiceStub consumerServiceStub; + public SendAlertsToBackend(Connection conn, KafkaConfig trafficConfig, String topic) { super(trafficConfig, topic); this.maliciousEventDao = new MaliciousEventDao(conn); + + String target = "localhost:8980"; + ManagedChannel channel = + Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); + this.consumerServiceStub = + ConsumerServiceGrpc.newStub(channel) + .withCallCredentials( + new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); } ExecutorService getPollingExecutor() { @@ -38,7 +52,7 @@ protected void processRecords(ConsumerRecords records) { records.forEach( r -> { String message = r.value(); - MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); + SmartEvent.Builder builder = SmartEvent.newBuilder(); MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); if (m == null) { return; @@ -51,25 +65,54 @@ protected void processRecords(ConsumerRecords records) { return; } - MaliciousEvent evt = builder.build(); - - events.add( - MaliciousEventModel.newBuilder() - .setActorId(m.getAccountId()) - .setFilterId(evt.getFilterId()) - .setUrl(evt.getUrl()) - .setMethod(URLMethods.Method.fromString(evt.getMethod())) - .setTimestamp(evt.getTimestamp()) - .setOrig(evt.getPayload()) - .setIp(evt.getIp()) - .setCountry("US") // TODO: Call maxmind to get country code for IP - .build()); - }); + SmartEvent evt = builder.build(); + + // Get sample data from postgres for this alert + try { + List sampleData = + this.maliciousEventDao.findGivenActorIdAndFilterId( + evt.getActorId(), evt.getFilterId(), 50); + + int totalEvents = + this.maliciousEventDao.countTotalMaliciousEventGivenActorIdAndFilterId( + evt.getActorId(), evt.getFilterId()); - try { - this.maliciousEventDao.batchInsert(events); - } catch (SQLException e) { - e.printStackTrace(); - } + this.consumerServiceStub.recordAlert( + RecordAlertRequest.newBuilder() + .setEvent(evt) + .setTotalEvents(totalEvents) + .addAllSampleMaliciousEvents( + sampleData.stream() + .map( + d -> + MaliciousEvent.newBuilder() + .setUrl(d.getUrl()) + .setMethod(d.getMethod().name()) + .setTimestamp(d.getTimestamp()) + .setPayload(d.getOrig()) + .setIp(d.getIp()) + .build()) + .collect(Collectors.toList())) + .build(), + new StreamObserver() { + @Override + public void onNext(RecordAlertResponse value) { + // Do nothing + } + + @Override + public void onError(Throwable t) { + t.printStackTrace(); + } + + @Override + public void onCompleted() { + // Do nothing + } + }); + } catch (Exception e) { + e.printStackTrace(); + } + }); } } diff --git a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto index fffa2d49ef..4d44e2c040 100644 --- a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto +++ b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto @@ -7,10 +7,7 @@ package threat_protection.consumer_service.v1; option java_outer_classname = "ConsumerServiceProto"; option java_package = "threat_protection.v1.consumer_service"; -message SaveMaliciousEventResponse { -} - -message SaveSmartEventResponse { +message RecordAlertResponse { } message SmartEvent { @@ -31,16 +28,12 @@ message MaliciousEvent { string payload = 8; } -message SaveMaliciousEventRequest { - repeated MaliciousEvent events = 2; -} - -message SaveSmartEventRequest { - SmartEvent event = 2; +message RecordAlertRequest { + SmartEvent event = 1; + repeated MaliciousEvent sample_malicious_events = 2; + int32 total_events = 3; } service ConsumerService { - rpc SaveMaliciousEvent(SaveMaliciousEventRequest) returns (SaveMaliciousEventResponse) {} - - rpc SaveSmartEvent(SaveSmartEventRequest) returns (SaveSmartEventResponse) {} + rpc RecordAlert(RecordAlertRequest) returns (RecordAlertResponse) {} } \ No newline at end of file From bc83b53ab32cccba3a807fe61e4db5a7090a377f Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Thu, 28 Nov 2024 23:37:15 +0530 Subject: [PATCH 30/73] exposing single rpc method for recording alerts --- .../ConsumerMaliciousEventService.java | 29 +++++++------------ .../v1/consumer_service.proto | 4 +-- 2 files changed, 12 insertions(+), 21 deletions(-) diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java index 781dd3557e..b18c438ca7 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java @@ -4,10 +4,8 @@ import java.util.List; import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; -import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventRequest; -import com.akto.proto.threat_protection.consumer_service.v1.SaveMaliciousEventResponse; -import com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventRequest; -import com.akto.proto.threat_protection.consumer_service.v1.SaveSmartEventResponse; +import com.akto.proto.threat_protection.consumer_service.v1.RecordAlertRequest; +import com.akto.proto.threat_protection.consumer_service.v1.RecordAlertResponse; import com.akto.proto.threat_protection.consumer_service.v1.SmartEvent; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.db.SmartEventModel; @@ -28,20 +26,19 @@ public ConsumerMaliciousEventService(MongoClient mongoClient) { } @Override - public void saveMaliciousEvent( - SaveMaliciousEventRequest request, - StreamObserver responseObserver) { + public void recordAlert( + RecordAlertRequest request, StreamObserver responseObserver) { List> bulkUpdates = new ArrayList<>(); request - .getEventsList() + .getSampleMaliciousEventsList() .forEach( event -> { bulkUpdates.add( new InsertOneModel<>( new MaliciousEventModel( event.getFilterId(), - event.getActorId(), + event.getActor(), event.getIp(), event.getUrl(), event.getMethod(), @@ -53,21 +50,15 @@ public void saveMaliciousEvent( .getDatabase(accountId + "") .getCollection("malicious_events", MaliciousEventModel.class) .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); - responseObserver.onNext(SaveMaliciousEventResponse.newBuilder().build()); - responseObserver.onCompleted(); - } - @Override - public void saveSmartEvent( - SaveSmartEventRequest request, StreamObserver responseObserver) { - SmartEvent event = request.getEvent(); - int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); + SmartEvent smartEvt = request.getEvent(); this.mongoClient .getDatabase(accountId + "") .getCollection("smart_events", SmartEventModel.class) .insertOne( - new SmartEventModel(event.getFilterId(), event.getActorId(), event.getDetectedAt())); - responseObserver.onNext(SaveSmartEventResponse.newBuilder().build()); + new SmartEventModel( + smartEvt.getFilterId(), smartEvt.getActor(), smartEvt.getDetectedAt())); + responseObserver.onNext(RecordAlertResponse.newBuilder().build()); responseObserver.onCompleted(); } } diff --git a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto index 4d44e2c040..6efdc1774b 100644 --- a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto +++ b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto @@ -11,14 +11,14 @@ message RecordAlertResponse { } message SmartEvent { - string actor_id= 1; + string actor= 1; string filter_id = 2; int64 detected_at = 3; string rule_id = 4; } message MaliciousEvent { - string actor_id= 1; + string actor= 1; string filter_id = 2; string ip = 3; int64 timestamp = 4; From e265de2d794855ad22a8f6cc4f973fa5c1c7fdff Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 29 Nov 2024 09:34:01 +0530 Subject: [PATCH 31/73] updated proto definition --- .../WindowBasedThresholdNotifier.java | 2 +- .../tasks/AbstractKafkaConsumerTask.java | 1 + .../detection/tasks/FlushSampleDataTask.java | 2 +- .../tasks/MaliciousTrafficDetectorTask.java | 7 +- .../detection/tasks/SendAlertsToBackend.java | 21 ++--- .../ConsumerMaliciousEventService.java | 18 ++--- .../threat/protection/db/SmartEventModel.java | 81 ++++++++++--------- .../v1/consumer_service.proto | 39 --------- .../message/malicious_event/v1/message.proto | 17 ++++ .../message/smart_event/v1/message.proto | 13 +++ .../v1/consumer_service.proto | 32 ++++++++ 11 files changed, 135 insertions(+), 98 deletions(-) delete mode 100644 protobuf/threat_protection/consumer_service/v1/consumer_service.proto create mode 100644 protobuf/threat_protection/message/malicious_event/v1/message.proto create mode 100644 protobuf/threat_protection/message/smart_event/v1/message.proto create mode 100644 protobuf/threat_protection/service/consumer_service/v1/consumer_service.proto diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java index 2d4bf3a076..218e5ac165 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java @@ -1,8 +1,8 @@ package com.akto.threat.detection.smart_event_detector.window_based; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; import com.akto.threat.detection.cache.CounterCache; import com.akto.dto.api_protection_parse_layer.Rule; -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ConcurrentHashMap; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java index 9054786ba8..047e93e3ef 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java @@ -19,6 +19,7 @@ public abstract class AbstractKafkaConsumerTask implements Task { public AbstractKafkaConsumerTask(KafkaConfig kafkaConfig, String kafkaTopic) { this.kafkaTopic = kafkaTopic; + this.kafkaConfig = kafkaConfig; String kafkaBrokerUrl = kafkaConfig.getBootstrapServers(); String groupId = kafkaConfig.getGroupId(); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 03f619a1d0..65e31aee30 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -1,7 +1,7 @@ package com.akto.threat.detection.tasks; import com.akto.dto.type.URLMethods; -import com.akto.proto.threat_protection.consumer_service.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 4b4e8d597a..641df40514 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -1,6 +1,8 @@ package com.akto.threat.detection.tasks; import com.akto.dao.context.Context; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent; import com.akto.threat.detection.actor.SourceIPActorGenerator; import com.akto.threat.detection.cache.RedisBackedCounterCache; import com.akto.threat.detection.config.kafka.KafkaConfig; @@ -19,7 +21,6 @@ import com.akto.dto.type.URLMethods; import com.akto.hybrid_parsers.HttpCallParser; import com.akto.kafka.Kafka; -import com.akto.proto.threat_protection.consumer_service.v1.*; import com.akto.rules.TestPlugin; import com.akto.runtime.utils.Utils; import com.akto.threat.detection.smart_event_detector.window_based.WindowBasedThresholdNotifier; @@ -182,7 +183,7 @@ private void processRecord(ConsumerRecord record) { MaliciousEvent maliciousEvent = MaliciousEvent.newBuilder() - .setActorId(actor) + .setActor(actor) .setFilterId(apiFilter.getId()) .setUrl(responseParam.getRequestParams().getURL()) .setMethod(responseParam.getRequestParams().getMethod()) @@ -209,7 +210,7 @@ private void processRecord(ConsumerRecord record) { SmartEvent smartEvent = SmartEvent.newBuilder() .setFilterId(apiFilter.getId()) - .setActorId(actor) + .setActor(actor) .setDetectedAt(responseParam.getTime()) .setRuleId(rule.getName()) .build(); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java index 1357b9adc0..2b1bd9ea86 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java @@ -1,6 +1,10 @@ package com.akto.threat.detection.tasks; -import com.akto.proto.threat_protection.consumer_service.v1.*; +import com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent; +import com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceGrpc; +import com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest; +import com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse; +import com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent; import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; @@ -15,14 +19,13 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import java.sql.Connection; -import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.stream.Collectors; /* -This will read sample malicious data from kafka topic and save it to DB. +This will send alerts to threat detection backend */ public class SendAlertsToBackend extends AbstractKafkaConsumerTask { @@ -48,7 +51,6 @@ ExecutorService getPollingExecutor() { } protected void processRecords(ConsumerRecords records) { - List events = new ArrayList<>(); records.forEach( r -> { String message = r.value(); @@ -71,21 +73,22 @@ protected void processRecords(ConsumerRecords records) { try { List sampleData = this.maliciousEventDao.findGivenActorIdAndFilterId( - evt.getActorId(), evt.getFilterId(), 50); + evt.getActor(), evt.getFilterId(), 50); int totalEvents = this.maliciousEventDao.countTotalMaliciousEventGivenActorIdAndFilterId( - evt.getActorId(), evt.getFilterId()); + evt.getActor(), evt.getFilterId()); this.consumerServiceStub.recordAlert( RecordAlertRequest.newBuilder() - .setEvent(evt) + .setActor(evt.getActor()) + .setFilterId(evt.getFilterId()) .setTotalEvents(totalEvents) - .addAllSampleMaliciousEvents( + .addAllSampleData( sampleData.stream() .map( d -> - MaliciousEvent.newBuilder() + SampleMaliciousEvent.newBuilder() .setUrl(d.getUrl()) .setMethod(d.getMethod().name()) .setTimestamp(d.getTimestamp()) diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java index b18c438ca7..b6fa9cff62 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java @@ -3,10 +3,9 @@ import java.util.ArrayList; import java.util.List; -import com.akto.proto.threat_protection.consumer_service.v1.ConsumerServiceGrpc; -import com.akto.proto.threat_protection.consumer_service.v1.RecordAlertRequest; -import com.akto.proto.threat_protection.consumer_service.v1.RecordAlertResponse; -import com.akto.proto.threat_protection.consumer_service.v1.SmartEvent; +import com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceGrpc; +import com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest; +import com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.db.SmartEventModel; import com.akto.threat.protection.interceptors.Constants; @@ -29,16 +28,18 @@ public ConsumerMaliciousEventService(MongoClient mongoClient) { public void recordAlert( RecordAlertRequest request, StreamObserver responseObserver) { + String actor = request.getActor(); + String filterId = request.getFilterId(); List> bulkUpdates = new ArrayList<>(); request - .getSampleMaliciousEventsList() + .getSampleDataList() .forEach( event -> { bulkUpdates.add( new InsertOneModel<>( new MaliciousEventModel( - event.getFilterId(), - event.getActor(), + filterId, + actor, event.getIp(), event.getUrl(), event.getMethod(), @@ -51,13 +52,12 @@ public void recordAlert( .getCollection("malicious_events", MaliciousEventModel.class) .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); - SmartEvent smartEvt = request.getEvent(); this.mongoClient .getDatabase(accountId + "") .getCollection("smart_events", SmartEventModel.class) .insertOne( new SmartEventModel( - smartEvt.getFilterId(), smartEvt.getActor(), smartEvt.getDetectedAt())); + filterId, actor, request.getTotalEvents(), request.getDetectedAt())); responseObserver.onNext(RecordAlertResponse.newBuilder().build()); responseObserver.onCompleted(); } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java index d9748d5092..e28aeb9f30 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java @@ -4,54 +4,63 @@ public class SmartEventModel { - private String id; + private String id; - private String filterId; + private String filterId; - private long detectedAt; + private String actor; - private String actor; + private long totalHits; - public SmartEventModel() { - } + private long detectedAt; - public SmartEventModel(String filterId, String actor, long detectedAt) { - this.id = UUID.randomUUID().toString(); - this.filterId = filterId; - this.detectedAt = detectedAt; - this.actor = actor; - } + public SmartEventModel() {} - public String getId() { - return id; - } + public SmartEventModel(String filterId, String actor, long totalHits, long detectedAt) { + this.id = UUID.randomUUID().toString(); + this.filterId = filterId; + this.detectedAt = detectedAt; + this.totalHits = totalHits; + this.actor = actor; + } - public void setId(String id) { - this.id = id; - } + public String getId() { + return id; + } - public String getFilterId() { - return filterId; - } + public void setId(String id) { + this.id = id; + } - public void setFilterId(String filterId) { - this.filterId = filterId; - } + public String getFilterId() { + return filterId; + } - public long getDetectedAt() { - return detectedAt; - } + public void setFilterId(String filterId) { + this.filterId = filterId; + } - public void setDetectedAt(long detectedAt) { - this.detectedAt = detectedAt; - } + public long getDetectedAt() { + return detectedAt; + } - public String getActor() { - return actor; - } + public void setDetectedAt(long detectedAt) { + this.detectedAt = detectedAt; + } - public void setActor(String actor) { - this.actor = actor; - } + public String getActor() { + return actor; + } + public void setActor(String actor) { + this.actor = actor; + } + + public long getTotalHits() { + return totalHits; + } + + public void setTotalHits(long totalHits) { + this.totalHits = totalHits; + } } diff --git a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto b/protobuf/threat_protection/consumer_service/v1/consumer_service.proto deleted file mode 100644 index 6efdc1774b..0000000000 --- a/protobuf/threat_protection/consumer_service/v1/consumer_service.proto +++ /dev/null @@ -1,39 +0,0 @@ -syntax = "proto3"; - -package threat_protection.consumer_service.v1; - -// This is a consumer service for saving malicious and smart events. -// For dashboard purposes we will have a separate service to retrieve these events. -option java_outer_classname = "ConsumerServiceProto"; -option java_package = "threat_protection.v1.consumer_service"; - -message RecordAlertResponse { -} - -message SmartEvent { - string actor= 1; - string filter_id = 2; - int64 detected_at = 3; - string rule_id = 4; -} - -message MaliciousEvent { - string actor= 1; - string filter_id = 2; - string ip = 3; - int64 timestamp = 4; - string url = 5; - string method = 6; - int32 api_collection_id = 7; - string payload = 8; -} - -message RecordAlertRequest { - SmartEvent event = 1; - repeated MaliciousEvent sample_malicious_events = 2; - int32 total_events = 3; -} - -service ConsumerService { - rpc RecordAlert(RecordAlertRequest) returns (RecordAlertResponse) {} -} \ No newline at end of file diff --git a/protobuf/threat_protection/message/malicious_event/v1/message.proto b/protobuf/threat_protection/message/malicious_event/v1/message.proto new file mode 100644 index 0000000000..837d22b661 --- /dev/null +++ b/protobuf/threat_protection/message/malicious_event/v1/message.proto @@ -0,0 +1,17 @@ +syntax = "proto3"; + +package threat_protection.message.malicious_event.v1; + +option java_outer_classname = "MaliciousEventProto"; +option java_package = "threat_protection.v1.message.malicious_event"; + +message MaliciousEvent { + string actor= 1; + string filter_id = 2; + string ip = 3; + int64 timestamp = 4; + string url = 5; + string method = 6; + int32 api_collection_id = 7; + string payload = 8; +} diff --git a/protobuf/threat_protection/message/smart_event/v1/message.proto b/protobuf/threat_protection/message/smart_event/v1/message.proto new file mode 100644 index 0000000000..b7dacf6e53 --- /dev/null +++ b/protobuf/threat_protection/message/smart_event/v1/message.proto @@ -0,0 +1,13 @@ +syntax = "proto3"; + +package threat_protection.message.smart_event.v1; + +option java_outer_classname = "SmartEventProto"; +option java_package = "threat_protection.v1.message.smart_event"; + +message SmartEvent { + string actor= 1; + string filter_id = 2; + int64 detected_at = 3; + string rule_id = 4; +} diff --git a/protobuf/threat_protection/service/consumer_service/v1/consumer_service.proto b/protobuf/threat_protection/service/consumer_service/v1/consumer_service.proto new file mode 100644 index 0000000000..f7648d1dee --- /dev/null +++ b/protobuf/threat_protection/service/consumer_service/v1/consumer_service.proto @@ -0,0 +1,32 @@ +syntax = "proto3"; + +package threat_protection.service.consumer_service.v1; + +// This is a consumer service for saving malicious and smart events. +// For dashboard purposes we will have a separate service to retrieve these events. +option java_outer_classname = "ConsumerServiceProto"; +option java_package = "threat_protection.service.consumer_service.v1"; + +message RecordAlertResponse { +} + +message SampleMaliciousEvent { + string ip = 1; + int64 timestamp = 2; + string url = 3; + string method = 4; + int32 api_collection_id = 5; + string payload = 6; +} + +message RecordAlertRequest { + string actor = 1; + string filter_id = 2; + int32 total_events = 3; + int64 detected_at = 4; + repeated SampleMaliciousEvent sample_data = 5; +} + +service ConsumerService { + rpc RecordAlert(RecordAlertRequest) returns (RecordAlertResponse) {} +} \ No newline at end of file From 8ae9eedc07b37bfd8fd096dcc489be74f84ab18b Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 29 Nov 2024 09:44:37 +0530 Subject: [PATCH 32/73] removed cleanup task from threat protection backend module --- .../java/com/akto/threat/protection/Main.java | 2 - .../protection/db/CleanupAuditModel.java | 43 ------ .../threat/protection/tasks/CleanupTask.java | 123 ------------------ 3 files changed, 168 deletions(-) delete mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/CleanupAuditModel.java delete mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index 9f800c4db3..2b1dae1fcd 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -1,7 +1,6 @@ package com.akto.threat.protection; import com.akto.DaoInit; -import com.akto.threat.protection.tasks.CleanupTask; import com.mongodb.ConnectionString; import com.mongodb.ReadPreference; import com.mongodb.client.MongoClient; @@ -17,7 +16,6 @@ public static void main(String[] args) throws Exception { new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")), ReadPreference.secondary()); - new CleanupTask(threatProtectionMongo).init(); int port = Integer.parseInt( System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/CleanupAuditModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/CleanupAuditModel.java deleted file mode 100644 index 2be4e6e7d8..0000000000 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/CleanupAuditModel.java +++ /dev/null @@ -1,43 +0,0 @@ -package com.akto.threat.protection.db; - -import org.bson.types.ObjectId; - -public class CleanupAuditModel { - - private String id; - private long alertWindowStart; - private long alertWindowEnd; - - public CleanupAuditModel() {} - - public CleanupAuditModel(long alertWindowStart, long alertWindowEnd) { - this.id = new ObjectId().toString(); - this.alertWindowStart = alertWindowStart; - this.alertWindowEnd = alertWindowEnd; - } - - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - - public long getAlertWindowStart() { - return alertWindowStart; - } - - public void setAlertWindowStart(long alertWindowStart) { - this.alertWindowStart = alertWindowStart; - } - - public long getAlertWindowEnd() { - return alertWindowEnd; - } - - public void setAlertWindowEnd(long alertWindowEnd) { - this.alertWindowEnd = alertWindowEnd; - } -} - diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java deleted file mode 100644 index cea1a3b298..0000000000 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/CleanupTask.java +++ /dev/null @@ -1,123 +0,0 @@ -package com.akto.threat.protection.tasks; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; - -import org.bson.conversions.Bson; - -import com.akto.dto.Account; -import com.akto.threat.protection.db.CleanupAuditModel; -import com.akto.threat.protection.db.MaliciousEventModel; -import com.akto.util.AccountTask; -import com.mongodb.BasicDBList; -import com.mongodb.BasicDBObject; -import com.mongodb.client.MongoClient; -import com.mongodb.client.MongoCursor; -import com.mongodb.client.model.Accumulators; -import com.mongodb.client.model.Aggregates; -import com.mongodb.client.model.Filters; -import com.mongodb.client.model.Projections; - -public class CleanupTask { - private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(2); - - private final MongoClient mongoClient; - - public CleanupTask(MongoClient mongoClient) { - this.mongoClient = mongoClient; - } - - public void init() { - this.executor.scheduleAtFixedRate(this::cleanUp, 0, 5, TimeUnit.HOURS); - } - - public void cleanUp() { - AccountTask.instance.executeTask( - new Consumer() { - @Override - public void accept(Account account) { - cleanUpForAccount(account.getId()); - } - }, - "cleanup-malicious-requests"); - } - - public Optional getLatestEntry(String accountId) { - return Optional.ofNullable(this.mongoClient.getDatabase(accountId + "") - .getCollection("cleanup_malicious_requests_audit", CleanupAuditModel.class) - .find( - new BasicDBObject("sort", new BasicDBObject("alertWindowEnd", -1))) - .first()); - } - - public void cleanUpForAccount(int accountId) { - // Remove all the requests that have passed their expiry. - // AND those requests whose actor and filter that don't have any alerts - // associated with - // them. - long now = System.currentTimeMillis() / 1000L; - - // Get the latest cleanup audit if exists - Optional audit = this.getLatestEntry(accountId + ""); - long start = audit.map(CleanupAuditModel::getAlertWindowEnd).orElse(0L); - - List pipeline = Arrays.asList( - Aggregates.match( - Filters.and( - Filters.gte("detectedAt", start), - Filters.lt("detectedAt", now))), - Aggregates.group( - 0, - Accumulators.addToSet( - "validFilters", - new BasicDBObject("filterId", "$filterId") - .append("actor", "$actor"))), - Aggregates.project( - Projections.fields( - Projections.include("validFilters"), - Projections.excludeId()))); - - try (MongoCursor result = this.mongoClient.getDatabase(accountId + "") - .getCollection("smart_events") - .aggregate(pipeline, BasicDBObject.class) - .cursor()) { - - BasicDBObject validFilters = result.tryNext(); - if (validFilters == null) { - return; - } - - BasicDBList filters = (BasicDBList) validFilters.get("validFilters"); - List filterList = new ArrayList<>(); - for (Object filter : filters) { - BasicDBObject filterObj = (BasicDBObject) filter; - filterList.add( - new BasicDBObject("filterId", filterObj.getString("filterId")) - .append("actor", filterObj.getString("actor"))); - } - - // Remove all the requests that have passed their expiry. - this.mongoClient.getDatabase(accountId + "") - .getCollection("malicious_events", MaliciousEventModel.class) - .deleteMany( - Filters.and( - Filters.lt("expiry", now), - Filters.nor( - filterList.stream() - .map(Filters::and) - .toArray(Bson[]::new)))); - - // TODO: For any given filter, only keep last 1000 requests - - this.mongoClient.getDatabase(accountId + "") - .getCollection("cleanup_malicious_requests_audit", CleanupAuditModel.class) - .insertOne(new CleanupAuditModel(start, now)); - } - } -} \ No newline at end of file From 15f04fd6a65ae0f6296d3dbb176a45221e64e628 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 29 Nov 2024 10:09:17 +0530 Subject: [PATCH 33/73] updated kafka topic names --- .../src/main/java/com/akto/threat/detection/Main.java | 4 ++-- .../detection/tasks/MaliciousTrafficDetectorTask.java | 7 ++----- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index 729caa1a6c..ea3546ea27 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -47,8 +47,8 @@ public static void main(String[] args) throws Exception { Connection postgres = createPostgresConnection(); new MaliciousTrafficDetectorTask(trafficKafka, internalKafka, createRedisClient()).run(); - new FlushSampleDataTask(postgres, internalKafka, "akto,malicious").run(); - new SendAlertsToBackend(postgres, internalKafka, "akto.smart_event").run(); + new FlushSampleDataTask(postgres, internalKafka, "akto.threat_detection.malicious").run(); + new SendAlertsToBackend(postgres, internalKafka, "akto.threat_detection.alerts").run(); } public static RedisClient createRedisClient() { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 641df40514..5ac8e4046c 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -55,9 +55,6 @@ public class MaliciousTrafficDetectorTask { private final Kafka internalKafka; - private static final String KAFKA_MALICIOUS_TOPIC = "akto.malicious"; - private static final String KAFKA_SMART_EVENT_TOPIC = "akto.smart_event"; - private static final DataActor dataActor = DataActorFactory.fetchInstance(); public MaliciousTrafficDetectorTask( @@ -219,7 +216,7 @@ private void processRecord(ConsumerRecord record) { .marshal() .ifPresent( data -> { - internalKafka.send(data, KAFKA_SMART_EVENT_TOPIC); + internalKafka.send(data, "akto.threat_detection.alerts"); }); } catch (InvalidProtocolBufferException e) { e.printStackTrace(); @@ -239,7 +236,7 @@ private void processRecord(ConsumerRecord record) { .marshal() .ifPresent( data -> { - internalKafka.send(data, KAFKA_MALICIOUS_TOPIC); + internalKafka.send(data, "akto.threat_detection.malicious"); }); }); } catch (Exception e) { From 2382b2d559b3b75accd8e5ddd5779b991e4ef48e Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 29 Nov 2024 12:48:20 +0530 Subject: [PATCH 34/73] running flyway migrations programatically --- apps/threat-detection/pom.xml | 12 +++- .../java/com/akto/threat/detection/Main.java | 70 ++++++++++++------- .../db/malicious_event/MaliciousEventDao.java | 14 ++-- .../malicious_event/MaliciousEventModel.java | 10 +-- .../detection/tasks/SendAlertsToBackend.java | 4 +- .../db/migration/V1__create_base_tables.sql | 14 ++++ 6 files changed, 82 insertions(+), 42 deletions(-) create mode 100644 apps/threat-detection/src/main/resources/db/migration/V1__create_base_tables.sql diff --git a/apps/threat-detection/pom.xml b/apps/threat-detection/pom.xml index c239e1e8e7..847adc1668 100644 --- a/apps/threat-detection/pom.xml +++ b/apps/threat-detection/pom.xml @@ -1,7 +1,7 @@ + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 @@ -101,6 +101,13 @@ 42.7.4 + + org.flywaydb + flyway-core + 9.22.3 + + + @@ -138,6 +145,7 @@ true **/version.txt + **/db/migration/*.sql diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index ea3546ea27..fe9d754390 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -14,35 +14,37 @@ import java.sql.DriverManager; import java.sql.SQLException; +import org.flywaydb.core.Flyway; + public class Main { public static void main(String[] args) throws Exception { + runMigrations(); + DaoInit.init(new ConnectionString(System.getenv("AKTO_MONGO_CONN"))); - KafkaConfig trafficKafka = - KafkaConfig.newBuilder() - .setGroupId("akto.threat.detection") - .setBootstrapServers("localhost:29092") - .setConsumerConfig( - KafkaConsumerConfig.newBuilder() - .setMaxPollRecords(100) - .setPollDurationMilli(100) - .build()) - .setProducerConfig( - KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) - .build(); - - KafkaConfig internalKafka = - KafkaConfig.newBuilder() - .setGroupId("akto.threat.detection") - .setBootstrapServers("localhost:29092") - .setConsumerConfig( - KafkaConsumerConfig.newBuilder() - .setMaxPollRecords(100) - .setPollDurationMilli(100) - .build()) - .setProducerConfig( - KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) - .build(); + KafkaConfig trafficKafka = KafkaConfig.newBuilder() + .setGroupId("akto.threat.detection") + .setBootstrapServers("localhost:29092") + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) + .build(); + + KafkaConfig internalKafka = KafkaConfig.newBuilder() + .setGroupId("akto.threat.detection") + .setBootstrapServers("localhost:29092") + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) + .build(); Connection postgres = createPostgresConnection(); @@ -57,6 +59,22 @@ public static RedisClient createRedisClient() { public static Connection createPostgresConnection() throws SQLException { String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); - return DriverManager.getConnection(url); + String user = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_USER"); + String password = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_PASSWORD"); + return DriverManager.getConnection(url, user, password); + } + + public static void runMigrations() { + String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); + String user = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_USER"); + String password = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_PASSWORD"); + Flyway flyway = Flyway + .configure() + .dataSource(url, user, password) + .locations("classpath:db/migration") + .schemas("flyway") + .load(); + + flyway.migrate(); } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java index 9b3e1736cc..4cf9b9e0a5 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java @@ -20,13 +20,13 @@ public MaliciousEventDao(Connection conn) { public void batchInsert(List events) throws SQLException { String sql = - "INSERT INTO threat_detection.malicious_event (id, actor_id, filter_id, url, method, timestamp, data, ip, country) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)"; + "INSERT INTO threat_detection.malicious_event (id, actor, filter_id, url, method, timestamp, data, ip, country) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)"; conn.setAutoCommit(false); for (int i = 0; i < events.size(); i++) { MaliciousEventModel event = events.get(i); PreparedStatement stmt = this.conn.prepareStatement(sql); stmt.setString(1, event.getId()); - stmt.setString(2, event.getActorId()); + stmt.setString(2, event.getActor()); stmt.setString(3, event.getFilterId()); stmt.setString(4, event.getUrl()); stmt.setString(5, event.getMethod().name()); @@ -45,10 +45,10 @@ public void batchInsert(List events) throws SQLException { conn.commit(); } - public List findGivenActorIdAndFilterId( + public List findGivenActorAndFilterId( String actor, String filterId, int limit) throws SQLException { String sql = - "SELECT * FROM threat_detection.malicious_event WHERE actor_id = ? AND filter_id = ? LIMIT ?"; + "SELECT * FROM threat_detection.malicious_event WHERE actor = ? AND filter_id = ? LIMIT ?"; PreparedStatement stmt = this.conn.prepareStatement(sql); stmt.setString(1, actor); stmt.setString(2, filterId); @@ -59,7 +59,7 @@ public List findGivenActorIdAndFilterId( MaliciousEventModel model = MaliciousEventModel.newBuilder() .setId(rs.getString("id")) - .setActorId(rs.getString("actor_id")) + .setActorId(rs.getString("actor")) .setFilterId(rs.getString("filter_id")) .setUrl(rs.getString("url")) .setMethod(URLMethods.Method.fromString(rs.getString("method"))) @@ -73,10 +73,10 @@ public List findGivenActorIdAndFilterId( } } - public int countTotalMaliciousEventGivenActorIdAndFilterId(String actor, String filterId) + public int countTotalMaliciousEventGivenActorAndFilterId(String actor, String filterId) throws SQLException { String sql = - "SELECT COUNT(*) FROM threat_detection.malicious_event WHERE actor_id = ? AND filter_id = ?"; + "SELECT COUNT(*) FROM threat_detection.malicious_event WHERE actor = ? AND filter_id = ?"; PreparedStatement stmt = this.conn.prepareStatement(sql); stmt.setString(1, actor); stmt.setString(2, filterId); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java index aa15837261..49142070f7 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java @@ -7,7 +7,7 @@ public class MaliciousEventModel { private String id; - private String actorId; + private String actor; private String filterId; private String url; private URLMethods.Method method; @@ -21,7 +21,7 @@ public MaliciousEventModel() {} public MaliciousEventModel(Builder builder) { this.id = builder.id == null ? UUID.randomUUID().toString() : builder.id; - this.actorId = builder.actorId; + this.actor = builder.actorId; this.filterId = builder.filterId; this.url = builder.url; this.method = builder.method; @@ -93,8 +93,8 @@ public String getId() { return id; } - public String getActorId() { - return actorId; + public String getActor() { + return actor; } public String getFilterId() { @@ -128,7 +128,7 @@ public String toString() { + id + '\'' + ", actorId='" - + actorId + + actor + '\'' + ", filterId='" + filterId diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java index 2b1bd9ea86..39f0101cf3 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java @@ -72,11 +72,11 @@ protected void processRecords(ConsumerRecords records) { // Get sample data from postgres for this alert try { List sampleData = - this.maliciousEventDao.findGivenActorIdAndFilterId( + this.maliciousEventDao.findGivenActorAndFilterId( evt.getActor(), evt.getFilterId(), 50); int totalEvents = - this.maliciousEventDao.countTotalMaliciousEventGivenActorIdAndFilterId( + this.maliciousEventDao.countTotalMaliciousEventGivenActorAndFilterId( evt.getActor(), evt.getFilterId()); this.consumerServiceStub.recordAlert( diff --git a/apps/threat-detection/src/main/resources/db/migration/V1__create_base_tables.sql b/apps/threat-detection/src/main/resources/db/migration/V1__create_base_tables.sql new file mode 100644 index 0000000000..c084f6f3bf --- /dev/null +++ b/apps/threat-detection/src/main/resources/db/migration/V1__create_base_tables.sql @@ -0,0 +1,14 @@ +CREATE SCHEMA IF NOT EXISTS THREAT_DETECTION; +CREATE TABLE IF NOT EXISTS THREAT_DETECTION.MALICIOUS_EVENTS ( + ID VARCHAR(36) PRIMARY KEY, + ACTOR VARCHAR(255), + FILTER_ID VARCHAR(255), + URL VARCHAR(1024), + IP VARCHAR(255), + METHOD VARCHAR(255), + TIMESTAMP BIGINT, + ORIG TEXT +); + +-- add index on actor and filter_id and sort data by timestamp +CREATE INDEX MALICIOUS_EVENTS_ACTOR_FILTER_ID_TIMESTAMP_IDX ON THREAT_DETECTION.MALICIOUS_EVENTS(ACTOR, FILTER_ID, TIMESTAMP DESC); From 2438ea6aa60a469d9bbd0bc622d3f763ff6753c2 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 29 Nov 2024 15:54:29 +0530 Subject: [PATCH 35/73] refactored code --- .gitignore | 2 +- Makefile | 10 +- .../com/akto/utils/jobs/CleanInventory.java | 266 ++-- .../akto/hybrid_parsers/HttpCallParser.java | 3 - .../java/com/akto/threat/detection/Main.java | 15 +- .../detection/constants/KafkaTopic.java | 8 + .../db/malicious_event/MaliciousEventDao.java | 5 +- .../malicious_event/MaliciousEventModel.java | 17 +- .../tasks/MaliciousTrafficDetectorTask.java | 5 +- .../detection/tasks/SendAlertsToBackend.java | 13 +- .../db/migration/V1__create_base_tables.sql | 14 - .../db/migration/V1__enable_uuid.sql | 2 + .../V2__create_malicious_event_table.sql | 16 + .../akto/threat/protection/BackendServer.java | 2 +- ...ervice.java => MaliciousAlertService.java} | 10 +- .../message/smart_event/v1/MessageProto.java | 70 - .../message/smart_event/v1/SmartEvent.java | 840 ------------ .../smart_event/v1/SmartEventOrBuilder.java | 53 - .../v1/ConsumerServiceGrpc.java | 293 ----- .../v1/ConsumerServiceProto.java | 106 -- .../v1/RecordAlertRequest.java | 1124 ----------------- .../v1/RecordAlertRequestOrBuilder.java | 71 -- .../v1/RecordAlertResponse.java | 358 ------ .../v1/RecordAlertResponseOrBuilder.java | 11 - .../v1/SampleMaliciousEvent.java | 1042 --------------- .../v1/SampleMaliciousEventOrBuilder.java | 71 -- .../v1/consumer_service.proto | 10 +- scripts/proto-gen.sh | 12 + 28 files changed, 189 insertions(+), 4260 deletions(-) create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/constants/KafkaTopic.java delete mode 100644 apps/threat-detection/src/main/resources/db/migration/V1__create_base_tables.sql create mode 100644 apps/threat-detection/src/main/resources/db/migration/V1__enable_uuid.sql create mode 100644 apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql rename apps/threat-protection-backend/src/main/java/com/akto/threat/protection/{ConsumerMaliciousEventService.java => MaliciousAlertService.java} (81%) delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/MessageProto.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEvent.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEventOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceGrpc.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceProto.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponse.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponseOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEvent.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEventOrBuilder.java rename protobuf/threat_protection/service/{consumer_service => malicious_alert_service}/v1/consumer_service.proto (65%) create mode 100644 scripts/proto-gen.sh diff --git a/.gitignore b/.gitignore index b6e575853c..b02305d731 100644 --- a/.gitignore +++ b/.gitignore @@ -13,7 +13,7 @@ **/.settings **/dist **/gen -libs/protobuf/src/main/java/com/akto/proto +libs/protobuf/src libawesome.dylib temp_* *.templates-config.json diff --git a/Makefile b/Makefile index c5d4de70d7..b79b0909a1 100644 --- a/Makefile +++ b/Makefile @@ -1,4 +1,8 @@ proto-gen: - buf lint protobuf && \ - rm -rf ./libs/protobuf/src/main/java/com/akto && \ - buf generate protobuf \ No newline at end of file + sh ./scripts/proto-gen.sh + +build: proto-gen + mvn install -DskipTests + +build-clean: proto-gen + mvn clean install -DskipTests \ No newline at end of file diff --git a/apps/dashboard/src/main/java/com/akto/utils/jobs/CleanInventory.java b/apps/dashboard/src/main/java/com/akto/utils/jobs/CleanInventory.java index 18ec2476e0..231939d22c 100644 --- a/apps/dashboard/src/main/java/com/akto/utils/jobs/CleanInventory.java +++ b/apps/dashboard/src/main/java/com/akto/utils/jobs/CleanInventory.java @@ -59,46 +59,36 @@ public class CleanInventory { - private static final LoggerMaker loggerMaker = - new LoggerMaker(CleanInventory.class, LogDb.DASHBOARD); + private static final LoggerMaker loggerMaker = new LoggerMaker(CleanInventory.class, LogDb.DASHBOARD); private static final Logger logger = LoggerFactory.getLogger(CleanInventory.class); - static final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + final static ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); public static void cleanInventoryJobRunner() { - scheduler.scheduleAtFixedRate( - new Runnable() { - public void run() { - - int now = Context.now(); - logger.info("Starting cleanInventoryJob for all accounts at " + now); - - AccountTask.instance.executeTask( - new Consumer() { - @Override - public void accept(Account t) { - try { - cleanInventoryJob(); - } catch (Exception e) { - loggerMaker.errorAndAddToDb( - e, "Error in cleanInventoryJob"); - } - } - }, - "clean-inventory-job"); - - int now2 = Context.now(); - int diffNow = now2 - now; - logger.info( - String.format( - "Completed cleanInventoryJob for all accounts at %d , time taken : %d", - now2, diffNow)); + scheduler.scheduleAtFixedRate(new Runnable() { + public void run() { + + int now = Context.now(); + logger.info("Starting cleanInventoryJob for all accounts at " + now); + + AccountTask.instance.executeTask(new Consumer() { + @Override + public void accept(Account t) { + try { + cleanInventoryJob(); + } catch (Exception e) { + loggerMaker.errorAndAddToDb(e, "Error in cleanInventoryJob"); + } } - }, - 0, - 5, - TimeUnit.HOURS); + }, "clean-inventory-job"); + + int now2 = Context.now(); + int diffNow = now2-now; + logger.info(String.format("Completed cleanInventoryJob for all accounts at %d , time taken : %d", now2, diffNow)); + } + }, 0, 5, TimeUnit.HOURS); + } private static Set methodSet = new HashSet<>(); @@ -109,14 +99,10 @@ private static Set getMethodSet() { return methodSet; } - List lowerCaseMethods = - Arrays.asList(URLMethods.Method.getValuesArray()).stream() - .map(s -> s.name().toLowerCase()) - .collect(Collectors.toList()); - List upperCaseMethods = - Arrays.asList(URLMethods.Method.getValuesArray()).stream() - .map(s -> s.name().toUpperCase()) - .collect(Collectors.toList()); + List lowerCaseMethods = Arrays.asList(URLMethods.Method.getValuesArray()).stream() + .map(s -> s.name().toLowerCase()).collect(Collectors.toList()); + List upperCaseMethods = Arrays.asList(URLMethods.Method.getValuesArray()).stream() + .map(s -> s.name().toUpperCase()).collect(Collectors.toList()); methodSet.addAll(upperCaseMethods); methodSet.addAll(lowerCaseMethods); return methodSet; @@ -138,33 +124,23 @@ private static void cleanInventoryJob() { int diff = now2 - now; if (diff >= 2) { - loggerMaker.infoAndAddToDb( - String.format("cleanInventoryJob finished, time taken: %d ", diff)); + loggerMaker.infoAndAddToDb(String.format("cleanInventoryJob finished, time taken: %d ", diff)); } + } + + public static void cleanFilteredSampleDataFromAdvancedFilters(List apiCollections, List yamlTemplates, List redundantUrlList, String filePath, boolean shouldDeleteRequest, boolean saveLogsToDB) throws IOException{ - public static void cleanFilteredSampleDataFromAdvancedFilters( - List apiCollections, - List yamlTemplates, - List redundantUrlList, - String filePath, - boolean shouldDeleteRequest, - boolean saveLogsToDB) - throws IOException { - - Map apiCollectionMap = - apiCollections.stream() - .collect(Collectors.toMap(ApiCollection::getId, Function.identity())); + Map apiCollectionMap = apiCollections.stream().collect(Collectors.toMap(ApiCollection::getId, Function.identity())); // BufferedWriter writer = new BufferedWriter(new FileWriter(new File(filePath))); List sampleDataList = new ArrayList<>(); Bson filters = Filters.empty(); int skip = 0; int limit = 100; Bson sort = Sorts.ascending("_id.apiCollectionId", "_id.url", "_id.method"); - Map collectionWiseDeletionCountMap = new HashMap<>(); + Map collectionWiseDeletionCountMap = new HashMap<>(); - Map filterMap = - FilterYamlTemplateDao.fetchFilterConfig(false, yamlTemplates, true); + Map filterMap = FilterYamlTemplateDao.instance.fetchFilterConfig(false, yamlTemplates, true); Pattern pattern = createRegexPatternFromList(redundantUrlList); do { sampleDataList = SampleDataDao.instance.findAll(filters, skip, limit, sort); @@ -177,52 +153,41 @@ public static void cleanFilteredSampleDataFromAdvancedFilters( List samples = sampleData.getSamples(); remainingSamples.clear(); if (samples == null || samples.isEmpty()) { - logger.info( - "[BadApisRemover] No samples found for : " + sampleData.getId()); + logger.info("[BadApisRemover] No samples found for : " + sampleData.getId()); continue; } - ApiCollection apiCollection = - apiCollectionMap.get(sampleData.getId().getApiCollectionId()); + ApiCollection apiCollection = apiCollectionMap.get(sampleData.getId().getApiCollectionId()); if (apiCollection == null) { - if (!DashboardMode.isMetered()) { - logger.info( - "[BadApisRemover] No apiCollection found for : " - + sampleData.getId()); + if(!DashboardMode.isMetered()){ + logger.info("[BadApisRemover] No apiCollection found for : " + sampleData.getId()); } continue; } + boolean isRedundant = false; boolean isNetsparkerPresent = false; boolean movingApi = false; for (String sample : samples) { - HttpResponseParams httpResponseParams = - HttpCallParser.parseKafkaMessage(sample); + HttpResponseParams httpResponseParams = HttpCallParser.parseKafkaMessage(sample); isNetsparkerPresent |= sample.toLowerCase().contains("netsparker"); - if (httpResponseParams != null) { - isRedundant = - HttpCallParser.isRedundantEndpoint( - httpResponseParams.getRequestParams().getURL(), - pattern); - if (!isRedundant) { - Map> executorNodesMap = - ParseAndExecute.createExecutorNodeMap(filterMap); - Pair temp = - HttpCallParser.applyAdvancedFilters( - httpResponseParams, executorNodesMap, filterMap); + if(httpResponseParams != null){ + isRedundant = HttpCallParser.isRedundantEndpoint(httpResponseParams.getRequestParams().getURL(), pattern); + if(!isRedundant){ + Map> executorNodesMap = ParseAndExecute.createExecutorNodeMap(filterMap); + Pair temp = HttpCallParser.applyAdvancedFilters(httpResponseParams, executorNodesMap, filterMap); HttpResponseParams param = temp.getFirst(); FILTER_TYPE filterType = temp.getSecond(); - if (param != null) { - // comes when Filter_Block is not valid {Remaining => Unchanged, - // Modified, Allowed} - if (filterType.equals(FILTER_TYPE.MODIFIED)) { + if(param != null){ + // comes when Filter_Block is not valid {Remaining => Unchanged, Modified, Allowed} + if(filterType.equals(FILTER_TYPE.MODIFIED)){ // filter passed and modified movingApi = true; remainingSamples.add(sample); break; - } else if (filterType.equals(FILTER_TYPE.ALLOWED)) { + }else if(filterType.equals(FILTER_TYPE.ALLOWED)){ // filter passed and not modified remainingSamples.add(sample); }else if(filterMap.size() == 1){ @@ -237,7 +202,7 @@ public static void cleanFilteredSampleDataFromAdvancedFilters( } } - if (movingApi) { + if(movingApi){ // any 1 of the sample is modifiable, we print this block toMove.add(sampleData.getId()); if(saveLogsToDB){ @@ -275,28 +240,18 @@ public static void cleanFilteredSampleDataFromAdvancedFilters( } } } else { - // other cases like: => filter from advanced filter is passed || filter from - // block filter fails - if (saveLogsToDB) { + // other cases like: => filter from advanced filter is passed || filter from block filter fails + if(saveLogsToDB){ loggerMaker.infoAndAddToDb( - "Filter did not pass, keeping api found from filter: " - + sampleData.getId(), - LogDb.DASHBOARD); - } else { - logger.info( - "[BadApisRemover] " - + isNetsparkerPresent - + " Keeping API from template: " - + sampleData.getId(), - LogDb.DASHBOARD); - } + "Filter did not pass, keeping api found from filter: " + sampleData.getId(), LogDb.DASHBOARD + ); + }else{ + logger.info("[BadApisRemover] " + isNetsparkerPresent + " Keeping API from template: " + sampleData.getId(), LogDb.DASHBOARD); + } + } } catch (Exception e) { - loggerMaker.errorAndAddToDb( - "[BadApisRemover] Couldn't delete an api for default payload: " - + sampleData.getId() - + e.getMessage(), - LogDb.DASHBOARD); + loggerMaker.errorAndAddToDb("[BadApisRemover] Couldn't delete an api for default payload: " + sampleData.getId() + e.getMessage(), LogDb.DASHBOARD); } } if (shouldDeleteRequest) { @@ -308,15 +263,13 @@ public static void cleanFilteredSampleDataFromAdvancedFilters( } while (!sampleDataList.isEmpty()); - for (Map.Entry iterator : collectionWiseDeletionCountMap.entrySet()) { + for(Map.Entry iterator: collectionWiseDeletionCountMap.entrySet()){ int collId = iterator.getKey(); int deletionCount = iterator.getValue(); String name = apiCollectionMap.get(collId).getDisplayName(); - if (saveLogsToDB) { - loggerMaker.infoAndAddToDb( - "Total apis deleted from collection: " + name + " are: " + deletionCount, - LogDb.DASHBOARD); + if(saveLogsToDB){ + loggerMaker.infoAndAddToDb("Total apis deleted from collection: " + name + " are: " + deletionCount, LogDb.DASHBOARD); } } @@ -324,29 +277,22 @@ public static void cleanFilteredSampleDataFromAdvancedFilters( // writer.close(); } - public static void removeUnnecessaryEndpoints( - List apiCollections, boolean shouldDeleteRequest) { + public static void removeUnnecessaryEndpoints(List apiCollections, boolean shouldDeleteRequest){ try { - for (ApiCollection apiCollection : apiCollections) { + for (ApiCollection apiCollection: apiCollections) { List toBeDeleted = new ArrayList<>(); if (apiCollection.getHostName() == null) { continue; } - List endpoints = - com.akto.action.observe.Utils.fetchEndpointsInCollectionUsingHost( - apiCollection.getId(), 0); + List endpoints = com.akto.action.observe.Utils.fetchEndpointsInCollectionUsingHost(apiCollection.getId(), 0); if (endpoints == null || endpoints.isEmpty()) { continue; } - logger.info( - "[BadApisRemover] Starting for APICollection: " + apiCollection.getId(), - LogDb.DASHBOARD); - for (BasicDBObject singleTypeInfo : endpoints) { - singleTypeInfo = - (BasicDBObject) - (singleTypeInfo.getOrDefault("_id", new BasicDBObject())); + logger.info("[BadApisRemover] Starting for APICollection: " + apiCollection.getId(), LogDb.DASHBOARD); + for (BasicDBObject singleTypeInfo: endpoints) { + singleTypeInfo = (BasicDBObject) (singleTypeInfo.getOrDefault("_id", new BasicDBObject())); int apiCollectionId = singleTypeInfo.getInt("apiCollectionId"); String url = singleTypeInfo.getString("url"); String method = singleTypeInfo.getString("method"); @@ -354,60 +300,41 @@ public static void removeUnnecessaryEndpoints( Key key = new Key(apiCollectionId, url, Method.fromString(method), -1, 0, 0); if (method.equalsIgnoreCase("options")) { - logger.info( - "[BadApisRemover] OPTIONS Deleting bad API: " + key, - LogDb.DASHBOARD); + logger.info("[BadApisRemover] OPTIONS Deleting bad API: " + key, LogDb.DASHBOARD); toBeDeleted.add(key); continue; } if (!method.equalsIgnoreCase("get")) { - logger.info( - "[BadApisRemover] Non-get Deleting bad API: " + key, - LogDb.DASHBOARD); + logger.info("[BadApisRemover] Non-get Deleting bad API: " + key, LogDb.DASHBOARD); continue; } Bson filter = ApiInfoDao.getFilter(url, method, apiCollectionId); - + SampleData sampleData = SampleDataDao.instance.findOne(filter); - if (sampleData == null - || sampleData.getSamples() == null - || sampleData.getSamples().isEmpty()) { - Bson stiFilterReq = - Filters.and( - Filters.eq("url", url), - Filters.eq("method", method), - Filters.in( - "responseCode", - new Integer[] {-1, 200, 201, 204, 302}), - Filters.eq("isHeader", false), - Filters.or( - Filters.eq("isUrlParam", false), - Filters.exists("isUrlParam", false)), - Filters.eq("apiCollectionId", apiCollectionId)); - SingleTypeInfo singleTypeInfoForApi = - SingleTypeInfoDao.instance.findOne(stiFilterReq); + if (sampleData == null || sampleData.getSamples() == null || sampleData.getSamples().isEmpty()) { + Bson stiFilterReq = Filters.and( + Filters.eq("url", url), + Filters.eq("method", method), + Filters.in("responseCode", new Integer[]{-1, 200, 201, 204, 302}), + Filters.eq("isHeader", false), + Filters.or(Filters.eq("isUrlParam", false), Filters.exists("isUrlParam", false)), + Filters.eq("apiCollectionId", apiCollectionId) + ); + SingleTypeInfo singleTypeInfoForApi = SingleTypeInfoDao.instance.findOne(stiFilterReq); if (singleTypeInfoForApi == null) { - logger.info( - "[BadApisRemover] no-sample Deleting bad API: " + key, - LogDb.DASHBOARD); - toBeDeleted.add(key); + logger.info("[BadApisRemover] no-sample Deleting bad API: " + key, LogDb.DASHBOARD); + toBeDeleted.add(key); } else { - logger.info( - "[BadApisRemover] yes-sti Deleting bad API: " - + key - + " " - + singleTypeInfoForApi.composeKey(), - LogDb.DASHBOARD); + logger.info("[BadApisRemover] yes-sti Deleting bad API: " + key + " " + singleTypeInfoForApi.composeKey(), LogDb.DASHBOARD); } } else { - logger.info( - "[BadApisRemover] yes-sample Deleting bad API: " + key, - LogDb.DASHBOARD); + logger.info("[BadApisRemover] yes-sample Deleting bad API: " + key, LogDb.DASHBOARD); } } + if (shouldDeleteRequest) { logger.info("starting deletion of apis"); deleteApis(toBeDeleted); @@ -415,28 +342,24 @@ public static void removeUnnecessaryEndpoints( } } catch (Exception e) { - loggerMaker.errorAndAddToDb( - "Couldn't complete scan for APIs remover: " + e.getMessage(), LogDb.DASHBOARD); + loggerMaker.errorAndAddToDb("Couldn't complete scan for APIs remover: " + e.getMessage(), LogDb.DASHBOARD); e.printStackTrace(); } } - public static void deleteOptionsAPIs(List apiCollections) { - for (ApiCollection apiCollection : apiCollections) { + public static void deleteOptionsAPIs(List apiCollections){ + for (ApiCollection apiCollection: apiCollections) { List toBeDeleted = new ArrayList<>(); if (apiCollection.getHostName() == null) { continue; } - List endpoints = - com.akto.action.observe.Utils.fetchEndpointsInCollectionUsingHost( - apiCollection.getId(), 0); + List endpoints = com.akto.action.observe.Utils.fetchEndpointsInCollectionUsingHost(apiCollection.getId(), 0); if (endpoints == null || endpoints.isEmpty()) { continue; } - for (BasicDBObject singleTypeInfo : endpoints) { - singleTypeInfo = - (BasicDBObject) (singleTypeInfo.getOrDefault("_id", new BasicDBObject())); + for (BasicDBObject singleTypeInfo: endpoints) { + singleTypeInfo = (BasicDBObject) (singleTypeInfo.getOrDefault("_id", new BasicDBObject())); int apiCollectionId = singleTypeInfo.getInt("apiCollectionId"); String url = singleTypeInfo.getString("url"); String method = singleTypeInfo.getString("method"); @@ -451,4 +374,5 @@ public static void deleteOptionsAPIs(List apiCollections) { deleteApis(toBeDeleted); } } -} + +} \ No newline at end of file diff --git a/apps/mini-runtime/src/main/java/com/akto/hybrid_parsers/HttpCallParser.java b/apps/mini-runtime/src/main/java/com/akto/hybrid_parsers/HttpCallParser.java index ddf0fce03d..c8f51b93fa 100644 --- a/apps/mini-runtime/src/main/java/com/akto/hybrid_parsers/HttpCallParser.java +++ b/apps/mini-runtime/src/main/java/com/akto/hybrid_parsers/HttpCallParser.java @@ -1,7 +1,5 @@ package com.akto.hybrid_parsers; -import com.akto.dao.ApiCollectionsDao; -import com.akto.dao.billing.OrganizationsDao; import com.akto.dao.context.Context; import com.akto.dao.traffic_metrics.TrafficMetricsDao; import com.akto.hybrid_dependency.DependencyAnalyser; @@ -29,7 +27,6 @@ import com.mongodb.client.model.*; import okhttp3.*; import org.apache.commons.lang3.math.NumberUtils; -import org.bson.conversions.Bson; import com.alibaba.fastjson2.*; import java.io.IOException; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index fe9d754390..b59fd2b897 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -4,6 +4,7 @@ import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.config.kafka.KafkaConsumerConfig; import com.akto.threat.detection.config.kafka.KafkaProducerConfig; +import com.akto.threat.detection.constants.KafkaTopic; import com.akto.threat.detection.tasks.FlushSampleDataTask; import com.akto.threat.detection.tasks.MaliciousTrafficDetectorTask; import com.akto.threat.detection.tasks.SendAlertsToBackend; @@ -18,13 +19,15 @@ public class Main { + private static final String CONSUMER_GROUP_ID = "akto.threat_detection"; + public static void main(String[] args) throws Exception { runMigrations(); DaoInit.init(new ConnectionString(System.getenv("AKTO_MONGO_CONN"))); KafkaConfig trafficKafka = KafkaConfig.newBuilder() - .setGroupId("akto.threat.detection") - .setBootstrapServers("localhost:29092") + .setGroupId(CONSUMER_GROUP_ID) + .setBootstrapServers(System.getenv("AKTO_TRAFFIC_KAFKA_BOOTSTRAP_SERVER")) .setConsumerConfig( KafkaConsumerConfig.newBuilder() .setMaxPollRecords(100) @@ -35,8 +38,8 @@ public static void main(String[] args) throws Exception { .build(); KafkaConfig internalKafka = KafkaConfig.newBuilder() - .setGroupId("akto.threat.detection") - .setBootstrapServers("localhost:29092") + .setGroupId(CONSUMER_GROUP_ID) + .setBootstrapServers(System.getenv("AKTO_INTERNAL_KAFKA_BOOTSTRAP_SERVER")) .setConsumerConfig( KafkaConsumerConfig.newBuilder() .setMaxPollRecords(100) @@ -49,8 +52,8 @@ public static void main(String[] args) throws Exception { Connection postgres = createPostgresConnection(); new MaliciousTrafficDetectorTask(trafficKafka, internalKafka, createRedisClient()).run(); - new FlushSampleDataTask(postgres, internalKafka, "akto.threat_detection.malicious").run(); - new SendAlertsToBackend(postgres, internalKafka, "akto.threat_detection.alerts").run(); + new FlushSampleDataTask(postgres, internalKafka, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS).run(); + new SendAlertsToBackend(postgres, internalKafka, KafkaTopic.ThreatDetection.ALERTS).run(); } public static RedisClient createRedisClient() { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/constants/KafkaTopic.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/constants/KafkaTopic.java new file mode 100644 index 0000000000..7c13df8d87 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/constants/KafkaTopic.java @@ -0,0 +1,8 @@ +package com.akto.threat.detection.constants; + +public class KafkaTopic { + public static class ThreatDetection { + public static final String MALICIOUS_EVENTS = "akto.threat_detection.malicious_events"; + public static final String ALERTS = "akto.threat_detection.alerts"; + } +} \ No newline at end of file diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java index 4cf9b9e0a5..906c0785a4 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java @@ -20,7 +20,7 @@ public MaliciousEventDao(Connection conn) { public void batchInsert(List events) throws SQLException { String sql = - "INSERT INTO threat_detection.malicious_event (id, actor, filter_id, url, method, timestamp, data, ip, country) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)"; + "INSERT INTO threat_detection.malicious_event (id, actor, filter_id, url, method, timestamp, orig, ip, country) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)"; conn.setAutoCommit(false); for (int i = 0; i < events.size(); i++) { MaliciousEventModel event = events.get(i); @@ -64,8 +64,9 @@ public List findGivenActorAndFilterId( .setUrl(rs.getString("url")) .setMethod(URLMethods.Method.fromString(rs.getString("method"))) .setTimestamp(rs.getLong("timestamp")) - .setOrig(rs.getString("data")) + .setOrig(rs.getString("orig")) .setIp(rs.getString("ip")) + .setCreatedAt(rs.getDate("created_at").toLocalDate()) .build(); models.add(model); } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java index 49142070f7..adbf25bc33 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java @@ -2,7 +2,7 @@ import com.akto.dto.type.URLMethods; -import java.util.UUID; +import java.time.LocalDate; public class MaliciousEventModel { @@ -17,10 +17,12 @@ public class MaliciousEventModel { // Geo location data private String ip; + private LocalDate createdAt; + public MaliciousEventModel() {} public MaliciousEventModel(Builder builder) { - this.id = builder.id == null ? UUID.randomUUID().toString() : builder.id; + this.id = builder.id; this.actor = builder.actorId; this.filterId = builder.filterId; this.url = builder.url; @@ -28,6 +30,7 @@ public MaliciousEventModel(Builder builder) { this.timestamp = builder.timestamp; this.orig = builder.orig; this.ip = builder.ip; + this.createdAt = builder.createdAt; } public static class Builder { @@ -39,6 +42,7 @@ public static class Builder { private long timestamp; private String orig; private String ip; + private LocalDate createdAt; public Builder setId(String id) { this.id = id; @@ -83,6 +87,11 @@ public Builder setIp(String ip) { public MaliciousEventModel build() { return new MaliciousEventModel(this); } + + public Builder setCreatedAt(LocalDate createdAt) { + this.createdAt = createdAt; + return this; + } } public static Builder newBuilder() { @@ -121,6 +130,10 @@ public String getIp() { return ip; } + public LocalDate getCreatedAt() { + return createdAt; + } + @Override public String toString() { return "MaliciousEventModel{" diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 5ac8e4046c..a9f1702b02 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -6,6 +6,7 @@ import com.akto.threat.detection.actor.SourceIPActorGenerator; import com.akto.threat.detection.cache.RedisBackedCounterCache; import com.akto.threat.detection.config.kafka.KafkaConfig; +import com.akto.threat.detection.constants.KafkaTopic; import com.akto.dao.monitoring.FilterYamlTemplateDao; import com.akto.data_actor.DataActor; import com.akto.data_actor.DataActorFactory; @@ -216,7 +217,7 @@ private void processRecord(ConsumerRecord record) { .marshal() .ifPresent( data -> { - internalKafka.send(data, "akto.threat_detection.alerts"); + internalKafka.send(data, KafkaTopic.ThreatDetection.ALERTS); }); } catch (InvalidProtocolBufferException e) { e.printStackTrace(); @@ -236,7 +237,7 @@ private void processRecord(ConsumerRecord record) { .marshal() .ifPresent( data -> { - internalKafka.send(data, "akto.threat_detection.malicious"); + internalKafka.send(data, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS); }); }); } catch (Exception e) { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java index 39f0101cf3..ae7a0e3863 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java @@ -1,10 +1,11 @@ package com.akto.threat.detection.tasks; import com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent; -import com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceGrpc; -import com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest; -import com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse; -import com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousAlertServiceGrpc; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousAlertServiceGrpc.MaliciousAlertServiceStub; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent; import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; @@ -31,7 +32,7 @@ public class SendAlertsToBackend extends AbstractKafkaConsumerTask { private final MaliciousEventDao maliciousEventDao; - private final ConsumerServiceGrpc.ConsumerServiceStub consumerServiceStub; + private final MaliciousAlertServiceStub consumerServiceStub; public SendAlertsToBackend(Connection conn, KafkaConfig trafficConfig, String topic) { super(trafficConfig, topic); @@ -41,7 +42,7 @@ public SendAlertsToBackend(Connection conn, KafkaConfig trafficConfig, String to ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); this.consumerServiceStub = - ConsumerServiceGrpc.newStub(channel) + MaliciousAlertServiceGrpc.newStub(channel) .withCallCredentials( new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); } diff --git a/apps/threat-detection/src/main/resources/db/migration/V1__create_base_tables.sql b/apps/threat-detection/src/main/resources/db/migration/V1__create_base_tables.sql deleted file mode 100644 index c084f6f3bf..0000000000 --- a/apps/threat-detection/src/main/resources/db/migration/V1__create_base_tables.sql +++ /dev/null @@ -1,14 +0,0 @@ -CREATE SCHEMA IF NOT EXISTS THREAT_DETECTION; -CREATE TABLE IF NOT EXISTS THREAT_DETECTION.MALICIOUS_EVENTS ( - ID VARCHAR(36) PRIMARY KEY, - ACTOR VARCHAR(255), - FILTER_ID VARCHAR(255), - URL VARCHAR(1024), - IP VARCHAR(255), - METHOD VARCHAR(255), - TIMESTAMP BIGINT, - ORIG TEXT -); - --- add index on actor and filter_id and sort data by timestamp -CREATE INDEX MALICIOUS_EVENTS_ACTOR_FILTER_ID_TIMESTAMP_IDX ON THREAT_DETECTION.MALICIOUS_EVENTS(ACTOR, FILTER_ID, TIMESTAMP DESC); diff --git a/apps/threat-detection/src/main/resources/db/migration/V1__enable_uuid.sql b/apps/threat-detection/src/main/resources/db/migration/V1__enable_uuid.sql new file mode 100644 index 0000000000..e5c340505a --- /dev/null +++ b/apps/threat-detection/src/main/resources/db/migration/V1__enable_uuid.sql @@ -0,0 +1,2 @@ +-- enabling uuid extension +create extension if not exists "uuid-ossp"; \ No newline at end of file diff --git a/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql b/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql new file mode 100644 index 0000000000..a860ee20b1 --- /dev/null +++ b/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql @@ -0,0 +1,16 @@ +-- create schema and table for malicious events +create schema if not exists threat_detection; +create table if not exists threat_detection.malicious_events ( + id uuid primary key default uuid_generate_v4(), + actor varchar(255) not null, + filter_id varchar(255) not null, + url varchar(1024), + ip varchar(255), + method varchar(255), + timestamp bigint not null, + orig text not null, + created_at timestamp default (timezone('utc', now())) +); + +-- add index on actor and filter_id and sort data by timestamp +create index malicious_events_actor_filter_id_timestamp_idx on threat_detection.malicious_events(actor, filter_id, timestamp desc); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java index 2fb64553e7..462cbbfe7f 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java @@ -17,7 +17,7 @@ public BackendServer(int port, MongoClient mongoClient) { this.port = port; this.server = ServerBuilder.forPort(port) - .addService(new ConsumerMaliciousEventService(mongoClient)) + .addService(new MaliciousAlertService(mongoClient)) .intercept(new AuthenticationInterceptor()) .build(); } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java similarity index 81% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java rename to apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java index b6fa9cff62..f9f79bc8b6 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/ConsumerMaliciousEventService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java @@ -3,9 +3,9 @@ import java.util.ArrayList; import java.util.List; -import com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceGrpc; -import com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest; -import com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousAlertServiceGrpc.MaliciousAlertServiceImplBase; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.db.SmartEventModel; import com.akto.threat.protection.interceptors.Constants; @@ -16,11 +16,11 @@ import io.grpc.stub.StreamObserver; -public class ConsumerMaliciousEventService extends ConsumerServiceGrpc.ConsumerServiceImplBase { +public class MaliciousAlertService extends MaliciousAlertServiceImplBase { private final MongoClient mongoClient; - public ConsumerMaliciousEventService(MongoClient mongoClient) { + public MaliciousAlertService(MongoClient mongoClient) { this.mongoClient = mongoClient; } diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/MessageProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/MessageProto.java deleted file mode 100644 index f64eb0cb39..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/MessageProto.java +++ /dev/null @@ -1,70 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/smart_event/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.smart_event.v1; - -public final class MessageProto { - private MessageProto() {} - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - MessageProto.class.getName()); - } - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { - } - - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); - } - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_message_smart_event_v1_SmartEvent_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n6threat_protection/message/smart_event/" + - "v1/message.proto\022(threat_protection.mess" + - "age.smart_event.v1\"y\n\nSmartEvent\022\024\n\005acto" + - "r\030\001 \001(\tR\005actor\022\033\n\tfilter_id\030\002 \001(\tR\010filte" + - "rId\022\037\n\013detected_at\030\003 \001(\003R\ndetectedAt\022\027\n\007" + - "rule_id\030\004 \001(\tR\006ruleIdB\202\002\n7com.akto.proto" + - ".threat_protection.message.smart_event.v" + - "1B\014MessageProtoP\001\242\002\003TMS\252\002&ThreatProtecti" + - "on.Message.SmartEvent.V1\312\002&ThreatProtect" + - "ion\\Message\\SmartEvent\\V1\342\0022ThreatProtec" + - "tion\\Message\\SmartEvent\\V1\\GPBMetadata\352\002" + - ")ThreatProtection::Message::SmartEvent::" + - "V1b\006proto3" - }; - descriptor = com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - }); - internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_threat_protection_message_smart_event_v1_SmartEvent_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor, - new java.lang.String[] { "Actor", "FilterId", "DetectedAt", "RuleId", }); - descriptor.resolveAllFeaturesImmutable(); - } - - // @@protoc_insertion_point(outer_class_scope) -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEvent.java deleted file mode 100644 index 51b3f4684b..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEvent.java +++ /dev/null @@ -1,840 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/smart_event/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.smart_event.v1; - -/** - * Protobuf type {@code threat_protection.message.smart_event.v1.SmartEvent} - */ -public final class SmartEvent extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.message.smart_event.v1.SmartEvent) - SmartEventOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SmartEvent.class.getName()); - } - // Use SmartEvent.newBuilder() to construct. - private SmartEvent(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SmartEvent() { - actor_ = ""; - filterId_ = ""; - ruleId_ = ""; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.class, com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.Builder.class); - } - - public static final int ACTOR_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - @java.lang.Override - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int DETECTED_AT_FIELD_NUMBER = 3; - private long detectedAt_ = 0L; - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - - public static final int RULE_ID_FIELD_NUMBER = 4; - @SuppressWarnings("serial") - private volatile java.lang.Object ruleId_ = ""; - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The ruleId. - */ - @java.lang.Override - public java.lang.String getRuleId() { - java.lang.Object ref = ruleId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ruleId_ = s; - return s; - } - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The bytes for ruleId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getRuleIdBytes() { - java.lang.Object ref = ruleId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ruleId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); - } - if (detectedAt_ != 0L) { - output.writeInt64(3, detectedAt_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ruleId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 4, ruleId_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); - } - if (detectedAt_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(3, detectedAt_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ruleId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(4, ruleId_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent other = (com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent) obj; - - if (!getActor() - .equals(other.getActor())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (getDetectedAt() - != other.getDetectedAt()) return false; - if (!getRuleId() - .equals(other.getRuleId())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACTOR_FIELD_NUMBER; - hash = (53 * hash) + getActor().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getDetectedAt()); - hash = (37 * hash) + RULE_ID_FIELD_NUMBER; - hash = (53 * hash) + getRuleId().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.message.smart_event.v1.SmartEvent} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.message.smart_event.v1.SmartEvent) - com.akto.proto.threat_protection.message.smart_event.v1.SmartEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.class, com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - actor_ = ""; - filterId_ = ""; - detectedAt_ = 0L; - ruleId_ = ""; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.message.smart_event.v1.MessageProto.internal_static_threat_protection_message_smart_event_v1_SmartEvent_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent getDefaultInstanceForType() { - return com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent build() { - com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent buildPartial() { - com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent result = new com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.actor_ = actor_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.filterId_ = filterId_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.detectedAt_ = detectedAt_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.ruleId_ = ruleId_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent) { - return mergeFrom((com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent other) { - if (other == com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent.getDefaultInstance()) return this; - if (!other.getActor().isEmpty()) { - actor_ = other.actor_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000002; - onChanged(); - } - if (other.getDetectedAt() != 0L) { - setDetectedAt(other.getDetectedAt()); - } - if (!other.getRuleId().isEmpty()) { - ruleId_ = other.ruleId_; - bitField0_ |= 0x00000008; - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - actor_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000002; - break; - } // case 18 - case 24: { - detectedAt_ = input.readInt64(); - bitField0_ |= 0x00000004; - break; - } // case 24 - case 34: { - ruleId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000008; - break; - } // case 34 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The actor to set. - * @return This builder for chaining. - */ - public Builder setActor( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @return This builder for chaining. - */ - public Builder clearActor() { - actor_ = getDefaultInstance().getActor(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The bytes for actor to set. - * @return This builder for chaining. - */ - public Builder setActorBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - private long detectedAt_ ; - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @param value The detectedAt to set. - * @return This builder for chaining. - */ - public Builder setDetectedAt(long value) { - - detectedAt_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return This builder for chaining. - */ - public Builder clearDetectedAt() { - bitField0_ = (bitField0_ & ~0x00000004); - detectedAt_ = 0L; - onChanged(); - return this; - } - - private java.lang.Object ruleId_ = ""; - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The ruleId. - */ - public java.lang.String getRuleId() { - java.lang.Object ref = ruleId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ruleId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The bytes for ruleId. - */ - public com.google.protobuf.ByteString - getRuleIdBytes() { - java.lang.Object ref = ruleId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ruleId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @param value The ruleId to set. - * @return This builder for chaining. - */ - public Builder setRuleId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ruleId_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return This builder for chaining. - */ - public Builder clearRuleId() { - ruleId_ = getDefaultInstance().getRuleId(); - bitField0_ = (bitField0_ & ~0x00000008); - onChanged(); - return this; - } - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @param value The bytes for ruleId to set. - * @return This builder for chaining. - */ - public Builder setRuleIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ruleId_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.message.smart_event.v1.SmartEvent) - } - - // @@protoc_insertion_point(class_scope:threat_protection.message.smart_event.v1.SmartEvent) - private static final com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent(); - } - - public static com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SmartEvent parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEventOrBuilder.java deleted file mode 100644 index 7790cfaa43..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/smart_event/v1/SmartEventOrBuilder.java +++ /dev/null @@ -1,53 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/smart_event/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.smart_event.v1; - -public interface SmartEventOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.message.smart_event.v1.SmartEvent) - com.google.protobuf.MessageOrBuilder { - - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - java.lang.String getActor(); - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - com.google.protobuf.ByteString - getActorBytes(); - - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - java.lang.String getFilterId(); - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - com.google.protobuf.ByteString - getFilterIdBytes(); - - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - long getDetectedAt(); - - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The ruleId. - */ - java.lang.String getRuleId(); - /** - * string rule_id = 4 [json_name = "ruleId"]; - * @return The bytes for ruleId. - */ - com.google.protobuf.ByteString - getRuleIdBytes(); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceGrpc.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceGrpc.java deleted file mode 100644 index d9e12f2b9d..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceGrpc.java +++ /dev/null @@ -1,293 +0,0 @@ -package com.akto.proto.threat_protection.service.consumer_service.v1; - -import static io.grpc.MethodDescriptor.generateFullMethodName; - -/** - */ -@javax.annotation.Generated( - value = "by gRPC proto compiler (version 1.68.1)", - comments = "Source: threat_protection/service/consumer_service/v1/consumer_service.proto") -@io.grpc.stub.annotations.GrpcGenerated -public final class ConsumerServiceGrpc { - - private ConsumerServiceGrpc() {} - - public static final java.lang.String SERVICE_NAME = "threat_protection.service.consumer_service.v1.ConsumerService"; - - // Static method descriptors that strictly reflect the proto. - private static volatile io.grpc.MethodDescriptor getRecordAlertMethod; - - @io.grpc.stub.annotations.RpcMethod( - fullMethodName = SERVICE_NAME + '/' + "RecordAlert", - requestType = com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.class, - responseType = com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.class, - methodType = io.grpc.MethodDescriptor.MethodType.UNARY) - public static io.grpc.MethodDescriptor getRecordAlertMethod() { - io.grpc.MethodDescriptor getRecordAlertMethod; - if ((getRecordAlertMethod = ConsumerServiceGrpc.getRecordAlertMethod) == null) { - synchronized (ConsumerServiceGrpc.class) { - if ((getRecordAlertMethod = ConsumerServiceGrpc.getRecordAlertMethod) == null) { - ConsumerServiceGrpc.getRecordAlertMethod = getRecordAlertMethod = - io.grpc.MethodDescriptor.newBuilder() - .setType(io.grpc.MethodDescriptor.MethodType.UNARY) - .setFullMethodName(generateFullMethodName(SERVICE_NAME, "RecordAlert")) - .setSampledToLocalTracing(true) - .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.getDefaultInstance())) - .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.getDefaultInstance())) - .setSchemaDescriptor(new ConsumerServiceMethodDescriptorSupplier("RecordAlert")) - .build(); - } - } - } - return getRecordAlertMethod; - } - - /** - * Creates a new async stub that supports all call types for the service - */ - public static ConsumerServiceStub newStub(io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public ConsumerServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceStub(channel, callOptions); - } - }; - return ConsumerServiceStub.newStub(factory, channel); - } - - /** - * Creates a new blocking-style stub that supports unary and streaming output calls on the service - */ - public static ConsumerServiceBlockingStub newBlockingStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public ConsumerServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceBlockingStub(channel, callOptions); - } - }; - return ConsumerServiceBlockingStub.newStub(factory, channel); - } - - /** - * Creates a new ListenableFuture-style stub that supports unary calls on the service - */ - public static ConsumerServiceFutureStub newFutureStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public ConsumerServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceFutureStub(channel, callOptions); - } - }; - return ConsumerServiceFutureStub.newStub(factory, channel); - } - - /** - */ - public interface AsyncService { - - /** - */ - default void recordAlert(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getRecordAlertMethod(), responseObserver); - } - } - - /** - * Base class for the server implementation of the service ConsumerService. - */ - public static abstract class ConsumerServiceImplBase - implements io.grpc.BindableService, AsyncService { - - @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { - return ConsumerServiceGrpc.bindService(this); - } - } - - /** - * A stub to allow clients to do asynchronous rpc calls to service ConsumerService. - */ - public static final class ConsumerServiceStub - extends io.grpc.stub.AbstractAsyncStub { - private ConsumerServiceStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected ConsumerServiceStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceStub(channel, callOptions); - } - - /** - */ - public void recordAlert(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ClientCalls.asyncUnaryCall( - getChannel().newCall(getRecordAlertMethod(), getCallOptions()), request, responseObserver); - } - } - - /** - * A stub to allow clients to do synchronous rpc calls to service ConsumerService. - */ - public static final class ConsumerServiceBlockingStub - extends io.grpc.stub.AbstractBlockingStub { - private ConsumerServiceBlockingStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected ConsumerServiceBlockingStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceBlockingStub(channel, callOptions); - } - - /** - */ - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse recordAlert(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest request) { - return io.grpc.stub.ClientCalls.blockingUnaryCall( - getChannel(), getRecordAlertMethod(), getCallOptions(), request); - } - } - - /** - * A stub to allow clients to do ListenableFuture-style rpc calls to service ConsumerService. - */ - public static final class ConsumerServiceFutureStub - extends io.grpc.stub.AbstractFutureStub { - private ConsumerServiceFutureStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected ConsumerServiceFutureStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new ConsumerServiceFutureStub(channel, callOptions); - } - - /** - */ - public com.google.common.util.concurrent.ListenableFuture recordAlert( - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest request) { - return io.grpc.stub.ClientCalls.futureUnaryCall( - getChannel().newCall(getRecordAlertMethod(), getCallOptions()), request); - } - } - - private static final int METHODID_RECORD_ALERT = 0; - - private static final class MethodHandlers implements - io.grpc.stub.ServerCalls.UnaryMethod, - io.grpc.stub.ServerCalls.ServerStreamingMethod, - io.grpc.stub.ServerCalls.ClientStreamingMethod, - io.grpc.stub.ServerCalls.BidiStreamingMethod { - private final AsyncService serviceImpl; - private final int methodId; - - MethodHandlers(AsyncService serviceImpl, int methodId) { - this.serviceImpl = serviceImpl; - this.methodId = methodId; - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - case METHODID_RECORD_ALERT: - serviceImpl.recordAlert((com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest) request, - (io.grpc.stub.StreamObserver) responseObserver); - break; - default: - throw new AssertionError(); - } - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public io.grpc.stub.StreamObserver invoke( - io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - default: - throw new AssertionError(); - } - } - } - - public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { - return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) - .addMethod( - getRecordAlertMethod(), - io.grpc.stub.ServerCalls.asyncUnaryCall( - new MethodHandlers< - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest, - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse>( - service, METHODID_RECORD_ALERT))) - .build(); - } - - private static abstract class ConsumerServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { - ConsumerServiceBaseDescriptorSupplier() {} - - @java.lang.Override - public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.getDescriptor(); - } - - @java.lang.Override - public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { - return getFileDescriptor().findServiceByName("ConsumerService"); - } - } - - private static final class ConsumerServiceFileDescriptorSupplier - extends ConsumerServiceBaseDescriptorSupplier { - ConsumerServiceFileDescriptorSupplier() {} - } - - private static final class ConsumerServiceMethodDescriptorSupplier - extends ConsumerServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { - private final java.lang.String methodName; - - ConsumerServiceMethodDescriptorSupplier(java.lang.String methodName) { - this.methodName = methodName; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { - return getServiceDescriptor().findMethodByName(methodName); - } - } - - private static volatile io.grpc.ServiceDescriptor serviceDescriptor; - - public static io.grpc.ServiceDescriptor getServiceDescriptor() { - io.grpc.ServiceDescriptor result = serviceDescriptor; - if (result == null) { - synchronized (ConsumerServiceGrpc.class) { - result = serviceDescriptor; - if (result == null) { - serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) - .setSchemaDescriptor(new ConsumerServiceFileDescriptorSupplier()) - .addMethod(getRecordAlertMethod()) - .build(); - } - } - } - return result; - } -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceProto.java deleted file mode 100644 index f1960e1aab..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/ConsumerServiceProto.java +++ /dev/null @@ -1,106 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.consumer_service.v1; - -public final class ConsumerServiceProto { - private ConsumerServiceProto() {} - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - ConsumerServiceProto.class.getName()); - } - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { - } - - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); - } - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\nDthreat_protection/service/consumer_ser" + - "vice/v1/consumer_service.proto\022-threat_p" + - "rotection.service.consumer_service.v1\"\025\n" + - "\023RecordAlertResponse\"\264\001\n\024SampleMalicious" + - "Event\022\016\n\002ip\030\001 \001(\tR\002ip\022\034\n\ttimestamp\030\002 \001(\003" + - "R\ttimestamp\022\020\n\003url\030\003 \001(\tR\003url\022\026\n\006method\030" + - "\004 \001(\tR\006method\022*\n\021api_collection_id\030\005 \001(\005" + - "R\017apiCollectionId\022\030\n\007payload\030\006 \001(\tR\007payl" + - "oad\"\361\001\n\022RecordAlertRequest\022\024\n\005actor\030\001 \001(" + - "\tR\005actor\022\033\n\tfilter_id\030\002 \001(\tR\010filterId\022!\n" + - "\014total_events\030\003 \001(\005R\013totalEvents\022\037\n\013dete" + - "cted_at\030\004 \001(\003R\ndetectedAt\022d\n\013sample_data" + - "\030\005 \003(\0132C.threat_protection.service.consu" + - "mer_service.v1.SampleMaliciousEventR\nsam" + - "pleData2\252\001\n\017ConsumerService\022\226\001\n\013RecordAl" + - "ert\022A.threat_protection.service.consumer" + - "_service.v1.RecordAlertRequest\032B.threat_" + - "protection.service.consumer_service.v1.R" + - "ecordAlertResponse\"\000B\243\002\n builder) { - super(builder); - } - private RecordAlertRequest() { - actor_ = ""; - filterId_ = ""; - sampleData_ = java.util.Collections.emptyList(); - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.class, com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.Builder.class); - } - - public static final int ACTOR_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - @java.lang.Override - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TOTAL_EVENTS_FIELD_NUMBER = 3; - private int totalEvents_ = 0; - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @return The totalEvents. - */ - @java.lang.Override - public int getTotalEvents() { - return totalEvents_; - } - - public static final int DETECTED_AT_FIELD_NUMBER = 4; - private long detectedAt_ = 0L; - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - - public static final int SAMPLE_DATA_FIELD_NUMBER = 5; - @SuppressWarnings("serial") - private java.util.List sampleData_; - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public java.util.List getSampleDataList() { - return sampleData_; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public java.util.List - getSampleDataOrBuilderList() { - return sampleData_; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public int getSampleDataCount() { - return sampleData_.size(); - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getSampleData(int index) { - return sampleData_.get(index); - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( - int index) { - return sampleData_.get(index); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); - } - if (totalEvents_ != 0) { - output.writeInt32(3, totalEvents_); - } - if (detectedAt_ != 0L) { - output.writeInt64(4, detectedAt_); - } - for (int i = 0; i < sampleData_.size(); i++) { - output.writeMessage(5, sampleData_.get(i)); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); - } - if (totalEvents_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(3, totalEvents_); - } - if (detectedAt_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(4, detectedAt_); - } - for (int i = 0; i < sampleData_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(5, sampleData_.get(i)); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest other = (com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest) obj; - - if (!getActor() - .equals(other.getActor())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (getTotalEvents() - != other.getTotalEvents()) return false; - if (getDetectedAt() - != other.getDetectedAt()) return false; - if (!getSampleDataList() - .equals(other.getSampleDataList())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACTOR_FIELD_NUMBER; - hash = (53 * hash) + getActor().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (37 * hash) + TOTAL_EVENTS_FIELD_NUMBER; - hash = (53 * hash) + getTotalEvents(); - hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getDetectedAt()); - if (getSampleDataCount() > 0) { - hash = (37 * hash) + SAMPLE_DATA_FIELD_NUMBER; - hash = (53 * hash) + getSampleDataList().hashCode(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.consumer_service.v1.RecordAlertRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.consumer_service.v1.RecordAlertRequest) - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.class, com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - actor_ = ""; - filterId_ = ""; - totalEvents_ = 0; - detectedAt_ = 0L; - if (sampleDataBuilder_ == null) { - sampleData_ = java.util.Collections.emptyList(); - } else { - sampleData_ = null; - sampleDataBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest build() { - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest buildPartial() { - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest result = new com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest(this); - buildPartialRepeatedFields(result); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartialRepeatedFields(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest result) { - if (sampleDataBuilder_ == null) { - if (((bitField0_ & 0x00000010) != 0)) { - sampleData_ = java.util.Collections.unmodifiableList(sampleData_); - bitField0_ = (bitField0_ & ~0x00000010); - } - result.sampleData_ = sampleData_; - } else { - result.sampleData_ = sampleDataBuilder_.build(); - } - } - - private void buildPartial0(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.actor_ = actor_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.filterId_ = filterId_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.totalEvents_ = totalEvents_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.detectedAt_ = detectedAt_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest) { - return mergeFrom((com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest other) { - if (other == com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest.getDefaultInstance()) return this; - if (!other.getActor().isEmpty()) { - actor_ = other.actor_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000002; - onChanged(); - } - if (other.getTotalEvents() != 0) { - setTotalEvents(other.getTotalEvents()); - } - if (other.getDetectedAt() != 0L) { - setDetectedAt(other.getDetectedAt()); - } - if (sampleDataBuilder_ == null) { - if (!other.sampleData_.isEmpty()) { - if (sampleData_.isEmpty()) { - sampleData_ = other.sampleData_; - bitField0_ = (bitField0_ & ~0x00000010); - } else { - ensureSampleDataIsMutable(); - sampleData_.addAll(other.sampleData_); - } - onChanged(); - } - } else { - if (!other.sampleData_.isEmpty()) { - if (sampleDataBuilder_.isEmpty()) { - sampleDataBuilder_.dispose(); - sampleDataBuilder_ = null; - sampleData_ = other.sampleData_; - bitField0_ = (bitField0_ & ~0x00000010); - sampleDataBuilder_ = - com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getSampleDataFieldBuilder() : null; - } else { - sampleDataBuilder_.addAllMessages(other.sampleData_); - } - } - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - actor_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000002; - break; - } // case 18 - case 24: { - totalEvents_ = input.readInt32(); - bitField0_ |= 0x00000004; - break; - } // case 24 - case 32: { - detectedAt_ = input.readInt64(); - bitField0_ |= 0x00000008; - break; - } // case 32 - case 42: { - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent m = - input.readMessage( - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.parser(), - extensionRegistry); - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.add(m); - } else { - sampleDataBuilder_.addMessage(m); - } - break; - } // case 42 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The actor to set. - * @return This builder for chaining. - */ - public Builder setActor( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @return This builder for chaining. - */ - public Builder clearActor() { - actor_ = getDefaultInstance().getActor(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The bytes for actor to set. - * @return This builder for chaining. - */ - public Builder setActorBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - private int totalEvents_ ; - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @return The totalEvents. - */ - @java.lang.Override - public int getTotalEvents() { - return totalEvents_; - } - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @param value The totalEvents to set. - * @return This builder for chaining. - */ - public Builder setTotalEvents(int value) { - - totalEvents_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @return This builder for chaining. - */ - public Builder clearTotalEvents() { - bitField0_ = (bitField0_ & ~0x00000004); - totalEvents_ = 0; - onChanged(); - return this; - } - - private long detectedAt_ ; - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @param value The detectedAt to set. - * @return This builder for chaining. - */ - public Builder setDetectedAt(long value) { - - detectedAt_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @return This builder for chaining. - */ - public Builder clearDetectedAt() { - bitField0_ = (bitField0_ & ~0x00000008); - detectedAt_ = 0L; - onChanged(); - return this; - } - - private java.util.List sampleData_ = - java.util.Collections.emptyList(); - private void ensureSampleDataIsMutable() { - if (!((bitField0_ & 0x00000010) != 0)) { - sampleData_ = new java.util.ArrayList(sampleData_); - bitField0_ |= 0x00000010; - } - } - - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder> sampleDataBuilder_; - - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public java.util.List getSampleDataList() { - if (sampleDataBuilder_ == null) { - return java.util.Collections.unmodifiableList(sampleData_); - } else { - return sampleDataBuilder_.getMessageList(); - } - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public int getSampleDataCount() { - if (sampleDataBuilder_ == null) { - return sampleData_.size(); - } else { - return sampleDataBuilder_.getCount(); - } - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getSampleData(int index) { - if (sampleDataBuilder_ == null) { - return sampleData_.get(index); - } else { - return sampleDataBuilder_.getMessage(index); - } - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder setSampleData( - int index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent value) { - if (sampleDataBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleDataIsMutable(); - sampleData_.set(index, value); - onChanged(); - } else { - sampleDataBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder setSampleData( - int index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder builderForValue) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.set(index, builderForValue.build()); - onChanged(); - } else { - sampleDataBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addSampleData(com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent value) { - if (sampleDataBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleDataIsMutable(); - sampleData_.add(value); - onChanged(); - } else { - sampleDataBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addSampleData( - int index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent value) { - if (sampleDataBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleDataIsMutable(); - sampleData_.add(index, value); - onChanged(); - } else { - sampleDataBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addSampleData( - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder builderForValue) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.add(builderForValue.build()); - onChanged(); - } else { - sampleDataBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addSampleData( - int index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder builderForValue) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.add(index, builderForValue.build()); - onChanged(); - } else { - sampleDataBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addAllSampleData( - java.lang.Iterable values) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, sampleData_); - onChanged(); - } else { - sampleDataBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder clearSampleData() { - if (sampleDataBuilder_ == null) { - sampleData_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000010); - onChanged(); - } else { - sampleDataBuilder_.clear(); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder removeSampleData(int index) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.remove(index); - onChanged(); - } else { - sampleDataBuilder_.remove(index); - } - return this; - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder getSampleDataBuilder( - int index) { - return getSampleDataFieldBuilder().getBuilder(index); - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( - int index) { - if (sampleDataBuilder_ == null) { - return sampleData_.get(index); } else { - return sampleDataBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public java.util.List - getSampleDataOrBuilderList() { - if (sampleDataBuilder_ != null) { - return sampleDataBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(sampleData_); - } - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder addSampleDataBuilder() { - return getSampleDataFieldBuilder().addBuilder( - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.getDefaultInstance()); - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder addSampleDataBuilder( - int index) { - return getSampleDataFieldBuilder().addBuilder( - index, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.getDefaultInstance()); - } - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public java.util.List - getSampleDataBuilderList() { - return getSampleDataFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder> - getSampleDataFieldBuilder() { - if (sampleDataBuilder_ == null) { - sampleDataBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder>( - sampleData_, - ((bitField0_ & 0x00000010) != 0), - getParentForChildren(), - isClean()); - sampleData_ = null; - } - return sampleDataBuilder_; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.consumer_service.v1.RecordAlertRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.consumer_service.v1.RecordAlertRequest) - private static final com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest(); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RecordAlertRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequestOrBuilder.java deleted file mode 100644 index 108e4ea234..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertRequestOrBuilder.java +++ /dev/null @@ -1,71 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.consumer_service.v1; - -public interface RecordAlertRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.consumer_service.v1.RecordAlertRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - java.lang.String getActor(); - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - com.google.protobuf.ByteString - getActorBytes(); - - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - java.lang.String getFilterId(); - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - com.google.protobuf.ByteString - getFilterIdBytes(); - - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @return The totalEvents. - */ - int getTotalEvents(); - - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - long getDetectedAt(); - - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - java.util.List - getSampleDataList(); - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getSampleData(int index); - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - int getSampleDataCount(); - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - java.util.List - getSampleDataOrBuilderList(); - /** - * repeated .threat_protection.service.consumer_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( - int index); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponse.java deleted file mode 100644 index dcc5c5325c..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponse.java +++ /dev/null @@ -1,358 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.consumer_service.v1; - -/** - * Protobuf type {@code threat_protection.service.consumer_service.v1.RecordAlertResponse} - */ -public final class RecordAlertResponse extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.consumer_service.v1.RecordAlertResponse) - RecordAlertResponseOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - RecordAlertResponse.class.getName()); - } - // Use RecordAlertResponse.newBuilder() to construct. - private RecordAlertResponse(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private RecordAlertResponse() { - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.class, com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.Builder.class); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse other = (com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse) obj; - - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.consumer_service.v1.RecordAlertResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.consumer_service.v1.RecordAlertResponse) - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.class, com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_RecordAlertResponse_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse build() { - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse buildPartial() { - com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse result = new com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse(this); - onBuilt(); - return result; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse) { - return mergeFrom((com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse other) { - if (other == com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse.getDefaultInstance()) return this; - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.consumer_service.v1.RecordAlertResponse) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.consumer_service.v1.RecordAlertResponse) - private static final com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse(); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RecordAlertResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.RecordAlertResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponseOrBuilder.java deleted file mode 100644 index 2c1bd6c68c..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/RecordAlertResponseOrBuilder.java +++ /dev/null @@ -1,11 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.consumer_service.v1; - -public interface RecordAlertResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.consumer_service.v1.RecordAlertResponse) - com.google.protobuf.MessageOrBuilder { -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEvent.java deleted file mode 100644 index 78a761f47d..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEvent.java +++ /dev/null @@ -1,1042 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.consumer_service.v1; - -/** - * Protobuf type {@code threat_protection.service.consumer_service.v1.SampleMaliciousEvent} - */ -public final class SampleMaliciousEvent extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) - SampleMaliciousEventOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SampleMaliciousEvent.class.getName()); - } - // Use SampleMaliciousEvent.newBuilder() to construct. - private SampleMaliciousEvent(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SampleMaliciousEvent() { - ip_ = ""; - url_ = ""; - method_ = ""; - payload_ = ""; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.class, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder.class); - } - - public static final int IP_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object ip_ = ""; - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - @java.lang.Override - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TIMESTAMP_FIELD_NUMBER = 2; - private long timestamp_ = 0L; - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - - public static final int URL_FIELD_NUMBER = 3; - @SuppressWarnings("serial") - private volatile java.lang.Object url_ = ""; - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - @java.lang.Override - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } - } - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int METHOD_FIELD_NUMBER = 4; - @SuppressWarnings("serial") - private volatile java.lang.Object method_ = ""; - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - @java.lang.Override - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } - } - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int API_COLLECTION_ID_FIELD_NUMBER = 5; - private int apiCollectionId_ = 0; - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - - public static final int PAYLOAD_FIELD_NUMBER = 6; - @SuppressWarnings("serial") - private volatile java.lang.Object payload_ = ""; - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - @java.lang.Override - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, ip_); - } - if (timestamp_ != 0L) { - output.writeInt64(2, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 3, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 4, method_); - } - if (apiCollectionId_ != 0) { - output.writeInt32(5, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 6, payload_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, ip_); - } - if (timestamp_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(2, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(3, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(4, method_); - } - if (apiCollectionId_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(5, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(6, payload_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent other = (com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent) obj; - - if (!getIp() - .equals(other.getIp())) return false; - if (getTimestamp() - != other.getTimestamp()) return false; - if (!getUrl() - .equals(other.getUrl())) return false; - if (!getMethod() - .equals(other.getMethod())) return false; - if (getApiCollectionId() - != other.getApiCollectionId()) return false; - if (!getPayload() - .equals(other.getPayload())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + IP_FIELD_NUMBER; - hash = (53 * hash) + getIp().hashCode(); - hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getTimestamp()); - hash = (37 * hash) + URL_FIELD_NUMBER; - hash = (53 * hash) + getUrl().hashCode(); - hash = (37 * hash) + METHOD_FIELD_NUMBER; - hash = (53 * hash) + getMethod().hashCode(); - hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; - hash = (53 * hash) + getApiCollectionId(); - hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; - hash = (53 * hash) + getPayload().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.consumer_service.v1.SampleMaliciousEvent} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.class, com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - ip_ = ""; - timestamp_ = 0L; - url_ = ""; - method_ = ""; - apiCollectionId_ = 0; - payload_ = ""; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.consumer_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_consumer_service_v1_SampleMaliciousEvent_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent build() { - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent buildPartial() { - com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent result = new com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.ip_ = ip_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.timestamp_ = timestamp_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.url_ = url_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.method_ = method_; - } - if (((from_bitField0_ & 0x00000010) != 0)) { - result.apiCollectionId_ = apiCollectionId_; - } - if (((from_bitField0_ & 0x00000020) != 0)) { - result.payload_ = payload_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent) { - return mergeFrom((com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent other) { - if (other == com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent.getDefaultInstance()) return this; - if (!other.getIp().isEmpty()) { - ip_ = other.ip_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (other.getTimestamp() != 0L) { - setTimestamp(other.getTimestamp()); - } - if (!other.getUrl().isEmpty()) { - url_ = other.url_; - bitField0_ |= 0x00000004; - onChanged(); - } - if (!other.getMethod().isEmpty()) { - method_ = other.method_; - bitField0_ |= 0x00000008; - onChanged(); - } - if (other.getApiCollectionId() != 0) { - setApiCollectionId(other.getApiCollectionId()); - } - if (!other.getPayload().isEmpty()) { - payload_ = other.payload_; - bitField0_ |= 0x00000020; - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - ip_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 16: { - timestamp_ = input.readInt64(); - bitField0_ |= 0x00000002; - break; - } // case 16 - case 26: { - url_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000004; - break; - } // case 26 - case 34: { - method_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000008; - break; - } // case 34 - case 40: { - apiCollectionId_ = input.readInt32(); - bitField0_ |= 0x00000010; - break; - } // case 40 - case 50: { - payload_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000020; - break; - } // case 50 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object ip_ = ""; - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @param value The ip to set. - * @return This builder for chaining. - */ - public Builder setIp( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ip_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string ip = 1 [json_name = "ip"]; - * @return This builder for chaining. - */ - public Builder clearIp() { - ip_ = getDefaultInstance().getIp(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string ip = 1 [json_name = "ip"]; - * @param value The bytes for ip to set. - * @return This builder for chaining. - */ - public Builder setIpBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ip_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private long timestamp_ ; - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @param value The timestamp to set. - * @return This builder for chaining. - */ - public Builder setTimestamp(long value) { - - timestamp_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return This builder for chaining. - */ - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000002); - timestamp_ = 0L; - onChanged(); - return this; - } - - private java.lang.Object url_ = ""; - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string url = 3 [json_name = "url"]; - * @param value The url to set. - * @return This builder for chaining. - */ - public Builder setUrl( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - url_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * string url = 3 [json_name = "url"]; - * @return This builder for chaining. - */ - public Builder clearUrl() { - url_ = getDefaultInstance().getUrl(); - bitField0_ = (bitField0_ & ~0x00000004); - onChanged(); - return this; - } - /** - * string url = 3 [json_name = "url"]; - * @param value The bytes for url to set. - * @return This builder for chaining. - */ - public Builder setUrlBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - url_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - - private java.lang.Object method_ = ""; - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string method = 4 [json_name = "method"]; - * @param value The method to set. - * @return This builder for chaining. - */ - public Builder setMethod( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - method_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * string method = 4 [json_name = "method"]; - * @return This builder for chaining. - */ - public Builder clearMethod() { - method_ = getDefaultInstance().getMethod(); - bitField0_ = (bitField0_ & ~0x00000008); - onChanged(); - return this; - } - /** - * string method = 4 [json_name = "method"]; - * @param value The bytes for method to set. - * @return This builder for chaining. - */ - public Builder setMethodBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - method_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - - private int apiCollectionId_ ; - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @param value The apiCollectionId to set. - * @return This builder for chaining. - */ - public Builder setApiCollectionId(int value) { - - apiCollectionId_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return This builder for chaining. - */ - public Builder clearApiCollectionId() { - bitField0_ = (bitField0_ & ~0x00000010); - apiCollectionId_ = 0; - onChanged(); - return this; - } - - private java.lang.Object payload_ = ""; - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @param value The payload to set. - * @return This builder for chaining. - */ - public Builder setPayload( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - payload_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - /** - * string payload = 6 [json_name = "payload"]; - * @return This builder for chaining. - */ - public Builder clearPayload() { - payload_ = getDefaultInstance().getPayload(); - bitField0_ = (bitField0_ & ~0x00000020); - onChanged(); - return this; - } - /** - * string payload = 6 [json_name = "payload"]; - * @param value The bytes for payload to set. - * @return This builder for chaining. - */ - public Builder setPayloadBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - payload_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) - private static final com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent(); - } - - public static com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SampleMaliciousEvent parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.consumer_service.v1.SampleMaliciousEvent getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEventOrBuilder.java deleted file mode 100644 index 52b1b9da96..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/consumer_service/v1/SampleMaliciousEventOrBuilder.java +++ /dev/null @@ -1,71 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/consumer_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.consumer_service.v1; - -public interface SampleMaliciousEventOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.consumer_service.v1.SampleMaliciousEvent) - com.google.protobuf.MessageOrBuilder { - - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - java.lang.String getIp(); - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - com.google.protobuf.ByteString - getIpBytes(); - - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - long getTimestamp(); - - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - java.lang.String getUrl(); - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - com.google.protobuf.ByteString - getUrlBytes(); - - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - java.lang.String getMethod(); - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - com.google.protobuf.ByteString - getMethodBytes(); - - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - int getApiCollectionId(); - - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - java.lang.String getPayload(); - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - com.google.protobuf.ByteString - getPayloadBytes(); -} diff --git a/protobuf/threat_protection/service/consumer_service/v1/consumer_service.proto b/protobuf/threat_protection/service/malicious_alert_service/v1/consumer_service.proto similarity index 65% rename from protobuf/threat_protection/service/consumer_service/v1/consumer_service.proto rename to protobuf/threat_protection/service/malicious_alert_service/v1/consumer_service.proto index f7648d1dee..19f3c36ba4 100644 --- a/protobuf/threat_protection/service/consumer_service/v1/consumer_service.proto +++ b/protobuf/threat_protection/service/malicious_alert_service/v1/consumer_service.proto @@ -1,11 +1,11 @@ syntax = "proto3"; -package threat_protection.service.consumer_service.v1; +package threat_protection.service.malicious_alert_service.v1; -// This is a consumer service for saving malicious and smart events. +// This is a consumer service for recording malicious alerts // For dashboard purposes we will have a separate service to retrieve these events. -option java_outer_classname = "ConsumerServiceProto"; -option java_package = "threat_protection.service.consumer_service.v1"; +option java_outer_classname = "MaliciousAlertServiceProto"; +option java_package = "threat_protection.service.malicious_alert_service.v1"; message RecordAlertResponse { } @@ -27,6 +27,6 @@ message RecordAlertRequest { repeated SampleMaliciousEvent sample_data = 5; } -service ConsumerService { +service MaliciousAlertService { rpc RecordAlert(RecordAlertRequest) returns (RecordAlertResponse) {} } \ No newline at end of file diff --git a/scripts/proto-gen.sh b/scripts/proto-gen.sh new file mode 100644 index 0000000000..1a816e4586 --- /dev/null +++ b/scripts/proto-gen.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +# Check if buf is installed or not +if ! command -v buf &> /dev/null +then + echo "buf is not installed. Please install buf by following the instructions at https://docs.buf.build/installation" + exit +fi + +buf lint protobuf +rm -rf ./libs/protobuf/src +buf generate protobuf \ No newline at end of file From d3704b793fc320a321062e39ace5e1dbc60f8d48 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 29 Nov 2024 17:41:13 +0530 Subject: [PATCH 36/73] added clean up for deleting all the malicious entries older than 7 days --- .../java/com/akto/threat/detection/Main.java | 2 ++ .../db/malicious_event/MaliciousEventDao.java | 8 +++++ .../threat/detection/tasks/CleanupTask.java | 35 +++++++++++++++++++ .../V2__create_malicious_event_table.sql | 4 +-- 4 files changed, 47 insertions(+), 2 deletions(-) create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index b59fd2b897..c11ed062a1 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -5,6 +5,7 @@ import com.akto.threat.detection.config.kafka.KafkaConsumerConfig; import com.akto.threat.detection.config.kafka.KafkaProducerConfig; import com.akto.threat.detection.constants.KafkaTopic; +import com.akto.threat.detection.tasks.CleanupTask; import com.akto.threat.detection.tasks.FlushSampleDataTask; import com.akto.threat.detection.tasks.MaliciousTrafficDetectorTask; import com.akto.threat.detection.tasks.SendAlertsToBackend; @@ -54,6 +55,7 @@ public static void main(String[] args) throws Exception { new MaliciousTrafficDetectorTask(trafficKafka, internalKafka, createRedisClient()).run(); new FlushSampleDataTask(postgres, internalKafka, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS).run(); new SendAlertsToBackend(postgres, internalKafka, KafkaTopic.ThreatDetection.ALERTS).run(); + new CleanupTask(postgres).run(); } public static RedisClient createRedisClient() { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java index 906c0785a4..2c88d3d25a 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java @@ -6,6 +6,7 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; +import java.time.LocalDate; import java.util.ArrayList; import java.util.List; @@ -88,4 +89,11 @@ public int countTotalMaliciousEventGivenActorAndFilterId(String actor, String fi } return 0; } + + public void deleteEventsBefore(LocalDate date) throws SQLException { + String sql = "DELETE FROM threat_detection.malicious_event WHERE created_at < ?"; + PreparedStatement stmt = this.conn.prepareStatement(sql); + stmt.setDate(1, java.sql.Date.valueOf(date)); + stmt.executeUpdate(); + } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java new file mode 100644 index 0000000000..2b30eaf8ad --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java @@ -0,0 +1,35 @@ +package com.akto.threat.detection.tasks; + +import java.sql.Connection; +import java.sql.SQLException; +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; + +public class CleanupTask implements Task { + + private final MaliciousEventDao maliciousEventDao; + private final ScheduledExecutorService cronExecutorService = Executors.newScheduledThreadPool(1); + + public CleanupTask(Connection conn) { + this.maliciousEventDao = new MaliciousEventDao(conn); + } + + @Override + public void run() { + this.cronExecutorService.scheduleAtFixedRate(this::cleanup, 5, 10 * 60, TimeUnit.SECONDS); + } + + private void cleanup() { + // Delete all records older than 7 days + try { + this.maliciousEventDao.deleteEventsBefore(LocalDate.now(ZoneOffset.UTC).minusDays(7)); + } catch (SQLException e) { + e.printStackTrace(); + } + } +} diff --git a/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql b/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql index a860ee20b1..a346d0bab5 100644 --- a/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql +++ b/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql @@ -1,6 +1,6 @@ -- create schema and table for malicious events create schema if not exists threat_detection; -create table if not exists threat_detection.malicious_events ( +create table if not exists threat_detection.malicious_event ( id uuid primary key default uuid_generate_v4(), actor varchar(255) not null, filter_id varchar(255) not null, @@ -13,4 +13,4 @@ create table if not exists threat_detection.malicious_events ( ); -- add index on actor and filter_id and sort data by timestamp -create index malicious_events_actor_filter_id_timestamp_idx on threat_detection.malicious_events(actor, filter_id, timestamp desc); +create index malicious_events_actor_filter_id_timestamp_idx on threat_detection.malicious_event(actor, filter_id, timestamp desc); From 89fec06d78d8ee3e2b660785f9334ea2cda78ca3 Mon Sep 17 00:00:00 2001 From: Ajinkya <109141486+ag060@users.noreply.github.com> Date: Tue, 10 Dec 2024 11:01:50 +0530 Subject: [PATCH 37/73] added hibernate orm (#1786) --- apps/threat-detection/pom.xml | 25 ++- .../java/com/akto/threat/detection/Main.java | 23 +-- .../db/entity/MaliciousEventEntity.java | 189 ++++++++++++++++++ .../db/malicious_event/MaliciousEventDao.java | 99 --------- .../malicious_event/MaliciousEventModel.java | 164 --------------- .../session_factory/SessionFactoryUtils.java | 30 +++ .../threat/detection/tasks/CleanupTask.java | 31 +-- .../detection/tasks/FlushSampleDataTask.java | 39 ++-- .../detection/tasks/SendAlertsToBackend.java | 76 ++++--- .../src/main/resources/log4j2.xml | 41 ++++ .../{consumer_service.proto => service.proto} | 2 +- 11 files changed, 382 insertions(+), 337 deletions(-) create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java delete mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java delete mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java create mode 100644 apps/threat-detection/src/main/resources/log4j2.xml rename protobuf/threat_protection/service/malicious_alert_service/v1/{consumer_service.proto => service.proto} (96%) diff --git a/apps/threat-detection/pom.xml b/apps/threat-detection/pom.xml index 847adc1668..a24201ca33 100644 --- a/apps/threat-detection/pom.xml +++ b/apps/threat-detection/pom.xml @@ -107,6 +107,24 @@ 9.22.3 + + org.hibernate + hibernate-core + 5.6.15.Final + + + + org.apache.logging.log4j + log4j-core + 2.24.2 + + + + org.apache.logging.log4j + log4j-api + 2.24.2 + + @@ -142,11 +160,12 @@ src/main/resources - true - + false + diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index c11ed062a1..d49b27672d 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -5,6 +5,7 @@ import com.akto.threat.detection.config.kafka.KafkaConsumerConfig; import com.akto.threat.detection.config.kafka.KafkaProducerConfig; import com.akto.threat.detection.constants.KafkaTopic; +import com.akto.threat.detection.session_factory.SessionFactoryUtils; import com.akto.threat.detection.tasks.CleanupTask; import com.akto.threat.detection.tasks.FlushSampleDataTask; import com.akto.threat.detection.tasks.MaliciousTrafficDetectorTask; @@ -12,11 +13,8 @@ import com.mongodb.ConnectionString; import io.lettuce.core.RedisClient; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; - import org.flywaydb.core.Flyway; +import org.hibernate.SessionFactory; public class Main { @@ -25,6 +23,8 @@ public class Main { public static void main(String[] args) throws Exception { runMigrations(); + SessionFactory sessionFactory = SessionFactoryUtils.createFactory(); + DaoInit.init(new ConnectionString(System.getenv("AKTO_MONGO_CONN"))); KafkaConfig trafficKafka = KafkaConfig.newBuilder() .setGroupId(CONSUMER_GROUP_ID) @@ -50,25 +50,16 @@ public static void main(String[] args) throws Exception { KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) .build(); - Connection postgres = createPostgresConnection(); - new MaliciousTrafficDetectorTask(trafficKafka, internalKafka, createRedisClient()).run(); - new FlushSampleDataTask(postgres, internalKafka, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS).run(); - new SendAlertsToBackend(postgres, internalKafka, KafkaTopic.ThreatDetection.ALERTS).run(); - new CleanupTask(postgres).run(); + new FlushSampleDataTask(sessionFactory, internalKafka, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS).run(); + new SendAlertsToBackend(sessionFactory, internalKafka, KafkaTopic.ThreatDetection.ALERTS).run(); + new CleanupTask(sessionFactory).run(); } public static RedisClient createRedisClient() { return RedisClient.create(System.getenv("AKTO_THREAT_DETECTION_REDIS_URI")); } - public static Connection createPostgresConnection() throws SQLException { - String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); - String user = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_USER"); - String password = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_PASSWORD"); - return DriverManager.getConnection(url, user, password); - } - public static void runMigrations() { String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); String user = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_USER"); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java new file mode 100644 index 0000000000..aff882819c --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java @@ -0,0 +1,189 @@ +package com.akto.threat.detection.db.entity; + +import com.akto.dto.type.URLMethods; + +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.UUID; + +import javax.persistence.Column; +import javax.persistence.Entity; +import javax.persistence.EnumType; +import javax.persistence.Enumerated; +import javax.persistence.GeneratedValue; +import javax.persistence.Id; +import javax.persistence.PrePersist; +import javax.persistence.Table; + +import org.hibernate.annotations.GenericGenerator; + +@Entity +@Table(name = "malicious_event", schema = "threat_detection") +public class MaliciousEventEntity { + + @Id + @GeneratedValue(generator = "UUID") + @GenericGenerator(name = "UUID", strategy = "org.hibernate.id.UUIDGenerator") + private UUID id; + + @Column(name = "actor") + private String actor; + + @Column(name = "filter_id") + private String filterId; + + @Column(name = "url") + private String url; + + @Column(name = "method") + @Enumerated(EnumType.STRING) + private URLMethods.Method method; + + @Column(name = "timestamp") + private long timestamp; + + @Column(name = "orig") + private String orig; + + // Geo location data + @Column(name = "ip") + private String ip; + + @Column(name = "created_at", updatable = false) + private LocalDateTime createdAt; + + public MaliciousEventEntity() { + } + + @PrePersist + protected void onCreate() { + this.createdAt = LocalDateTime.now(ZoneOffset.UTC); + } + + public MaliciousEventEntity(Builder builder) { + this.actor = builder.actorId; + this.filterId = builder.filterId; + this.url = builder.url; + this.method = builder.method; + this.timestamp = builder.timestamp; + this.orig = builder.orig; + this.ip = builder.ip; + } + + public static class Builder { + private String actorId; + private String filterId; + private String url; + private URLMethods.Method method; + private long timestamp; + private String orig; + private String ip; + + public Builder setActor(String actorId) { + this.actorId = actorId; + return this; + } + + public Builder setFilterId(String filterId) { + this.filterId = filterId; + return this; + } + + public Builder setUrl(String url) { + this.url = url; + return this; + } + + public Builder setMethod(URLMethods.Method method) { + this.method = method; + return this; + } + + public Builder setTimestamp(long timestamp) { + this.timestamp = timestamp; + return this; + } + + public Builder setOrig(String orig) { + this.orig = orig; + return this; + } + + public Builder setIp(String ip) { + this.ip = ip; + return this; + } + + public MaliciousEventEntity build() { + return new MaliciousEventEntity(this); + } + } + + public static Builder newBuilder() { + return new Builder(); + } + + public UUID getId() { + return id; + } + + public String getActor() { + return actor; + } + + public String getFilterId() { + return filterId; + } + + public String getUrl() { + return url; + } + + public URLMethods.Method getMethod() { + return method; + } + + public long getTimestamp() { + return timestamp; + } + + public String getOrig() { + return orig; + } + + public String getIp() { + return ip; + } + + public LocalDateTime getCreatedAt() { + return createdAt; + } + + @Override + public String toString() { + return "MaliciousEventModel{" + + "id='" + + id + + '\'' + + ", actorId='" + + actor + + '\'' + + ", filterId='" + + filterId + + '\'' + + ", url='" + + url + + '\'' + + ", method=" + + method + + ", timestamp=" + + timestamp + + ", orig='" + + orig + + '\'' + + ", ip='" + + ip + + '\'' + + '}'; + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java deleted file mode 100644 index 2c88d3d25a..0000000000 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventDao.java +++ /dev/null @@ -1,99 +0,0 @@ -package com.akto.threat.detection.db.malicious_event; - -import com.akto.dto.type.URLMethods; - -import java.sql.Connection; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.time.LocalDate; -import java.util.ArrayList; -import java.util.List; - -public class MaliciousEventDao { - - private final Connection conn; - private static final int BATCH_SIZE = 50; - - public MaliciousEventDao(Connection conn) { - this.conn = conn; - } - - public void batchInsert(List events) throws SQLException { - String sql = - "INSERT INTO threat_detection.malicious_event (id, actor, filter_id, url, method, timestamp, orig, ip, country) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)"; - conn.setAutoCommit(false); - for (int i = 0; i < events.size(); i++) { - MaliciousEventModel event = events.get(i); - PreparedStatement stmt = this.conn.prepareStatement(sql); - stmt.setString(1, event.getId()); - stmt.setString(2, event.getActor()); - stmt.setString(3, event.getFilterId()); - stmt.setString(4, event.getUrl()); - stmt.setString(5, event.getMethod().name()); - stmt.setLong(6, event.getTimestamp()); - stmt.setString(7, event.getOrig()); - stmt.setString(8, event.getIp()); - - stmt.addBatch(); - - if (i % BATCH_SIZE == 0 || i == events.size() - 1) { - stmt.executeBatch(); - stmt.clearBatch(); - } - } - - conn.commit(); - } - - public List findGivenActorAndFilterId( - String actor, String filterId, int limit) throws SQLException { - String sql = - "SELECT * FROM threat_detection.malicious_event WHERE actor = ? AND filter_id = ? LIMIT ?"; - PreparedStatement stmt = this.conn.prepareStatement(sql); - stmt.setString(1, actor); - stmt.setString(2, filterId); - stmt.setInt(3, limit); - try (ResultSet rs = stmt.executeQuery()) { - List models = new ArrayList<>(); - while (rs.next()) { - MaliciousEventModel model = - MaliciousEventModel.newBuilder() - .setId(rs.getString("id")) - .setActorId(rs.getString("actor")) - .setFilterId(rs.getString("filter_id")) - .setUrl(rs.getString("url")) - .setMethod(URLMethods.Method.fromString(rs.getString("method"))) - .setTimestamp(rs.getLong("timestamp")) - .setOrig(rs.getString("orig")) - .setIp(rs.getString("ip")) - .setCreatedAt(rs.getDate("created_at").toLocalDate()) - .build(); - models.add(model); - } - return models; - } - } - - public int countTotalMaliciousEventGivenActorAndFilterId(String actor, String filterId) - throws SQLException { - String sql = - "SELECT COUNT(*) FROM threat_detection.malicious_event WHERE actor = ? AND filter_id = ?"; - PreparedStatement stmt = this.conn.prepareStatement(sql); - stmt.setString(1, actor); - stmt.setString(2, filterId); - try (ResultSet rs = stmt.executeQuery()) { - if (rs.next()) { - return rs.getInt(1); - } - } - return 0; - } - - public void deleteEventsBefore(LocalDate date) throws SQLException { - String sql = "DELETE FROM threat_detection.malicious_event WHERE created_at < ?"; - PreparedStatement stmt = this.conn.prepareStatement(sql); - stmt.setDate(1, java.sql.Date.valueOf(date)); - stmt.executeUpdate(); - } -} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java deleted file mode 100644 index adbf25bc33..0000000000 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/malicious_event/MaliciousEventModel.java +++ /dev/null @@ -1,164 +0,0 @@ -package com.akto.threat.detection.db.malicious_event; - -import com.akto.dto.type.URLMethods; - -import java.time.LocalDate; - -public class MaliciousEventModel { - - private String id; - private String actor; - private String filterId; - private String url; - private URLMethods.Method method; - private long timestamp; - private String orig; - - // Geo location data - private String ip; - - private LocalDate createdAt; - - public MaliciousEventModel() {} - - public MaliciousEventModel(Builder builder) { - this.id = builder.id; - this.actor = builder.actorId; - this.filterId = builder.filterId; - this.url = builder.url; - this.method = builder.method; - this.timestamp = builder.timestamp; - this.orig = builder.orig; - this.ip = builder.ip; - this.createdAt = builder.createdAt; - } - - public static class Builder { - private String id; - private String actorId; - private String filterId; - private String url; - private URLMethods.Method method; - private long timestamp; - private String orig; - private String ip; - private LocalDate createdAt; - - public Builder setId(String id) { - this.id = id; - return this; - } - - public Builder setActorId(String actorId) { - this.actorId = actorId; - return this; - } - - public Builder setFilterId(String filterId) { - this.filterId = filterId; - return this; - } - - public Builder setUrl(String url) { - this.url = url; - return this; - } - - public Builder setMethod(URLMethods.Method method) { - this.method = method; - return this; - } - - public Builder setTimestamp(long timestamp) { - this.timestamp = timestamp; - return this; - } - - public Builder setOrig(String orig) { - this.orig = orig; - return this; - } - - public Builder setIp(String ip) { - this.ip = ip; - return this; - } - - public MaliciousEventModel build() { - return new MaliciousEventModel(this); - } - - public Builder setCreatedAt(LocalDate createdAt) { - this.createdAt = createdAt; - return this; - } - } - - public static Builder newBuilder() { - return new Builder(); - } - - public String getId() { - return id; - } - - public String getActor() { - return actor; - } - - public String getFilterId() { - return filterId; - } - - public String getUrl() { - return url; - } - - public URLMethods.Method getMethod() { - return method; - } - - public long getTimestamp() { - return timestamp; - } - - public String getOrig() { - return orig; - } - - public String getIp() { - return ip; - } - - public LocalDate getCreatedAt() { - return createdAt; - } - - @Override - public String toString() { - return "MaliciousEventModel{" - + "id='" - + id - + '\'' - + ", actorId='" - + actor - + '\'' - + ", filterId='" - + filterId - + '\'' - + ", url='" - + url - + '\'' - + ", method=" - + method - + ", timestamp=" - + timestamp - + ", orig='" - + orig - + '\'' - + ", ip='" - + ip - + '\'' - + '}'; - } -} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java new file mode 100644 index 0000000000..c725bd978c --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java @@ -0,0 +1,30 @@ +package com.akto.threat.detection.session_factory; + +import org.hibernate.SessionFactory; +import org.hibernate.boot.registry.StandardServiceRegistryBuilder; +import org.hibernate.cfg.Configuration; + +import com.akto.threat.detection.db.entity.MaliciousEventEntity; + +public class SessionFactoryUtils { + + public static SessionFactory createFactory() { + final String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); + final String user = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_USER"); + final String password = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_PASSWORD"); + + final Configuration cfg = new Configuration(); + cfg.setProperty("hibernate.connection.url", url); + cfg.setProperty("hibernate.connection.user", user); + cfg.setProperty("hibernate.connection.password", password); + cfg.setProperty("dialect", "org.hibernate.dialect.PostgreSQL92Dialect"); + cfg.setProperty("connection.driver_class", "org.postgresql.Driver"); + cfg.setProperty("show_sql", "true"); + cfg.setProperty("format_sql", "true"); + + cfg.addAnnotatedClass(MaliciousEventEntity.class); + + return cfg.buildSessionFactory(new StandardServiceRegistryBuilder().applySettings(cfg.getProperties()).build()); + } + +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java index 2b30eaf8ad..81914ef901 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java @@ -1,22 +1,23 @@ package com.akto.threat.detection.tasks; -import java.sql.Connection; -import java.sql.SQLException; -import java.time.LocalDate; +import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; +import org.hibernate.Session; +import org.hibernate.SessionFactory; +import org.hibernate.Transaction; public class CleanupTask implements Task { - private final MaliciousEventDao maliciousEventDao; + private final SessionFactory sessionFactory; + private final ScheduledExecutorService cronExecutorService = Executors.newScheduledThreadPool(1); - public CleanupTask(Connection conn) { - this.maliciousEventDao = new MaliciousEventDao(conn); + public CleanupTask(SessionFactory sessionFactory) { + this.sessionFactory = sessionFactory; } @Override @@ -25,11 +26,15 @@ public void run() { } private void cleanup() { - // Delete all records older than 7 days - try { - this.maliciousEventDao.deleteEventsBefore(LocalDate.now(ZoneOffset.UTC).minusDays(7)); - } catch (SQLException e) { - e.printStackTrace(); - } + Session session = this.sessionFactory.openSession(); + Transaction txn = session.beginTransaction(); + int deletedCount = session.createQuery("delete from MaliciousEventEntity m where m.createdAt < :startDate") + .setParameter("startDate", LocalDateTime.now(ZoneOffset.UTC).minusDays(7)) + .executeUpdate(); + + txn.commit(); + session.close(); + + System.out.println("Number of rows deleted: " + deletedCount); } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 65e31aee30..974885e379 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -3,15 +3,15 @@ import com.akto.dto.type.URLMethods; import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; import com.akto.threat.detection.config.kafka.KafkaConfig; -import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; -import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; +import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.hibernate.Session; +import org.hibernate.SessionFactory; +import org.hibernate.Transaction; -import java.sql.Connection; -import java.sql.SQLException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutorService; @@ -22,11 +22,11 @@ */ public class FlushSampleDataTask extends AbstractKafkaConsumerTask { - private final MaliciousEventDao maliciousEventDao; + private final SessionFactory sessionFactory; - public FlushSampleDataTask(Connection conn, KafkaConfig trafficConfig, String topic) { + public FlushSampleDataTask(SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { super(trafficConfig, topic); - this.maliciousEventDao = new MaliciousEventDao(conn); + this.sessionFactory = sessionFactory; } @Override @@ -35,7 +35,7 @@ ExecutorService getPollingExecutor() { } protected void processRecords(ConsumerRecords records) { - List events = new ArrayList<>(); + List events = new ArrayList<>(); records.forEach( r -> { String message = r.value(); @@ -55,8 +55,8 @@ protected void processRecords(ConsumerRecords records) { MaliciousEvent evt = builder.build(); events.add( - MaliciousEventModel.newBuilder() - .setActorId(m.getAccountId()) + MaliciousEventEntity.newBuilder() + .setActor(m.getAccountId()) .setFilterId(evt.getFilterId()) .setUrl(evt.getUrl()) .setMethod(URLMethods.Method.fromString(evt.getMethod())) @@ -66,10 +66,21 @@ protected void processRecords(ConsumerRecords records) { .build()); }); - try { - this.maliciousEventDao.batchInsert(events); - } catch (SQLException e) { - e.printStackTrace(); + Session session = this.sessionFactory.openSession(); + Transaction txn = session.beginTransaction(); + + txn.begin(); + + // Commit these events in 2 batches + for (int i = 0; i < events.size(); i += 2) { + session.persist(events.get(i)); + if (i % 50 == 0) { + session.flush(); + session.clear(); + } } + + txn.commit(); + session.close(); } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java index ae7a0e3863..cb1c3fe0a6 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java @@ -7,8 +7,7 @@ import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent; import com.akto.threat.detection.config.kafka.KafkaConfig; -import com.akto.threat.detection.db.malicious_event.MaliciousEventDao; -import com.akto.threat.detection.db.malicious_event.MaliciousEventModel; +import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; import com.akto.threat.detection.grpc.AuthToken; import com.google.protobuf.InvalidProtocolBufferException; @@ -17,9 +16,11 @@ import io.grpc.InsecureChannelCredentials; import io.grpc.ManagedChannel; import io.grpc.stub.StreamObserver; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import java.sql.Connection; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.hibernate.Session; +import org.hibernate.SessionFactory; +import org.hibernate.Transaction; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -30,27 +31,53 @@ */ public class SendAlertsToBackend extends AbstractKafkaConsumerTask { - private final MaliciousEventDao maliciousEventDao; + private final SessionFactory sessionFactory; private final MaliciousAlertServiceStub consumerServiceStub; - public SendAlertsToBackend(Connection conn, KafkaConfig trafficConfig, String topic) { + public SendAlertsToBackend(SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { super(trafficConfig, topic); - this.maliciousEventDao = new MaliciousEventDao(conn); + this.sessionFactory = sessionFactory; String target = "localhost:8980"; - ManagedChannel channel = - Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - this.consumerServiceStub = - MaliciousAlertServiceGrpc.newStub(channel) - .withCallCredentials( - new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); + ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); + this.consumerServiceStub = MaliciousAlertServiceGrpc.newStub(channel) + .withCallCredentials( + new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); } ExecutorService getPollingExecutor() { return Executors.newSingleThreadExecutor(); } + private List getSampleMaliciousEvents(String actor, String filterId) { + Session session = this.sessionFactory.openSession(); + Transaction txn = session.beginTransaction(); + try { + return session + .createQuery("from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId order by m.createdAt desc", MaliciousEventEntity.class) + .setParameter("actor", actor).setParameter("filterId", filterId) + .setMaxResults(50) + .getResultList(); + } finally { + txn.commit(); + session.close(); + } + } + + private long getTotalEvents(String actor, String filterId) { + Session session = this.sessionFactory.openSession(); + Transaction txn = session.beginTransaction(); + try { + return session + .createQuery("select count(m) from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId", Long.class) + .setParameter("actor", actor).setParameter("filterId", filterId).uniqueResult(); + } finally { + txn.commit(); + session.close(); + } + } + protected void processRecords(ConsumerRecords records) { records.forEach( r -> { @@ -72,13 +99,9 @@ protected void processRecords(ConsumerRecords records) { // Get sample data from postgres for this alert try { - List sampleData = - this.maliciousEventDao.findGivenActorAndFilterId( - evt.getActor(), evt.getFilterId(), 50); + List sampleData = this.getSampleMaliciousEvents(evt.getActor(), evt.getFilterId()); - int totalEvents = - this.maliciousEventDao.countTotalMaliciousEventGivenActorAndFilterId( - evt.getActor(), evt.getFilterId()); + long totalEvents = this.getTotalEvents(evt.getActor(), evt.getFilterId()); this.consumerServiceStub.recordAlert( RecordAlertRequest.newBuilder() @@ -88,14 +111,13 @@ protected void processRecords(ConsumerRecords records) { .addAllSampleData( sampleData.stream() .map( - d -> - SampleMaliciousEvent.newBuilder() - .setUrl(d.getUrl()) - .setMethod(d.getMethod().name()) - .setTimestamp(d.getTimestamp()) - .setPayload(d.getOrig()) - .setIp(d.getIp()) - .build()) + d -> SampleMaliciousEvent.newBuilder() + .setUrl(d.getUrl()) + .setMethod(d.getMethod().name()) + .setTimestamp(d.getTimestamp()) + .setPayload(d.getOrig()) + .setIp(d.getIp()) + .build()) .collect(Collectors.toList())) .build(), new StreamObserver() { diff --git a/apps/threat-detection/src/main/resources/log4j2.xml b/apps/threat-detection/src/main/resources/log4j2.xml new file mode 100644 index 0000000000..addedab226 --- /dev/null +++ b/apps/threat-detection/src/main/resources/log4j2.xml @@ -0,0 +1,41 @@ + + + + + + + + + + + + [%-5level] %d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %c{1} - %msg%n + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/protobuf/threat_protection/service/malicious_alert_service/v1/consumer_service.proto b/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto similarity index 96% rename from protobuf/threat_protection/service/malicious_alert_service/v1/consumer_service.proto rename to protobuf/threat_protection/service/malicious_alert_service/v1/service.proto index 19f3c36ba4..3aa4a6ee79 100644 --- a/protobuf/threat_protection/service/malicious_alert_service/v1/consumer_service.proto +++ b/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto @@ -22,7 +22,7 @@ message SampleMaliciousEvent { message RecordAlertRequest { string actor = 1; string filter_id = 2; - int32 total_events = 3; + int64 total_events = 3; int64 detected_at = 4; repeated SampleMaliciousEvent sample_data = 5; } From 3a90345126d851493eb977a6235226ed7a659c74 Mon Sep 17 00:00:00 2001 From: Ayush Agarwal <115138089+ayushaga14@users.noreply.github.com> Date: Tue, 10 Dec 2024 11:02:54 +0530 Subject: [PATCH 38/73] threat protection producer consumer db writes --- .../java/com/akto/threat/protection/Main.java | 8 + .../protection/MaliciousAlertService.java | 46 ++--- .../threat/protection/utils/KafkaUtils.java | 190 ++++++++++++++++++ 3 files changed, 219 insertions(+), 25 deletions(-) create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index 2b1dae1fcd..8bf93cf399 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -1,6 +1,7 @@ package com.akto.threat.protection; import com.akto.DaoInit; +import com.akto.threat.protection.utils.KafkaUtils; import com.mongodb.ConnectionString; import com.mongodb.ReadPreference; import com.mongodb.client.MongoClient; @@ -15,6 +16,13 @@ public static void main(String[] args) throws Exception { DaoInit.createMongoClient( new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")), ReadPreference.secondary()); + String initProducer = System.getenv("INIT_KAFKA_PRODUCER"); + if (initProducer != null && initProducer.equalsIgnoreCase("true")) { + KafkaUtils.initKafkaProducer(); + } else { + KafkaUtils.initMongoClient(threatProtectionMongo); + KafkaUtils.initKafkaConsumer(); + } int port = Integer.parseInt( diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java index f9f79bc8b6..c4664b92e7 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java @@ -3,18 +3,22 @@ import java.util.ArrayList; import java.util.List; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousAlertServiceGrpc.MaliciousAlertServiceImplBase; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.db.SmartEventModel; import com.akto.threat.protection.interceptors.Constants; +import com.akto.threat.protection.utils.KafkaUtils; import com.mongodb.client.MongoClient; import com.mongodb.client.model.BulkWriteOptions; import com.mongodb.client.model.InsertOneModel; import com.mongodb.client.model.WriteModel; import io.grpc.stub.StreamObserver; +import javassist.tools.rmi.Sample; public class MaliciousAlertService extends MaliciousAlertServiceImplBase { @@ -31,34 +35,26 @@ public void recordAlert( String actor = request.getActor(); String filterId = request.getFilterId(); List> bulkUpdates = new ArrayList<>(); - request - .getSampleDataList() - .forEach( - event -> { - bulkUpdates.add( - new InsertOneModel<>( - new MaliciousEventModel( - filterId, - actor, - event.getIp(), - event.getUrl(), - event.getMethod(), - event.getPayload(), - event.getTimestamp()))); - }); int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); - this.mongoClient - .getDatabase(accountId + "") - .getCollection("malicious_events", MaliciousEventModel.class) - .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); - this.mongoClient - .getDatabase(accountId + "") - .getCollection("smart_events", SmartEventModel.class) - .insertOne( - new SmartEventModel( - filterId, actor, request.getTotalEvents(), request.getDetectedAt())); + List maliciousEvents = request.getSampleDataList(); + List events = new ArrayList<>(); + for (SampleMaliciousEvent maliciousEvent : maliciousEvents) { + events.add(new MaliciousEventModel(filterId, + actor, + maliciousEvent.getIp(), + maliciousEvent.getUrl(), + maliciousEvent.getMethod(), + maliciousEvent.getPayload(), + maliciousEvent.getTimestamp())); + } + + KafkaUtils.insertData(events, "maliciousEvents", accountId); + KafkaUtils.insertData(new SmartEventModel(filterId, actor, request.getTotalEvents(), request.getDetectedAt()), "smartEvent", accountId); + + responseObserver.onNext(RecordAlertResponse.newBuilder().build()); responseObserver.onCompleted(); } + } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java new file mode 100644 index 0000000000..f772288a91 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java @@ -0,0 +1,190 @@ +package com.akto.threat.protection.utils; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.akto.dao.context.Context; +import com.akto.kafka.Kafka; +import com.akto.log.LoggerMaker; +import com.akto.log.LoggerMaker.LogDb; +import com.akto.threat.protection.db.MaliciousEventModel; +import com.akto.threat.protection.db.SmartEventModel; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import com.mongodb.BasicDBObject; +import com.mongodb.client.MongoClient; +import com.mongodb.client.model.BulkWriteOptions; +import com.mongodb.client.model.InsertOneModel; +import com.mongodb.client.model.WriteModel; + +public class KafkaUtils { + + private static Kafka kafkaProducer; + private static Consumer consumer; + private static final Logger logger = LoggerFactory.getLogger(KafkaUtils.class); + private static final LoggerMaker loggerMaker = new LoggerMaker(KafkaUtils.class); + private static final Gson gson = new Gson(); + private static MongoClient mClient; + private final static ObjectMapper mapper = new ObjectMapper(); + private static long lastSyncOffset = 0; + + public static void initKafkaProducer() { + String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); + int batchSize = Integer.parseInt(System.getenv("AKTO_KAFKA_PRODUCER_BATCH_SIZE")); + int kafkaLingerMS = Integer.parseInt(System.getenv("AKTO_KAFKA_PRODUCER_LINGER_MS")); + kafkaProducer = new Kafka(kafkaBrokerUrl, kafkaLingerMS, batchSize); + logger.info("Kafka Producer Init " + Context.now()); + } + + public static void insertData(Object writes, String eventType, int accountId) { + String topicName = System.getenv("THREAT_EVENTS_KAFKA_TOPIC"); + BasicDBObject obj = new BasicDBObject(); + obj.put("eventType", eventType); + String payloadStr = gson.toJson(writes); + obj.put("payload", payloadStr); + obj.put("accountId", accountId); + kafkaProducer.send(obj.toString(), topicName); + } + + public static void initMongoClient(MongoClient mongoClient) { + mClient = mongoClient; + } + + public static void initKafkaConsumer() { + System.out.println("Kafka Init consumer called"); + String topicName = System.getenv("THREAT_EVENTS_KAFKA_TOPIC"); + String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); // kafka1:19092 + String isKubernetes = System.getenv("IS_KUBERNETES"); + if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { + kafkaBrokerUrl = "127.0.0.1:29092"; + } + String groupIdConfig = System.getenv("THREAT_EVENTS_KAFKA_GROUP_ID_CONFIG"); + int maxPollRecordsConfig = Integer.parseInt(System.getenv("THREAT_EVENTS_KAFKA_MAX_POLL_RECORDS_CONFIG")); + + Properties properties = configProperties(kafkaBrokerUrl, groupIdConfig, maxPollRecordsConfig); + consumer = new KafkaConsumer<>(properties); + final Thread mainThread = Thread.currentThread(); + final AtomicBoolean exceptionOnCommitSync = new AtomicBoolean(false); + + + Runtime.getRuntime().addShutdownHook(new Thread() { + public void run() { + consumer.wakeup(); + try { + if (!exceptionOnCommitSync.get()) { + mainThread.join(); + } + } catch (InterruptedException e) { + e.printStackTrace(); + } catch (Error e){ + loggerMaker.errorAndAddToDb("Error in add shut down hook: "+ e.getMessage(), LogDb.DASHBOARD); + } + } + }); + + try { + consumer.subscribe(Arrays.asList(topicName)); + loggerMaker.infoAndAddToDb("Kafka Consumer subscribed", LogDb.DASHBOARD); + while (true) { + ConsumerRecords records = consumer.poll(Duration.ofMillis(10000)); + try { + consumer.commitSync(); + } catch (Exception e) { + throw e; + } + + for (ConsumerRecord r: records) { + try { + lastSyncOffset++; + if (lastSyncOffset % 100 == 0) { + logger.info("Committing offset at position: " + lastSyncOffset); + } + + parseAndTriggerWrites(r.value()); + } catch (Exception e) { + loggerMaker.errorAndAddToDb(e, "Error in parseAndTriggerWrites " + e, LogDb.DASHBOARD); + continue; + } + } + } + } catch (WakeupException ignored) { + // nothing to catch. This exception is called from the shutdown hook. + } catch (Exception e) { + exceptionOnCommitSync.set(true); + loggerMaker.errorAndAddToDb("Exception in init kafka consumer " + e.getMessage(),LogDb.DASHBOARD); + e.printStackTrace(); + System.exit(0); + } finally { + consumer.close(); + } + + } + + private static void parseAndTriggerWrites(String message) throws Exception { + Map json = gson.fromJson(message, Map.class); + String eventType = (String) json.get("eventType"); + String payload = (String) json.get("payload"); + Double accIdDouble = (Double) json.get("accountId"); + int accountId = accIdDouble.intValue(); + Context.accountId.set(accountId); + + switch (eventType) { + case "maliciousEvents": + List> bulkUpdates = new ArrayList<>(); + List events = mapper.readValue(payload, new TypeReference>(){}); + events + .forEach( + event -> { + bulkUpdates.add( + new InsertOneModel<>( + event)); + }); + + mClient.getDatabase(accountId + "") + .getCollection("malicious_events", MaliciousEventModel.class) + .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); + break; + + case "smartEvent": + SmartEventModel event = mapper.readValue(payload, new TypeReference(){}); + mClient + .getDatabase(accountId + "") + .getCollection("smart_events", SmartEventModel.class) + .insertOne(event); + break; + default: + break; + } + } + + public static Properties configProperties(String kafkaBrokerUrl, String groupIdConfig, int maxPollRecordsConfig) { + Properties properties = new Properties(); + properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaBrokerUrl); + properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecordsConfig); + properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupIdConfig); + properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + + return properties; + } + +} From 4d3e46355650c04cc8224f143af4adf6470c1d3d Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 10 Dec 2024 11:18:54 +0530 Subject: [PATCH 39/73] added proto generation to ci --- .github/workflows/staging.yml | 7 ++++++- scripts/proto-gen.sh | 7 +------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/.github/workflows/staging.yml b/.github/workflows/staging.yml index 02920d5ba9..d2b40efa9f 100644 --- a/.github/workflows/staging.yml +++ b/.github/workflows/staging.yml @@ -25,7 +25,12 @@ jobs: architecture: x64 - uses: actions/setup-node@v2 with: - node-version: "17" + node-version: '17' + - uses: bufbuild/buf-action@v1 + with: + setup_only: true + - name: Generate Proto files + run: make proto-gen - name: Convert github branch name to be compatible with docker tag name convention and generate tag name id: docker_tag run: echo "IMAGE_TAG=a-$(echo ${{ github.ref_name }} | sed 's/[^a-zA-Z0-9]/-/g')" >> $GITHUB_OUTPUT diff --git a/scripts/proto-gen.sh b/scripts/proto-gen.sh index 1a816e4586..f05dbd0230 100644 --- a/scripts/proto-gen.sh +++ b/scripts/proto-gen.sh @@ -1,12 +1,7 @@ #!/bin/bash # Check if buf is installed or not -if ! command -v buf &> /dev/null -then - echo "buf is not installed. Please install buf by following the instructions at https://docs.buf.build/installation" - exit -fi - +# Please install buf if not already installed by following the instructions at https://docs.buf.build/installation buf lint protobuf rm -rf ./libs/protobuf/src buf generate protobuf \ No newline at end of file From 8616f4639079137e0145d99337cc22180dc37bd3 Mon Sep 17 00:00:00 2001 From: ayushaga14 Date: Tue, 10 Dec 2024 11:30:00 +0530 Subject: [PATCH 40/73] fix create mongo client params --- .../src/main/java/com/akto/threat/protection/Main.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index 8bf93cf399..dcc78a1290 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -4,6 +4,7 @@ import com.akto.threat.protection.utils.KafkaUtils; import com.mongodb.ConnectionString; import com.mongodb.ReadPreference; +import com.mongodb.WriteConcern; import com.mongodb.client.MongoClient; public class Main { @@ -15,7 +16,7 @@ public static void main(String[] args) throws Exception { MongoClient threatProtectionMongo = DaoInit.createMongoClient( new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")), - ReadPreference.secondary()); + ReadPreference.secondary(), WriteConcern.ACKNOWLEDGED); String initProducer = System.getenv("INIT_KAFKA_PRODUCER"); if (initProducer != null && initProducer.equalsIgnoreCase("true")) { KafkaUtils.initKafkaProducer(); From 95f09f0c3c2074a15655c94d9520670dd4f9c514 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 10 Dec 2024 11:53:26 +0530 Subject: [PATCH 41/73] deleted generated proto files --- .../v1/DashboardServiceGrpc.java | 367 ---- .../v1/FetchAlertFiltersRequest.java | 358 ---- .../v1/FetchAlertFiltersRequestOrBuilder.java | 11 - .../v1/FetchAlertFiltersResponse.java | 743 -------- .../FetchAlertFiltersResponseOrBuilder.java | 61 - .../v1/ListMaliciousRequestsRequest.java | 538 ------ ...ListMaliciousRequestsRequestOrBuilder.java | 32 - .../v1/ListMaliciousRequestsResponse.java | 851 --------- ...istMaliciousRequestsResponseOrBuilder.java | 47 - .../v1/MaliciousRequest.java | 1520 ----------------- .../v1/MaliciousRequestOrBuilder.java | 113 -- .../dashboard_service/v1/ServiceProto.java | 136 -- .../v1/ConsumerServiceProto.java | 107 -- .../v1/MaliciousAlertServiceGrpc.java | 293 ---- .../v1/RecordAlertRequest.java | 1124 ------------ .../v1/RecordAlertRequestOrBuilder.java | 71 - .../v1/RecordAlertResponse.java | 358 ---- .../v1/RecordAlertResponseOrBuilder.java | 11 - .../v1/SampleMaliciousEvent.java | 1042 ----------- .../v1/SampleMaliciousEventOrBuilder.java | 71 - 20 files changed, 7854 deletions(-) delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/DashboardServiceGrpc.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponse.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponseOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponse.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponseOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ServiceProto.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ConsumerServiceProto.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousAlertServiceGrpc.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponse.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponseOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEvent.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEventOrBuilder.java diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/DashboardServiceGrpc.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/DashboardServiceGrpc.java deleted file mode 100644 index 002d8e8a71..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/DashboardServiceGrpc.java +++ /dev/null @@ -1,367 +0,0 @@ -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -import static io.grpc.MethodDescriptor.generateFullMethodName; - -/** - */ -@javax.annotation.Generated( - value = "by gRPC proto compiler (version 1.68.1)", - comments = "Source: threat_protection/service/dashboard_service/v1/service.proto") -@io.grpc.stub.annotations.GrpcGenerated -public final class DashboardServiceGrpc { - - private DashboardServiceGrpc() {} - - public static final java.lang.String SERVICE_NAME = "threat_protection.service.dashboard_service.v1.DashboardService"; - - // Static method descriptors that strictly reflect the proto. - private static volatile io.grpc.MethodDescriptor getListMaliciousRequestsMethod; - - @io.grpc.stub.annotations.RpcMethod( - fullMethodName = SERVICE_NAME + '/' + "ListMaliciousRequests", - requestType = com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.class, - responseType = com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.class, - methodType = io.grpc.MethodDescriptor.MethodType.UNARY) - public static io.grpc.MethodDescriptor getListMaliciousRequestsMethod() { - io.grpc.MethodDescriptor getListMaliciousRequestsMethod; - if ((getListMaliciousRequestsMethod = DashboardServiceGrpc.getListMaliciousRequestsMethod) == null) { - synchronized (DashboardServiceGrpc.class) { - if ((getListMaliciousRequestsMethod = DashboardServiceGrpc.getListMaliciousRequestsMethod) == null) { - DashboardServiceGrpc.getListMaliciousRequestsMethod = getListMaliciousRequestsMethod = - io.grpc.MethodDescriptor.newBuilder() - .setType(io.grpc.MethodDescriptor.MethodType.UNARY) - .setFullMethodName(generateFullMethodName(SERVICE_NAME, "ListMaliciousRequests")) - .setSampledToLocalTracing(true) - .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.getDefaultInstance())) - .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.getDefaultInstance())) - .setSchemaDescriptor(new DashboardServiceMethodDescriptorSupplier("ListMaliciousRequests")) - .build(); - } - } - } - return getListMaliciousRequestsMethod; - } - - private static volatile io.grpc.MethodDescriptor getFetchAlertFiltersMethod; - - @io.grpc.stub.annotations.RpcMethod( - fullMethodName = SERVICE_NAME + '/' + "FetchAlertFilters", - requestType = com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.class, - responseType = com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.class, - methodType = io.grpc.MethodDescriptor.MethodType.UNARY) - public static io.grpc.MethodDescriptor getFetchAlertFiltersMethod() { - io.grpc.MethodDescriptor getFetchAlertFiltersMethod; - if ((getFetchAlertFiltersMethod = DashboardServiceGrpc.getFetchAlertFiltersMethod) == null) { - synchronized (DashboardServiceGrpc.class) { - if ((getFetchAlertFiltersMethod = DashboardServiceGrpc.getFetchAlertFiltersMethod) == null) { - DashboardServiceGrpc.getFetchAlertFiltersMethod = getFetchAlertFiltersMethod = - io.grpc.MethodDescriptor.newBuilder() - .setType(io.grpc.MethodDescriptor.MethodType.UNARY) - .setFullMethodName(generateFullMethodName(SERVICE_NAME, "FetchAlertFilters")) - .setSampledToLocalTracing(true) - .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.getDefaultInstance())) - .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.getDefaultInstance())) - .setSchemaDescriptor(new DashboardServiceMethodDescriptorSupplier("FetchAlertFilters")) - .build(); - } - } - } - return getFetchAlertFiltersMethod; - } - - /** - * Creates a new async stub that supports all call types for the service - */ - public static DashboardServiceStub newStub(io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public DashboardServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new DashboardServiceStub(channel, callOptions); - } - }; - return DashboardServiceStub.newStub(factory, channel); - } - - /** - * Creates a new blocking-style stub that supports unary and streaming output calls on the service - */ - public static DashboardServiceBlockingStub newBlockingStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public DashboardServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new DashboardServiceBlockingStub(channel, callOptions); - } - }; - return DashboardServiceBlockingStub.newStub(factory, channel); - } - - /** - * Creates a new ListenableFuture-style stub that supports unary calls on the service - */ - public static DashboardServiceFutureStub newFutureStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public DashboardServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new DashboardServiceFutureStub(channel, callOptions); - } - }; - return DashboardServiceFutureStub.newStub(factory, channel); - } - - /** - */ - public interface AsyncService { - - /** - */ - default void listMaliciousRequests(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getListMaliciousRequestsMethod(), responseObserver); - } - - /** - */ - default void fetchAlertFilters(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getFetchAlertFiltersMethod(), responseObserver); - } - } - - /** - * Base class for the server implementation of the service DashboardService. - */ - public static abstract class DashboardServiceImplBase - implements io.grpc.BindableService, AsyncService { - - @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { - return DashboardServiceGrpc.bindService(this); - } - } - - /** - * A stub to allow clients to do asynchronous rpc calls to service DashboardService. - */ - public static final class DashboardServiceStub - extends io.grpc.stub.AbstractAsyncStub { - private DashboardServiceStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected DashboardServiceStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new DashboardServiceStub(channel, callOptions); - } - - /** - */ - public void listMaliciousRequests(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ClientCalls.asyncUnaryCall( - getChannel().newCall(getListMaliciousRequestsMethod(), getCallOptions()), request, responseObserver); - } - - /** - */ - public void fetchAlertFilters(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ClientCalls.asyncUnaryCall( - getChannel().newCall(getFetchAlertFiltersMethod(), getCallOptions()), request, responseObserver); - } - } - - /** - * A stub to allow clients to do synchronous rpc calls to service DashboardService. - */ - public static final class DashboardServiceBlockingStub - extends io.grpc.stub.AbstractBlockingStub { - private DashboardServiceBlockingStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected DashboardServiceBlockingStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new DashboardServiceBlockingStub(channel, callOptions); - } - - /** - */ - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse listMaliciousRequests(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest request) { - return io.grpc.stub.ClientCalls.blockingUnaryCall( - getChannel(), getListMaliciousRequestsMethod(), getCallOptions(), request); - } - - /** - */ - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse fetchAlertFilters(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest request) { - return io.grpc.stub.ClientCalls.blockingUnaryCall( - getChannel(), getFetchAlertFiltersMethod(), getCallOptions(), request); - } - } - - /** - * A stub to allow clients to do ListenableFuture-style rpc calls to service DashboardService. - */ - public static final class DashboardServiceFutureStub - extends io.grpc.stub.AbstractFutureStub { - private DashboardServiceFutureStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected DashboardServiceFutureStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new DashboardServiceFutureStub(channel, callOptions); - } - - /** - */ - public com.google.common.util.concurrent.ListenableFuture listMaliciousRequests( - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest request) { - return io.grpc.stub.ClientCalls.futureUnaryCall( - getChannel().newCall(getListMaliciousRequestsMethod(), getCallOptions()), request); - } - - /** - */ - public com.google.common.util.concurrent.ListenableFuture fetchAlertFilters( - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest request) { - return io.grpc.stub.ClientCalls.futureUnaryCall( - getChannel().newCall(getFetchAlertFiltersMethod(), getCallOptions()), request); - } - } - - private static final int METHODID_LIST_MALICIOUS_REQUESTS = 0; - private static final int METHODID_FETCH_ALERT_FILTERS = 1; - - private static final class MethodHandlers implements - io.grpc.stub.ServerCalls.UnaryMethod, - io.grpc.stub.ServerCalls.ServerStreamingMethod, - io.grpc.stub.ServerCalls.ClientStreamingMethod, - io.grpc.stub.ServerCalls.BidiStreamingMethod { - private final AsyncService serviceImpl; - private final int methodId; - - MethodHandlers(AsyncService serviceImpl, int methodId) { - this.serviceImpl = serviceImpl; - this.methodId = methodId; - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - case METHODID_LIST_MALICIOUS_REQUESTS: - serviceImpl.listMaliciousRequests((com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) request, - (io.grpc.stub.StreamObserver) responseObserver); - break; - case METHODID_FETCH_ALERT_FILTERS: - serviceImpl.fetchAlertFilters((com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) request, - (io.grpc.stub.StreamObserver) responseObserver); - break; - default: - throw new AssertionError(); - } - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public io.grpc.stub.StreamObserver invoke( - io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - default: - throw new AssertionError(); - } - } - } - - public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { - return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) - .addMethod( - getListMaliciousRequestsMethod(), - io.grpc.stub.ServerCalls.asyncUnaryCall( - new MethodHandlers< - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest, - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse>( - service, METHODID_LIST_MALICIOUS_REQUESTS))) - .addMethod( - getFetchAlertFiltersMethod(), - io.grpc.stub.ServerCalls.asyncUnaryCall( - new MethodHandlers< - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest, - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse>( - service, METHODID_FETCH_ALERT_FILTERS))) - .build(); - } - - private static abstract class DashboardServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { - DashboardServiceBaseDescriptorSupplier() {} - - @java.lang.Override - public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.getDescriptor(); - } - - @java.lang.Override - public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { - return getFileDescriptor().findServiceByName("DashboardService"); - } - } - - private static final class DashboardServiceFileDescriptorSupplier - extends DashboardServiceBaseDescriptorSupplier { - DashboardServiceFileDescriptorSupplier() {} - } - - private static final class DashboardServiceMethodDescriptorSupplier - extends DashboardServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { - private final java.lang.String methodName; - - DashboardServiceMethodDescriptorSupplier(java.lang.String methodName) { - this.methodName = methodName; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { - return getServiceDescriptor().findMethodByName(methodName); - } - } - - private static volatile io.grpc.ServiceDescriptor serviceDescriptor; - - public static io.grpc.ServiceDescriptor getServiceDescriptor() { - io.grpc.ServiceDescriptor result = serviceDescriptor; - if (result == null) { - synchronized (DashboardServiceGrpc.class) { - result = serviceDescriptor; - if (result == null) { - serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) - .setSchemaDescriptor(new DashboardServiceFileDescriptorSupplier()) - .addMethod(getListMaliciousRequestsMethod()) - .addMethod(getFetchAlertFiltersMethod()) - .build(); - } - } - } - return result; - } -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequest.java deleted file mode 100644 index 13696e35ee..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequest.java +++ /dev/null @@ -1,358 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -/** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest} - */ -public final class FetchAlertFiltersRequest extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) - FetchAlertFiltersRequestOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - FetchAlertFiltersRequest.class.getName()); - } - // Use FetchAlertFiltersRequest.newBuilder() to construct. - private FetchAlertFiltersRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private FetchAlertFiltersRequest() { - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.Builder.class); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest other = (com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) obj; - - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest build() { - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest buildPartial() { - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest result = new com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest(this); - onBuilt(); - return result; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) { - return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest other) { - if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest.getDefaultInstance()) return this; - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) - private static final com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest(); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public FetchAlertFiltersRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequestOrBuilder.java deleted file mode 100644 index c408feed12..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersRequestOrBuilder.java +++ /dev/null @@ -1,11 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -public interface FetchAlertFiltersRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest) - com.google.protobuf.MessageOrBuilder { -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponse.java deleted file mode 100644 index ff75ad076b..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponse.java +++ /dev/null @@ -1,743 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -/** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse} - */ -public final class FetchAlertFiltersResponse extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) - FetchAlertFiltersResponseOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - FetchAlertFiltersResponse.class.getName()); - } - // Use FetchAlertFiltersResponse.newBuilder() to construct. - private FetchAlertFiltersResponse(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private FetchAlertFiltersResponse() { - actors_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - urls_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.class, com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.Builder.class); - } - - public static final int ACTORS_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private com.google.protobuf.LazyStringArrayList actors_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - /** - * repeated string actors = 1 [json_name = "actors"]; - * @return A list containing the actors. - */ - public com.google.protobuf.ProtocolStringList - getActorsList() { - return actors_; - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @return The count of actors. - */ - public int getActorsCount() { - return actors_.size(); - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param index The index of the element to return. - * @return The actors at the given index. - */ - public java.lang.String getActors(int index) { - return actors_.get(index); - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param index The index of the value to return. - * @return The bytes of the actors at the given index. - */ - public com.google.protobuf.ByteString - getActorsBytes(int index) { - return actors_.getByteString(index); - } - - public static final int URLS_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private com.google.protobuf.LazyStringArrayList urls_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - /** - * repeated string urls = 2 [json_name = "urls"]; - * @return A list containing the urls. - */ - public com.google.protobuf.ProtocolStringList - getUrlsList() { - return urls_; - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @return The count of urls. - */ - public int getUrlsCount() { - return urls_.size(); - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param index The index of the element to return. - * @return The urls at the given index. - */ - public java.lang.String getUrls(int index) { - return urls_.get(index); - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param index The index of the value to return. - * @return The bytes of the urls at the given index. - */ - public com.google.protobuf.ByteString - getUrlsBytes(int index) { - return urls_.getByteString(index); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - for (int i = 0; i < actors_.size(); i++) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, actors_.getRaw(i)); - } - for (int i = 0; i < urls_.size(); i++) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, urls_.getRaw(i)); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - { - int dataSize = 0; - for (int i = 0; i < actors_.size(); i++) { - dataSize += computeStringSizeNoTag(actors_.getRaw(i)); - } - size += dataSize; - size += 1 * getActorsList().size(); - } - { - int dataSize = 0; - for (int i = 0; i < urls_.size(); i++) { - dataSize += computeStringSizeNoTag(urls_.getRaw(i)); - } - size += dataSize; - size += 1 * getUrlsList().size(); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse other = (com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) obj; - - if (!getActorsList() - .equals(other.getActorsList())) return false; - if (!getUrlsList() - .equals(other.getUrlsList())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - if (getActorsCount() > 0) { - hash = (37 * hash) + ACTORS_FIELD_NUMBER; - hash = (53 * hash) + getActorsList().hashCode(); - } - if (getUrlsCount() > 0) { - hash = (37 * hash) + URLS_FIELD_NUMBER; - hash = (53 * hash) + getUrlsList().hashCode(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.class, com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - actors_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - urls_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse build() { - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse buildPartial() { - com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse result = new com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - actors_.makeImmutable(); - result.actors_ = actors_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - urls_.makeImmutable(); - result.urls_ = urls_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) { - return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse other) { - if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse.getDefaultInstance()) return this; - if (!other.actors_.isEmpty()) { - if (actors_.isEmpty()) { - actors_ = other.actors_; - bitField0_ |= 0x00000001; - } else { - ensureActorsIsMutable(); - actors_.addAll(other.actors_); - } - onChanged(); - } - if (!other.urls_.isEmpty()) { - if (urls_.isEmpty()) { - urls_ = other.urls_; - bitField0_ |= 0x00000002; - } else { - ensureUrlsIsMutable(); - urls_.addAll(other.urls_); - } - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - java.lang.String s = input.readStringRequireUtf8(); - ensureActorsIsMutable(); - actors_.add(s); - break; - } // case 10 - case 18: { - java.lang.String s = input.readStringRequireUtf8(); - ensureUrlsIsMutable(); - urls_.add(s); - break; - } // case 18 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private com.google.protobuf.LazyStringArrayList actors_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - private void ensureActorsIsMutable() { - if (!actors_.isModifiable()) { - actors_ = new com.google.protobuf.LazyStringArrayList(actors_); - } - bitField0_ |= 0x00000001; - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @return A list containing the actors. - */ - public com.google.protobuf.ProtocolStringList - getActorsList() { - actors_.makeImmutable(); - return actors_; - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @return The count of actors. - */ - public int getActorsCount() { - return actors_.size(); - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param index The index of the element to return. - * @return The actors at the given index. - */ - public java.lang.String getActors(int index) { - return actors_.get(index); - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param index The index of the value to return. - * @return The bytes of the actors at the given index. - */ - public com.google.protobuf.ByteString - getActorsBytes(int index) { - return actors_.getByteString(index); - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param index The index to set the value at. - * @param value The actors to set. - * @return This builder for chaining. - */ - public Builder setActors( - int index, java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ensureActorsIsMutable(); - actors_.set(index, value); - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param value The actors to add. - * @return This builder for chaining. - */ - public Builder addActors( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ensureActorsIsMutable(); - actors_.add(value); - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param values The actors to add. - * @return This builder for chaining. - */ - public Builder addAllActors( - java.lang.Iterable values) { - ensureActorsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, actors_); - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @return This builder for chaining. - */ - public Builder clearActors() { - actors_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001);; - onChanged(); - return this; - } - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param value The bytes of the actors to add. - * @return This builder for chaining. - */ - public Builder addActorsBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ensureActorsIsMutable(); - actors_.add(value); - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private com.google.protobuf.LazyStringArrayList urls_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - private void ensureUrlsIsMutable() { - if (!urls_.isModifiable()) { - urls_ = new com.google.protobuf.LazyStringArrayList(urls_); - } - bitField0_ |= 0x00000002; - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @return A list containing the urls. - */ - public com.google.protobuf.ProtocolStringList - getUrlsList() { - urls_.makeImmutable(); - return urls_; - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @return The count of urls. - */ - public int getUrlsCount() { - return urls_.size(); - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param index The index of the element to return. - * @return The urls at the given index. - */ - public java.lang.String getUrls(int index) { - return urls_.get(index); - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param index The index of the value to return. - * @return The bytes of the urls at the given index. - */ - public com.google.protobuf.ByteString - getUrlsBytes(int index) { - return urls_.getByteString(index); - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param index The index to set the value at. - * @param value The urls to set. - * @return This builder for chaining. - */ - public Builder setUrls( - int index, java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ensureUrlsIsMutable(); - urls_.set(index, value); - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param value The urls to add. - * @return This builder for chaining. - */ - public Builder addUrls( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ensureUrlsIsMutable(); - urls_.add(value); - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param values The urls to add. - * @return This builder for chaining. - */ - public Builder addAllUrls( - java.lang.Iterable values) { - ensureUrlsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, urls_); - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @return This builder for chaining. - */ - public Builder clearUrls() { - urls_ = - com.google.protobuf.LazyStringArrayList.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002);; - onChanged(); - return this; - } - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param value The bytes of the urls to add. - * @return This builder for chaining. - */ - public Builder addUrlsBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ensureUrlsIsMutable(); - urls_.add(value); - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) - private static final com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse(); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public FetchAlertFiltersResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponseOrBuilder.java deleted file mode 100644 index 9d9cc50f6a..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/FetchAlertFiltersResponseOrBuilder.java +++ /dev/null @@ -1,61 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -public interface FetchAlertFiltersResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse) - com.google.protobuf.MessageOrBuilder { - - /** - * repeated string actors = 1 [json_name = "actors"]; - * @return A list containing the actors. - */ - java.util.List - getActorsList(); - /** - * repeated string actors = 1 [json_name = "actors"]; - * @return The count of actors. - */ - int getActorsCount(); - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param index The index of the element to return. - * @return The actors at the given index. - */ - java.lang.String getActors(int index); - /** - * repeated string actors = 1 [json_name = "actors"]; - * @param index The index of the value to return. - * @return The bytes of the actors at the given index. - */ - com.google.protobuf.ByteString - getActorsBytes(int index); - - /** - * repeated string urls = 2 [json_name = "urls"]; - * @return A list containing the urls. - */ - java.util.List - getUrlsList(); - /** - * repeated string urls = 2 [json_name = "urls"]; - * @return The count of urls. - */ - int getUrlsCount(); - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param index The index of the element to return. - * @return The urls at the given index. - */ - java.lang.String getUrls(int index); - /** - * repeated string urls = 2 [json_name = "urls"]; - * @param index The index of the value to return. - * @return The bytes of the urls at the given index. - */ - com.google.protobuf.ByteString - getUrlsBytes(int index); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequest.java deleted file mode 100644 index dfbacb88cd..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequest.java +++ /dev/null @@ -1,538 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -/** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest} - */ -public final class ListMaliciousRequestsRequest extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) - ListMaliciousRequestsRequestOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - ListMaliciousRequestsRequest.class.getName()); - } - // Use ListMaliciousRequestsRequest.newBuilder() to construct. - private ListMaliciousRequestsRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private ListMaliciousRequestsRequest() { - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.Builder.class); - } - - private int bitField0_; - public static final int LIMIT_FIELD_NUMBER = 3; - private int limit_ = 0; - /** - *
-   * The number of alerts to return
-   * 
- * - * int32 limit = 3 [json_name = "limit"]; - * @return The limit. - */ - @java.lang.Override - public int getLimit() { - return limit_; - } - - public static final int PAGE_FIELD_NUMBER = 4; - private int page_ = 0; - /** - * optional int32 page = 4 [json_name = "page"]; - * @return Whether the page field is set. - */ - @java.lang.Override - public boolean hasPage() { - return ((bitField0_ & 0x00000001) != 0); - } - /** - * optional int32 page = 4 [json_name = "page"]; - * @return The page. - */ - @java.lang.Override - public int getPage() { - return page_; - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (limit_ != 0) { - output.writeInt32(3, limit_); - } - if (((bitField0_ & 0x00000001) != 0)) { - output.writeInt32(4, page_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (limit_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(3, limit_); - } - if (((bitField0_ & 0x00000001) != 0)) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(4, page_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest other = (com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) obj; - - if (getLimit() - != other.getLimit()) return false; - if (hasPage() != other.hasPage()) return false; - if (hasPage()) { - if (getPage() - != other.getPage()) return false; - } - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + LIMIT_FIELD_NUMBER; - hash = (53 * hash) + getLimit(); - if (hasPage()) { - hash = (37 * hash) + PAGE_FIELD_NUMBER; - hash = (53 * hash) + getPage(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - limit_ = 0; - page_ = 0; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest build() { - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest buildPartial() { - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest result = new com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.limit_ = limit_; - } - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000002) != 0)) { - result.page_ = page_; - to_bitField0_ |= 0x00000001; - } - result.bitField0_ |= to_bitField0_; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) { - return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest other) { - if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest.getDefaultInstance()) return this; - if (other.getLimit() != 0) { - setLimit(other.getLimit()); - } - if (other.hasPage()) { - setPage(other.getPage()); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 24: { - limit_ = input.readInt32(); - bitField0_ |= 0x00000001; - break; - } // case 24 - case 32: { - page_ = input.readInt32(); - bitField0_ |= 0x00000002; - break; - } // case 32 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private int limit_ ; - /** - *
-     * The number of alerts to return
-     * 
- * - * int32 limit = 3 [json_name = "limit"]; - * @return The limit. - */ - @java.lang.Override - public int getLimit() { - return limit_; - } - /** - *
-     * The number of alerts to return
-     * 
- * - * int32 limit = 3 [json_name = "limit"]; - * @param value The limit to set. - * @return This builder for chaining. - */ - public Builder setLimit(int value) { - - limit_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - *
-     * The number of alerts to return
-     * 
- * - * int32 limit = 3 [json_name = "limit"]; - * @return This builder for chaining. - */ - public Builder clearLimit() { - bitField0_ = (bitField0_ & ~0x00000001); - limit_ = 0; - onChanged(); - return this; - } - - private int page_ ; - /** - * optional int32 page = 4 [json_name = "page"]; - * @return Whether the page field is set. - */ - @java.lang.Override - public boolean hasPage() { - return ((bitField0_ & 0x00000002) != 0); - } - /** - * optional int32 page = 4 [json_name = "page"]; - * @return The page. - */ - @java.lang.Override - public int getPage() { - return page_; - } - /** - * optional int32 page = 4 [json_name = "page"]; - * @param value The page to set. - * @return This builder for chaining. - */ - public Builder setPage(int value) { - - page_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * optional int32 page = 4 [json_name = "page"]; - * @return This builder for chaining. - */ - public Builder clearPage() { - bitField0_ = (bitField0_ & ~0x00000002); - page_ = 0; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) - private static final com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest(); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ListMaliciousRequestsRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequestOrBuilder.java deleted file mode 100644 index 654dd72745..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsRequestOrBuilder.java +++ /dev/null @@ -1,32 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -public interface ListMaliciousRequestsRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest) - com.google.protobuf.MessageOrBuilder { - - /** - *
-   * The number of alerts to return
-   * 
- * - * int32 limit = 3 [json_name = "limit"]; - * @return The limit. - */ - int getLimit(); - - /** - * optional int32 page = 4 [json_name = "page"]; - * @return Whether the page field is set. - */ - boolean hasPage(); - /** - * optional int32 page = 4 [json_name = "page"]; - * @return The page. - */ - int getPage(); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponse.java deleted file mode 100644 index c7e6041b01..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponse.java +++ /dev/null @@ -1,851 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -/** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse} - */ -public final class ListMaliciousRequestsResponse extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) - ListMaliciousRequestsResponseOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - ListMaliciousRequestsResponse.class.getName()); - } - // Use ListMaliciousRequestsResponse.newBuilder() to construct. - private ListMaliciousRequestsResponse(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private ListMaliciousRequestsResponse() { - maliciousRequests_ = java.util.Collections.emptyList(); - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.class, com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.Builder.class); - } - - public static final int MALICIOUS_REQUESTS_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private java.util.List maliciousRequests_; - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - @java.lang.Override - public java.util.List getMaliciousRequestsList() { - return maliciousRequests_; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - @java.lang.Override - public java.util.List - getMaliciousRequestsOrBuilderList() { - return maliciousRequests_; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - @java.lang.Override - public int getMaliciousRequestsCount() { - return maliciousRequests_.size(); - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getMaliciousRequests(int index) { - return maliciousRequests_.get(index); - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder getMaliciousRequestsOrBuilder( - int index) { - return maliciousRequests_.get(index); - } - - public static final int TOTAL_FIELD_NUMBER = 2; - private int total_ = 0; - /** - * int32 total = 2 [json_name = "total"]; - * @return The total. - */ - @java.lang.Override - public int getTotal() { - return total_; - } - - public static final int PAGE_FIELD_NUMBER = 3; - private int page_ = 0; - /** - * int32 page = 3 [json_name = "page"]; - * @return The page. - */ - @java.lang.Override - public int getPage() { - return page_; - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - for (int i = 0; i < maliciousRequests_.size(); i++) { - output.writeMessage(1, maliciousRequests_.get(i)); - } - if (total_ != 0) { - output.writeInt32(2, total_); - } - if (page_ != 0) { - output.writeInt32(3, page_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - for (int i = 0; i < maliciousRequests_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, maliciousRequests_.get(i)); - } - if (total_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(2, total_); - } - if (page_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(3, page_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse other = (com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) obj; - - if (!getMaliciousRequestsList() - .equals(other.getMaliciousRequestsList())) return false; - if (getTotal() - != other.getTotal()) return false; - if (getPage() - != other.getPage()) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - if (getMaliciousRequestsCount() > 0) { - hash = (37 * hash) + MALICIOUS_REQUESTS_FIELD_NUMBER; - hash = (53 * hash) + getMaliciousRequestsList().hashCode(); - } - hash = (37 * hash) + TOTAL_FIELD_NUMBER; - hash = (53 * hash) + getTotal(); - hash = (37 * hash) + PAGE_FIELD_NUMBER; - hash = (53 * hash) + getPage(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.class, com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - if (maliciousRequestsBuilder_ == null) { - maliciousRequests_ = java.util.Collections.emptyList(); - } else { - maliciousRequests_ = null; - maliciousRequestsBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000001); - total_ = 0; - page_ = 0; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse build() { - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse buildPartial() { - com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse result = new com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse(this); - buildPartialRepeatedFields(result); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartialRepeatedFields(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse result) { - if (maliciousRequestsBuilder_ == null) { - if (((bitField0_ & 0x00000001) != 0)) { - maliciousRequests_ = java.util.Collections.unmodifiableList(maliciousRequests_); - bitField0_ = (bitField0_ & ~0x00000001); - } - result.maliciousRequests_ = maliciousRequests_; - } else { - result.maliciousRequests_ = maliciousRequestsBuilder_.build(); - } - } - - private void buildPartial0(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000002) != 0)) { - result.total_ = total_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.page_ = page_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) { - return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse other) { - if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse.getDefaultInstance()) return this; - if (maliciousRequestsBuilder_ == null) { - if (!other.maliciousRequests_.isEmpty()) { - if (maliciousRequests_.isEmpty()) { - maliciousRequests_ = other.maliciousRequests_; - bitField0_ = (bitField0_ & ~0x00000001); - } else { - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.addAll(other.maliciousRequests_); - } - onChanged(); - } - } else { - if (!other.maliciousRequests_.isEmpty()) { - if (maliciousRequestsBuilder_.isEmpty()) { - maliciousRequestsBuilder_.dispose(); - maliciousRequestsBuilder_ = null; - maliciousRequests_ = other.maliciousRequests_; - bitField0_ = (bitField0_ & ~0x00000001); - maliciousRequestsBuilder_ = - com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getMaliciousRequestsFieldBuilder() : null; - } else { - maliciousRequestsBuilder_.addAllMessages(other.maliciousRequests_); - } - } - } - if (other.getTotal() != 0) { - setTotal(other.getTotal()); - } - if (other.getPage() != 0) { - setPage(other.getPage()); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest m = - input.readMessage( - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.parser(), - extensionRegistry); - if (maliciousRequestsBuilder_ == null) { - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.add(m); - } else { - maliciousRequestsBuilder_.addMessage(m); - } - break; - } // case 10 - case 16: { - total_ = input.readInt32(); - bitField0_ |= 0x00000002; - break; - } // case 16 - case 24: { - page_ = input.readInt32(); - bitField0_ |= 0x00000004; - break; - } // case 24 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.util.List maliciousRequests_ = - java.util.Collections.emptyList(); - private void ensureMaliciousRequestsIsMutable() { - if (!((bitField0_ & 0x00000001) != 0)) { - maliciousRequests_ = new java.util.ArrayList(maliciousRequests_); - bitField0_ |= 0x00000001; - } - } - - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder> maliciousRequestsBuilder_; - - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public java.util.List getMaliciousRequestsList() { - if (maliciousRequestsBuilder_ == null) { - return java.util.Collections.unmodifiableList(maliciousRequests_); - } else { - return maliciousRequestsBuilder_.getMessageList(); - } - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public int getMaliciousRequestsCount() { - if (maliciousRequestsBuilder_ == null) { - return maliciousRequests_.size(); - } else { - return maliciousRequestsBuilder_.getCount(); - } - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getMaliciousRequests(int index) { - if (maliciousRequestsBuilder_ == null) { - return maliciousRequests_.get(index); - } else { - return maliciousRequestsBuilder_.getMessage(index); - } - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder setMaliciousRequests( - int index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest value) { - if (maliciousRequestsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.set(index, value); - onChanged(); - } else { - maliciousRequestsBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder setMaliciousRequests( - int index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder builderForValue) { - if (maliciousRequestsBuilder_ == null) { - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.set(index, builderForValue.build()); - onChanged(); - } else { - maliciousRequestsBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder addMaliciousRequests(com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest value) { - if (maliciousRequestsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.add(value); - onChanged(); - } else { - maliciousRequestsBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder addMaliciousRequests( - int index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest value) { - if (maliciousRequestsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.add(index, value); - onChanged(); - } else { - maliciousRequestsBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder addMaliciousRequests( - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder builderForValue) { - if (maliciousRequestsBuilder_ == null) { - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.add(builderForValue.build()); - onChanged(); - } else { - maliciousRequestsBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder addMaliciousRequests( - int index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder builderForValue) { - if (maliciousRequestsBuilder_ == null) { - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.add(index, builderForValue.build()); - onChanged(); - } else { - maliciousRequestsBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder addAllMaliciousRequests( - java.lang.Iterable values) { - if (maliciousRequestsBuilder_ == null) { - ensureMaliciousRequestsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, maliciousRequests_); - onChanged(); - } else { - maliciousRequestsBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder clearMaliciousRequests() { - if (maliciousRequestsBuilder_ == null) { - maliciousRequests_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - } else { - maliciousRequestsBuilder_.clear(); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public Builder removeMaliciousRequests(int index) { - if (maliciousRequestsBuilder_ == null) { - ensureMaliciousRequestsIsMutable(); - maliciousRequests_.remove(index); - onChanged(); - } else { - maliciousRequestsBuilder_.remove(index); - } - return this; - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder getMaliciousRequestsBuilder( - int index) { - return getMaliciousRequestsFieldBuilder().getBuilder(index); - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder getMaliciousRequestsOrBuilder( - int index) { - if (maliciousRequestsBuilder_ == null) { - return maliciousRequests_.get(index); } else { - return maliciousRequestsBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public java.util.List - getMaliciousRequestsOrBuilderList() { - if (maliciousRequestsBuilder_ != null) { - return maliciousRequestsBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(maliciousRequests_); - } - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder addMaliciousRequestsBuilder() { - return getMaliciousRequestsFieldBuilder().addBuilder( - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.getDefaultInstance()); - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder addMaliciousRequestsBuilder( - int index) { - return getMaliciousRequestsFieldBuilder().addBuilder( - index, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.getDefaultInstance()); - } - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - public java.util.List - getMaliciousRequestsBuilderList() { - return getMaliciousRequestsFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder> - getMaliciousRequestsFieldBuilder() { - if (maliciousRequestsBuilder_ == null) { - maliciousRequestsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder>( - maliciousRequests_, - ((bitField0_ & 0x00000001) != 0), - getParentForChildren(), - isClean()); - maliciousRequests_ = null; - } - return maliciousRequestsBuilder_; - } - - private int total_ ; - /** - * int32 total = 2 [json_name = "total"]; - * @return The total. - */ - @java.lang.Override - public int getTotal() { - return total_; - } - /** - * int32 total = 2 [json_name = "total"]; - * @param value The total to set. - * @return This builder for chaining. - */ - public Builder setTotal(int value) { - - total_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * int32 total = 2 [json_name = "total"]; - * @return This builder for chaining. - */ - public Builder clearTotal() { - bitField0_ = (bitField0_ & ~0x00000002); - total_ = 0; - onChanged(); - return this; - } - - private int page_ ; - /** - * int32 page = 3 [json_name = "page"]; - * @return The page. - */ - @java.lang.Override - public int getPage() { - return page_; - } - /** - * int32 page = 3 [json_name = "page"]; - * @param value The page to set. - * @return This builder for chaining. - */ - public Builder setPage(int value) { - - page_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * int32 page = 3 [json_name = "page"]; - * @return This builder for chaining. - */ - public Builder clearPage() { - bitField0_ = (bitField0_ & ~0x00000004); - page_ = 0; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) - private static final com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse(); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ListMaliciousRequestsResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponseOrBuilder.java deleted file mode 100644 index e020064c48..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ListMaliciousRequestsResponseOrBuilder.java +++ /dev/null @@ -1,47 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -public interface ListMaliciousRequestsResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse) - com.google.protobuf.MessageOrBuilder { - - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - java.util.List - getMaliciousRequestsList(); - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getMaliciousRequests(int index); - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - int getMaliciousRequestsCount(); - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - java.util.List - getMaliciousRequestsOrBuilderList(); - /** - * repeated .threat_protection.service.dashboard_service.v1.MaliciousRequest malicious_requests = 1 [json_name = "maliciousRequests"]; - */ - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder getMaliciousRequestsOrBuilder( - int index); - - /** - * int32 total = 2 [json_name = "total"]; - * @return The total. - */ - int getTotal(); - - /** - * int32 page = 3 [json_name = "page"]; - * @return The page. - */ - int getPage(); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequest.java deleted file mode 100644 index fdcc670c24..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequest.java +++ /dev/null @@ -1,1520 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -/** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.MaliciousRequest} - */ -public final class MaliciousRequest extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.dashboard_service.v1.MaliciousRequest) - MaliciousRequestOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - MaliciousRequest.class.getName()); - } - // Use MaliciousRequest.newBuilder() to construct. - private MaliciousRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private MaliciousRequest() { - id_ = ""; - actor_ = ""; - filterId_ = ""; - url_ = ""; - method_ = ""; - orig_ = ""; - ip_ = ""; - country_ = ""; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder.class); - } - - public static final int ID_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object id_ = ""; - /** - * string id = 1 [json_name = "id"]; - * @return The id. - */ - @java.lang.Override - public java.lang.String getId() { - java.lang.Object ref = id_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - id_ = s; - return s; - } - } - /** - * string id = 1 [json_name = "id"]; - * @return The bytes for id. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getIdBytes() { - java.lang.Object ref = id_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - id_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int ACTOR_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private volatile java.lang.Object actor_ = ""; - /** - * string actor = 2 [json_name = "actor"]; - * @return The actor. - */ - @java.lang.Override - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } - } - /** - * string actor = 2 [json_name = "actor"]; - * @return The bytes for actor. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 3; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 3 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 3 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int URL_FIELD_NUMBER = 4; - @SuppressWarnings("serial") - private volatile java.lang.Object url_ = ""; - /** - * string url = 4 [json_name = "url"]; - * @return The url. - */ - @java.lang.Override - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } - } - /** - * string url = 4 [json_name = "url"]; - * @return The bytes for url. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int METHOD_FIELD_NUMBER = 5; - @SuppressWarnings("serial") - private volatile java.lang.Object method_ = ""; - /** - * string method = 5 [json_name = "method"]; - * @return The method. - */ - @java.lang.Override - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } - } - /** - * string method = 5 [json_name = "method"]; - * @return The bytes for method. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int ORIG_FIELD_NUMBER = 6; - @SuppressWarnings("serial") - private volatile java.lang.Object orig_ = ""; - /** - * string orig = 6 [json_name = "orig"]; - * @return The orig. - */ - @java.lang.Override - public java.lang.String getOrig() { - java.lang.Object ref = orig_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - orig_ = s; - return s; - } - } - /** - * string orig = 6 [json_name = "orig"]; - * @return The bytes for orig. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getOrigBytes() { - java.lang.Object ref = orig_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - orig_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int IP_FIELD_NUMBER = 7; - @SuppressWarnings("serial") - private volatile java.lang.Object ip_ = ""; - /** - * string ip = 7 [json_name = "ip"]; - * @return The ip. - */ - @java.lang.Override - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } - } - /** - * string ip = 7 [json_name = "ip"]; - * @return The bytes for ip. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int COUNTRY_FIELD_NUMBER = 8; - @SuppressWarnings("serial") - private volatile java.lang.Object country_ = ""; - /** - * string country = 8 [json_name = "country"]; - * @return The country. - */ - @java.lang.Override - public java.lang.String getCountry() { - java.lang.Object ref = country_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - country_ = s; - return s; - } - } - /** - * string country = 8 [json_name = "country"]; - * @return The bytes for country. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getCountryBytes() { - java.lang.Object ref = country_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - country_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TIMESTAMP_FIELD_NUMBER = 9; - private long timestamp_ = 0L; - /** - * int64 timestamp = 9 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(id_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, id_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 3, filterId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 4, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 5, method_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(orig_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 6, orig_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 7, ip_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(country_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 8, country_); - } - if (timestamp_ != 0L) { - output.writeInt64(9, timestamp_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(id_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, id_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(2, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(3, filterId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(4, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(5, method_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(orig_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(6, orig_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(7, ip_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(country_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(8, country_); - } - if (timestamp_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(9, timestamp_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest other = (com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest) obj; - - if (!getId() - .equals(other.getId())) return false; - if (!getActor() - .equals(other.getActor())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (!getUrl() - .equals(other.getUrl())) return false; - if (!getMethod() - .equals(other.getMethod())) return false; - if (!getOrig() - .equals(other.getOrig())) return false; - if (!getIp() - .equals(other.getIp())) return false; - if (!getCountry() - .equals(other.getCountry())) return false; - if (getTimestamp() - != other.getTimestamp()) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ID_FIELD_NUMBER; - hash = (53 * hash) + getId().hashCode(); - hash = (37 * hash) + ACTOR_FIELD_NUMBER; - hash = (53 * hash) + getActor().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (37 * hash) + URL_FIELD_NUMBER; - hash = (53 * hash) + getUrl().hashCode(); - hash = (37 * hash) + METHOD_FIELD_NUMBER; - hash = (53 * hash) + getMethod().hashCode(); - hash = (37 * hash) + ORIG_FIELD_NUMBER; - hash = (53 * hash) + getOrig().hashCode(); - hash = (37 * hash) + IP_FIELD_NUMBER; - hash = (53 * hash) + getIp().hashCode(); - hash = (37 * hash) + COUNTRY_FIELD_NUMBER; - hash = (53 * hash) + getCountry().hashCode(); - hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getTimestamp()); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.dashboard_service.v1.MaliciousRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.dashboard_service.v1.MaliciousRequest) - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.class, com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - id_ = ""; - actor_ = ""; - filterId_ = ""; - url_ = ""; - method_ = ""; - orig_ = ""; - ip_ = ""; - country_ = ""; - timestamp_ = 0L; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.ServiceProto.internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest build() { - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest buildPartial() { - com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest result = new com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.id_ = id_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.actor_ = actor_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.filterId_ = filterId_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.url_ = url_; - } - if (((from_bitField0_ & 0x00000010) != 0)) { - result.method_ = method_; - } - if (((from_bitField0_ & 0x00000020) != 0)) { - result.orig_ = orig_; - } - if (((from_bitField0_ & 0x00000040) != 0)) { - result.ip_ = ip_; - } - if (((from_bitField0_ & 0x00000080) != 0)) { - result.country_ = country_; - } - if (((from_bitField0_ & 0x00000100) != 0)) { - result.timestamp_ = timestamp_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest) { - return mergeFrom((com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest other) { - if (other == com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest.getDefaultInstance()) return this; - if (!other.getId().isEmpty()) { - id_ = other.id_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (!other.getActor().isEmpty()) { - actor_ = other.actor_; - bitField0_ |= 0x00000002; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000004; - onChanged(); - } - if (!other.getUrl().isEmpty()) { - url_ = other.url_; - bitField0_ |= 0x00000008; - onChanged(); - } - if (!other.getMethod().isEmpty()) { - method_ = other.method_; - bitField0_ |= 0x00000010; - onChanged(); - } - if (!other.getOrig().isEmpty()) { - orig_ = other.orig_; - bitField0_ |= 0x00000020; - onChanged(); - } - if (!other.getIp().isEmpty()) { - ip_ = other.ip_; - bitField0_ |= 0x00000040; - onChanged(); - } - if (!other.getCountry().isEmpty()) { - country_ = other.country_; - bitField0_ |= 0x00000080; - onChanged(); - } - if (other.getTimestamp() != 0L) { - setTimestamp(other.getTimestamp()); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - id_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - actor_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000002; - break; - } // case 18 - case 26: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000004; - break; - } // case 26 - case 34: { - url_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000008; - break; - } // case 34 - case 42: { - method_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000010; - break; - } // case 42 - case 50: { - orig_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000020; - break; - } // case 50 - case 58: { - ip_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000040; - break; - } // case 58 - case 66: { - country_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000080; - break; - } // case 66 - case 72: { - timestamp_ = input.readInt64(); - bitField0_ |= 0x00000100; - break; - } // case 72 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object id_ = ""; - /** - * string id = 1 [json_name = "id"]; - * @return The id. - */ - public java.lang.String getId() { - java.lang.Object ref = id_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - id_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string id = 1 [json_name = "id"]; - * @return The bytes for id. - */ - public com.google.protobuf.ByteString - getIdBytes() { - java.lang.Object ref = id_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - id_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string id = 1 [json_name = "id"]; - * @param value The id to set. - * @return This builder for chaining. - */ - public Builder setId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - id_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string id = 1 [json_name = "id"]; - * @return This builder for chaining. - */ - public Builder clearId() { - id_ = getDefaultInstance().getId(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string id = 1 [json_name = "id"]; - * @param value The bytes for id to set. - * @return This builder for chaining. - */ - public Builder setIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - id_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private java.lang.Object actor_ = ""; - /** - * string actor = 2 [json_name = "actor"]; - * @return The actor. - */ - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string actor = 2 [json_name = "actor"]; - * @return The bytes for actor. - */ - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string actor = 2 [json_name = "actor"]; - * @param value The actor to set. - * @return This builder for chaining. - */ - public Builder setActor( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - actor_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * string actor = 2 [json_name = "actor"]; - * @return This builder for chaining. - */ - public Builder clearActor() { - actor_ = getDefaultInstance().getActor(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - /** - * string actor = 2 [json_name = "actor"]; - * @param value The bytes for actor to set. - * @return This builder for chaining. - */ - public Builder setActorBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - actor_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 3 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 3 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 3 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * string filter_id = 3 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000004); - onChanged(); - return this; - } - /** - * string filter_id = 3 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - - private java.lang.Object url_ = ""; - /** - * string url = 4 [json_name = "url"]; - * @return The url. - */ - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string url = 4 [json_name = "url"]; - * @return The bytes for url. - */ - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string url = 4 [json_name = "url"]; - * @param value The url to set. - * @return This builder for chaining. - */ - public Builder setUrl( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - url_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * string url = 4 [json_name = "url"]; - * @return This builder for chaining. - */ - public Builder clearUrl() { - url_ = getDefaultInstance().getUrl(); - bitField0_ = (bitField0_ & ~0x00000008); - onChanged(); - return this; - } - /** - * string url = 4 [json_name = "url"]; - * @param value The bytes for url to set. - * @return This builder for chaining. - */ - public Builder setUrlBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - url_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - - private java.lang.Object method_ = ""; - /** - * string method = 5 [json_name = "method"]; - * @return The method. - */ - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string method = 5 [json_name = "method"]; - * @return The bytes for method. - */ - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string method = 5 [json_name = "method"]; - * @param value The method to set. - * @return This builder for chaining. - */ - public Builder setMethod( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - method_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - /** - * string method = 5 [json_name = "method"]; - * @return This builder for chaining. - */ - public Builder clearMethod() { - method_ = getDefaultInstance().getMethod(); - bitField0_ = (bitField0_ & ~0x00000010); - onChanged(); - return this; - } - /** - * string method = 5 [json_name = "method"]; - * @param value The bytes for method to set. - * @return This builder for chaining. - */ - public Builder setMethodBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - method_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - - private java.lang.Object orig_ = ""; - /** - * string orig = 6 [json_name = "orig"]; - * @return The orig. - */ - public java.lang.String getOrig() { - java.lang.Object ref = orig_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - orig_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string orig = 6 [json_name = "orig"]; - * @return The bytes for orig. - */ - public com.google.protobuf.ByteString - getOrigBytes() { - java.lang.Object ref = orig_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - orig_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string orig = 6 [json_name = "orig"]; - * @param value The orig to set. - * @return This builder for chaining. - */ - public Builder setOrig( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - orig_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - /** - * string orig = 6 [json_name = "orig"]; - * @return This builder for chaining. - */ - public Builder clearOrig() { - orig_ = getDefaultInstance().getOrig(); - bitField0_ = (bitField0_ & ~0x00000020); - onChanged(); - return this; - } - /** - * string orig = 6 [json_name = "orig"]; - * @param value The bytes for orig to set. - * @return This builder for chaining. - */ - public Builder setOrigBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - orig_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - - private java.lang.Object ip_ = ""; - /** - * string ip = 7 [json_name = "ip"]; - * @return The ip. - */ - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string ip = 7 [json_name = "ip"]; - * @return The bytes for ip. - */ - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string ip = 7 [json_name = "ip"]; - * @param value The ip to set. - * @return This builder for chaining. - */ - public Builder setIp( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ip_ = value; - bitField0_ |= 0x00000040; - onChanged(); - return this; - } - /** - * string ip = 7 [json_name = "ip"]; - * @return This builder for chaining. - */ - public Builder clearIp() { - ip_ = getDefaultInstance().getIp(); - bitField0_ = (bitField0_ & ~0x00000040); - onChanged(); - return this; - } - /** - * string ip = 7 [json_name = "ip"]; - * @param value The bytes for ip to set. - * @return This builder for chaining. - */ - public Builder setIpBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ip_ = value; - bitField0_ |= 0x00000040; - onChanged(); - return this; - } - - private java.lang.Object country_ = ""; - /** - * string country = 8 [json_name = "country"]; - * @return The country. - */ - public java.lang.String getCountry() { - java.lang.Object ref = country_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - country_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string country = 8 [json_name = "country"]; - * @return The bytes for country. - */ - public com.google.protobuf.ByteString - getCountryBytes() { - java.lang.Object ref = country_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - country_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string country = 8 [json_name = "country"]; - * @param value The country to set. - * @return This builder for chaining. - */ - public Builder setCountry( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - country_ = value; - bitField0_ |= 0x00000080; - onChanged(); - return this; - } - /** - * string country = 8 [json_name = "country"]; - * @return This builder for chaining. - */ - public Builder clearCountry() { - country_ = getDefaultInstance().getCountry(); - bitField0_ = (bitField0_ & ~0x00000080); - onChanged(); - return this; - } - /** - * string country = 8 [json_name = "country"]; - * @param value The bytes for country to set. - * @return This builder for chaining. - */ - public Builder setCountryBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - country_ = value; - bitField0_ |= 0x00000080; - onChanged(); - return this; - } - - private long timestamp_ ; - /** - * int64 timestamp = 9 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - /** - * int64 timestamp = 9 [json_name = "timestamp"]; - * @param value The timestamp to set. - * @return This builder for chaining. - */ - public Builder setTimestamp(long value) { - - timestamp_ = value; - bitField0_ |= 0x00000100; - onChanged(); - return this; - } - /** - * int64 timestamp = 9 [json_name = "timestamp"]; - * @return This builder for chaining. - */ - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000100); - timestamp_ = 0L; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.dashboard_service.v1.MaliciousRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.dashboard_service.v1.MaliciousRequest) - private static final com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest(); - } - - public static com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public MaliciousRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequestOrBuilder.java deleted file mode 100644 index 943f4f2e0e..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/MaliciousRequestOrBuilder.java +++ /dev/null @@ -1,113 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -public interface MaliciousRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.dashboard_service.v1.MaliciousRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * string id = 1 [json_name = "id"]; - * @return The id. - */ - java.lang.String getId(); - /** - * string id = 1 [json_name = "id"]; - * @return The bytes for id. - */ - com.google.protobuf.ByteString - getIdBytes(); - - /** - * string actor = 2 [json_name = "actor"]; - * @return The actor. - */ - java.lang.String getActor(); - /** - * string actor = 2 [json_name = "actor"]; - * @return The bytes for actor. - */ - com.google.protobuf.ByteString - getActorBytes(); - - /** - * string filter_id = 3 [json_name = "filterId"]; - * @return The filterId. - */ - java.lang.String getFilterId(); - /** - * string filter_id = 3 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - com.google.protobuf.ByteString - getFilterIdBytes(); - - /** - * string url = 4 [json_name = "url"]; - * @return The url. - */ - java.lang.String getUrl(); - /** - * string url = 4 [json_name = "url"]; - * @return The bytes for url. - */ - com.google.protobuf.ByteString - getUrlBytes(); - - /** - * string method = 5 [json_name = "method"]; - * @return The method. - */ - java.lang.String getMethod(); - /** - * string method = 5 [json_name = "method"]; - * @return The bytes for method. - */ - com.google.protobuf.ByteString - getMethodBytes(); - - /** - * string orig = 6 [json_name = "orig"]; - * @return The orig. - */ - java.lang.String getOrig(); - /** - * string orig = 6 [json_name = "orig"]; - * @return The bytes for orig. - */ - com.google.protobuf.ByteString - getOrigBytes(); - - /** - * string ip = 7 [json_name = "ip"]; - * @return The ip. - */ - java.lang.String getIp(); - /** - * string ip = 7 [json_name = "ip"]; - * @return The bytes for ip. - */ - com.google.protobuf.ByteString - getIpBytes(); - - /** - * string country = 8 [json_name = "country"]; - * @return The country. - */ - java.lang.String getCountry(); - /** - * string country = 8 [json_name = "country"]; - * @return The bytes for country. - */ - com.google.protobuf.ByteString - getCountryBytes(); - - /** - * int64 timestamp = 9 [json_name = "timestamp"]; - * @return The timestamp. - */ - long getTimestamp(); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ServiceProto.java deleted file mode 100644 index 4fec5af487..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/dashboard_service/v1/ServiceProto.java +++ /dev/null @@ -1,136 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/dashboard_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.dashboard_service.v1; - -public final class ServiceProto { - private ServiceProto() {} - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - ServiceProto.class.getName()); - } - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { - } - - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); - } - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_dashboard_service_v1_MaliciousRequest_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsResponse_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_dashboard_service_v1_ListMaliciousRequestsRequest_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersRequest_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_dashboard_service_v1_FetchAlertFiltersResponse_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n getRecordAlertMethod; - - @io.grpc.stub.annotations.RpcMethod( - fullMethodName = SERVICE_NAME + '/' + "RecordAlert", - requestType = com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.class, - responseType = com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.class, - methodType = io.grpc.MethodDescriptor.MethodType.UNARY) - public static io.grpc.MethodDescriptor getRecordAlertMethod() { - io.grpc.MethodDescriptor getRecordAlertMethod; - if ((getRecordAlertMethod = MaliciousAlertServiceGrpc.getRecordAlertMethod) == null) { - synchronized (MaliciousAlertServiceGrpc.class) { - if ((getRecordAlertMethod = MaliciousAlertServiceGrpc.getRecordAlertMethod) == null) { - MaliciousAlertServiceGrpc.getRecordAlertMethod = getRecordAlertMethod = - io.grpc.MethodDescriptor.newBuilder() - .setType(io.grpc.MethodDescriptor.MethodType.UNARY) - .setFullMethodName(generateFullMethodName(SERVICE_NAME, "RecordAlert")) - .setSampledToLocalTracing(true) - .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.getDefaultInstance())) - .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.getDefaultInstance())) - .setSchemaDescriptor(new MaliciousAlertServiceMethodDescriptorSupplier("RecordAlert")) - .build(); - } - } - } - return getRecordAlertMethod; - } - - /** - * Creates a new async stub that supports all call types for the service - */ - public static MaliciousAlertServiceStub newStub(io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public MaliciousAlertServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousAlertServiceStub(channel, callOptions); - } - }; - return MaliciousAlertServiceStub.newStub(factory, channel); - } - - /** - * Creates a new blocking-style stub that supports unary and streaming output calls on the service - */ - public static MaliciousAlertServiceBlockingStub newBlockingStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public MaliciousAlertServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousAlertServiceBlockingStub(channel, callOptions); - } - }; - return MaliciousAlertServiceBlockingStub.newStub(factory, channel); - } - - /** - * Creates a new ListenableFuture-style stub that supports unary calls on the service - */ - public static MaliciousAlertServiceFutureStub newFutureStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public MaliciousAlertServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousAlertServiceFutureStub(channel, callOptions); - } - }; - return MaliciousAlertServiceFutureStub.newStub(factory, channel); - } - - /** - */ - public interface AsyncService { - - /** - */ - default void recordAlert(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getRecordAlertMethod(), responseObserver); - } - } - - /** - * Base class for the server implementation of the service MaliciousAlertService. - */ - public static abstract class MaliciousAlertServiceImplBase - implements io.grpc.BindableService, AsyncService { - - @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { - return MaliciousAlertServiceGrpc.bindService(this); - } - } - - /** - * A stub to allow clients to do asynchronous rpc calls to service MaliciousAlertService. - */ - public static final class MaliciousAlertServiceStub - extends io.grpc.stub.AbstractAsyncStub { - private MaliciousAlertServiceStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected MaliciousAlertServiceStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousAlertServiceStub(channel, callOptions); - } - - /** - */ - public void recordAlert(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ClientCalls.asyncUnaryCall( - getChannel().newCall(getRecordAlertMethod(), getCallOptions()), request, responseObserver); - } - } - - /** - * A stub to allow clients to do synchronous rpc calls to service MaliciousAlertService. - */ - public static final class MaliciousAlertServiceBlockingStub - extends io.grpc.stub.AbstractBlockingStub { - private MaliciousAlertServiceBlockingStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected MaliciousAlertServiceBlockingStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousAlertServiceBlockingStub(channel, callOptions); - } - - /** - */ - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse recordAlert(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest request) { - return io.grpc.stub.ClientCalls.blockingUnaryCall( - getChannel(), getRecordAlertMethod(), getCallOptions(), request); - } - } - - /** - * A stub to allow clients to do ListenableFuture-style rpc calls to service MaliciousAlertService. - */ - public static final class MaliciousAlertServiceFutureStub - extends io.grpc.stub.AbstractFutureStub { - private MaliciousAlertServiceFutureStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected MaliciousAlertServiceFutureStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousAlertServiceFutureStub(channel, callOptions); - } - - /** - */ - public com.google.common.util.concurrent.ListenableFuture recordAlert( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest request) { - return io.grpc.stub.ClientCalls.futureUnaryCall( - getChannel().newCall(getRecordAlertMethod(), getCallOptions()), request); - } - } - - private static final int METHODID_RECORD_ALERT = 0; - - private static final class MethodHandlers implements - io.grpc.stub.ServerCalls.UnaryMethod, - io.grpc.stub.ServerCalls.ServerStreamingMethod, - io.grpc.stub.ServerCalls.ClientStreamingMethod, - io.grpc.stub.ServerCalls.BidiStreamingMethod { - private final AsyncService serviceImpl; - private final int methodId; - - MethodHandlers(AsyncService serviceImpl, int methodId) { - this.serviceImpl = serviceImpl; - this.methodId = methodId; - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - case METHODID_RECORD_ALERT: - serviceImpl.recordAlert((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) request, - (io.grpc.stub.StreamObserver) responseObserver); - break; - default: - throw new AssertionError(); - } - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public io.grpc.stub.StreamObserver invoke( - io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - default: - throw new AssertionError(); - } - } - } - - public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { - return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) - .addMethod( - getRecordAlertMethod(), - io.grpc.stub.ServerCalls.asyncUnaryCall( - new MethodHandlers< - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest, - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse>( - service, METHODID_RECORD_ALERT))) - .build(); - } - - private static abstract class MaliciousAlertServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { - MaliciousAlertServiceBaseDescriptorSupplier() {} - - @java.lang.Override - public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.getDescriptor(); - } - - @java.lang.Override - public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { - return getFileDescriptor().findServiceByName("MaliciousAlertService"); - } - } - - private static final class MaliciousAlertServiceFileDescriptorSupplier - extends MaliciousAlertServiceBaseDescriptorSupplier { - MaliciousAlertServiceFileDescriptorSupplier() {} - } - - private static final class MaliciousAlertServiceMethodDescriptorSupplier - extends MaliciousAlertServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { - private final java.lang.String methodName; - - MaliciousAlertServiceMethodDescriptorSupplier(java.lang.String methodName) { - this.methodName = methodName; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { - return getServiceDescriptor().findMethodByName(methodName); - } - } - - private static volatile io.grpc.ServiceDescriptor serviceDescriptor; - - public static io.grpc.ServiceDescriptor getServiceDescriptor() { - io.grpc.ServiceDescriptor result = serviceDescriptor; - if (result == null) { - synchronized (MaliciousAlertServiceGrpc.class) { - result = serviceDescriptor; - if (result == null) { - serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) - .setSchemaDescriptor(new MaliciousAlertServiceFileDescriptorSupplier()) - .addMethod(getRecordAlertMethod()) - .build(); - } - } - } - return result; - } -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequest.java deleted file mode 100644 index 25dbb34c2a..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequest.java +++ /dev/null @@ -1,1124 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -/** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordAlertRequest} - */ -public final class RecordAlertRequest extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) - RecordAlertRequestOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - RecordAlertRequest.class.getName()); - } - // Use RecordAlertRequest.newBuilder() to construct. - private RecordAlertRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private RecordAlertRequest() { - actor_ = ""; - filterId_ = ""; - sampleData_ = java.util.Collections.emptyList(); - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.Builder.class); - } - - public static final int ACTOR_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - @java.lang.Override - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TOTAL_EVENTS_FIELD_NUMBER = 3; - private int totalEvents_ = 0; - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @return The totalEvents. - */ - @java.lang.Override - public int getTotalEvents() { - return totalEvents_; - } - - public static final int DETECTED_AT_FIELD_NUMBER = 4; - private long detectedAt_ = 0L; - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - - public static final int SAMPLE_DATA_FIELD_NUMBER = 5; - @SuppressWarnings("serial") - private java.util.List sampleData_; - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public java.util.List getSampleDataList() { - return sampleData_; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public java.util.List - getSampleDataOrBuilderList() { - return sampleData_; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public int getSampleDataCount() { - return sampleData_.size(); - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getSampleData(int index) { - return sampleData_.get(index); - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( - int index) { - return sampleData_.get(index); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); - } - if (totalEvents_ != 0) { - output.writeInt32(3, totalEvents_); - } - if (detectedAt_ != 0L) { - output.writeInt64(4, detectedAt_); - } - for (int i = 0; i < sampleData_.size(); i++) { - output.writeMessage(5, sampleData_.get(i)); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); - } - if (totalEvents_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(3, totalEvents_); - } - if (detectedAt_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(4, detectedAt_); - } - for (int i = 0; i < sampleData_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(5, sampleData_.get(i)); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) obj; - - if (!getActor() - .equals(other.getActor())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (getTotalEvents() - != other.getTotalEvents()) return false; - if (getDetectedAt() - != other.getDetectedAt()) return false; - if (!getSampleDataList() - .equals(other.getSampleDataList())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACTOR_FIELD_NUMBER; - hash = (53 * hash) + getActor().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (37 * hash) + TOTAL_EVENTS_FIELD_NUMBER; - hash = (53 * hash) + getTotalEvents(); - hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getDetectedAt()); - if (getSampleDataCount() > 0) { - hash = (37 * hash) + SAMPLE_DATA_FIELD_NUMBER; - hash = (53 * hash) + getSampleDataList().hashCode(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordAlertRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - actor_ = ""; - filterId_ = ""; - totalEvents_ = 0; - detectedAt_ = 0L; - if (sampleDataBuilder_ == null) { - sampleData_ = java.util.Collections.emptyList(); - } else { - sampleData_ = null; - sampleDataBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest build() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest buildPartial() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest(this); - buildPartialRepeatedFields(result); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartialRepeatedFields(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest result) { - if (sampleDataBuilder_ == null) { - if (((bitField0_ & 0x00000010) != 0)) { - sampleData_ = java.util.Collections.unmodifiableList(sampleData_); - bitField0_ = (bitField0_ & ~0x00000010); - } - result.sampleData_ = sampleData_; - } else { - result.sampleData_ = sampleDataBuilder_.build(); - } - } - - private void buildPartial0(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.actor_ = actor_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.filterId_ = filterId_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.totalEvents_ = totalEvents_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.detectedAt_ = detectedAt_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) { - return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest other) { - if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest.getDefaultInstance()) return this; - if (!other.getActor().isEmpty()) { - actor_ = other.actor_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000002; - onChanged(); - } - if (other.getTotalEvents() != 0) { - setTotalEvents(other.getTotalEvents()); - } - if (other.getDetectedAt() != 0L) { - setDetectedAt(other.getDetectedAt()); - } - if (sampleDataBuilder_ == null) { - if (!other.sampleData_.isEmpty()) { - if (sampleData_.isEmpty()) { - sampleData_ = other.sampleData_; - bitField0_ = (bitField0_ & ~0x00000010); - } else { - ensureSampleDataIsMutable(); - sampleData_.addAll(other.sampleData_); - } - onChanged(); - } - } else { - if (!other.sampleData_.isEmpty()) { - if (sampleDataBuilder_.isEmpty()) { - sampleDataBuilder_.dispose(); - sampleDataBuilder_ = null; - sampleData_ = other.sampleData_; - bitField0_ = (bitField0_ & ~0x00000010); - sampleDataBuilder_ = - com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getSampleDataFieldBuilder() : null; - } else { - sampleDataBuilder_.addAllMessages(other.sampleData_); - } - } - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - actor_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000002; - break; - } // case 18 - case 24: { - totalEvents_ = input.readInt32(); - bitField0_ |= 0x00000004; - break; - } // case 24 - case 32: { - detectedAt_ = input.readInt64(); - bitField0_ |= 0x00000008; - break; - } // case 32 - case 42: { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent m = - input.readMessage( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.parser(), - extensionRegistry); - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.add(m); - } else { - sampleDataBuilder_.addMessage(m); - } - break; - } // case 42 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The actor to set. - * @return This builder for chaining. - */ - public Builder setActor( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @return This builder for chaining. - */ - public Builder clearActor() { - actor_ = getDefaultInstance().getActor(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The bytes for actor to set. - * @return This builder for chaining. - */ - public Builder setActorBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - private int totalEvents_ ; - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @return The totalEvents. - */ - @java.lang.Override - public int getTotalEvents() { - return totalEvents_; - } - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @param value The totalEvents to set. - * @return This builder for chaining. - */ - public Builder setTotalEvents(int value) { - - totalEvents_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @return This builder for chaining. - */ - public Builder clearTotalEvents() { - bitField0_ = (bitField0_ & ~0x00000004); - totalEvents_ = 0; - onChanged(); - return this; - } - - private long detectedAt_ ; - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @param value The detectedAt to set. - * @return This builder for chaining. - */ - public Builder setDetectedAt(long value) { - - detectedAt_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @return This builder for chaining. - */ - public Builder clearDetectedAt() { - bitField0_ = (bitField0_ & ~0x00000008); - detectedAt_ = 0L; - onChanged(); - return this; - } - - private java.util.List sampleData_ = - java.util.Collections.emptyList(); - private void ensureSampleDataIsMutable() { - if (!((bitField0_ & 0x00000010) != 0)) { - sampleData_ = new java.util.ArrayList(sampleData_); - bitField0_ |= 0x00000010; - } - } - - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder> sampleDataBuilder_; - - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public java.util.List getSampleDataList() { - if (sampleDataBuilder_ == null) { - return java.util.Collections.unmodifiableList(sampleData_); - } else { - return sampleDataBuilder_.getMessageList(); - } - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public int getSampleDataCount() { - if (sampleDataBuilder_ == null) { - return sampleData_.size(); - } else { - return sampleDataBuilder_.getCount(); - } - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getSampleData(int index) { - if (sampleDataBuilder_ == null) { - return sampleData_.get(index); - } else { - return sampleDataBuilder_.getMessage(index); - } - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder setSampleData( - int index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent value) { - if (sampleDataBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleDataIsMutable(); - sampleData_.set(index, value); - onChanged(); - } else { - sampleDataBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder setSampleData( - int index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder builderForValue) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.set(index, builderForValue.build()); - onChanged(); - } else { - sampleDataBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addSampleData(com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent value) { - if (sampleDataBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleDataIsMutable(); - sampleData_.add(value); - onChanged(); - } else { - sampleDataBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addSampleData( - int index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent value) { - if (sampleDataBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleDataIsMutable(); - sampleData_.add(index, value); - onChanged(); - } else { - sampleDataBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addSampleData( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder builderForValue) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.add(builderForValue.build()); - onChanged(); - } else { - sampleDataBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addSampleData( - int index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder builderForValue) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.add(index, builderForValue.build()); - onChanged(); - } else { - sampleDataBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder addAllSampleData( - java.lang.Iterable values) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, sampleData_); - onChanged(); - } else { - sampleDataBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder clearSampleData() { - if (sampleDataBuilder_ == null) { - sampleData_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000010); - onChanged(); - } else { - sampleDataBuilder_.clear(); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public Builder removeSampleData(int index) { - if (sampleDataBuilder_ == null) { - ensureSampleDataIsMutable(); - sampleData_.remove(index); - onChanged(); - } else { - sampleDataBuilder_.remove(index); - } - return this; - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder getSampleDataBuilder( - int index) { - return getSampleDataFieldBuilder().getBuilder(index); - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( - int index) { - if (sampleDataBuilder_ == null) { - return sampleData_.get(index); } else { - return sampleDataBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public java.util.List - getSampleDataOrBuilderList() { - if (sampleDataBuilder_ != null) { - return sampleDataBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(sampleData_); - } - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder addSampleDataBuilder() { - return getSampleDataFieldBuilder().addBuilder( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.getDefaultInstance()); - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder addSampleDataBuilder( - int index) { - return getSampleDataFieldBuilder().addBuilder( - index, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.getDefaultInstance()); - } - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - public java.util.List - getSampleDataBuilderList() { - return getSampleDataFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder> - getSampleDataFieldBuilder() { - if (sampleDataBuilder_ == null) { - sampleDataBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder>( - sampleData_, - ((bitField0_ & 0x00000010) != 0), - getParentForChildren(), - isClean()); - sampleData_ = null; - } - return sampleDataBuilder_; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) - private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest(); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RecordAlertRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequestOrBuilder.java deleted file mode 100644 index 30aa5d0f0e..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertRequestOrBuilder.java +++ /dev/null @@ -1,71 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -public interface RecordAlertRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.RecordAlertRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - java.lang.String getActor(); - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - com.google.protobuf.ByteString - getActorBytes(); - - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - java.lang.String getFilterId(); - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - com.google.protobuf.ByteString - getFilterIdBytes(); - - /** - * int32 total_events = 3 [json_name = "totalEvents"]; - * @return The totalEvents. - */ - int getTotalEvents(); - - /** - * int64 detected_at = 4 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - long getDetectedAt(); - - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - java.util.List - getSampleDataList(); - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getSampleData(int index); - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - int getSampleDataCount(); - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - java.util.List - getSampleDataOrBuilderList(); - /** - * repeated .threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent sample_data = 5 [json_name = "sampleData"]; - */ - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder getSampleDataOrBuilder( - int index); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponse.java deleted file mode 100644 index e5175020dd..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponse.java +++ /dev/null @@ -1,358 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -/** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordAlertResponse} - */ -public final class RecordAlertResponse extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) - RecordAlertResponseOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - RecordAlertResponse.class.getName()); - } - // Use RecordAlertResponse.newBuilder() to construct. - private RecordAlertResponse(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private RecordAlertResponse() { - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.Builder.class); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) obj; - - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordAlertResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordAlertResponse_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse build() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse buildPartial() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse(this); - onBuilt(); - return result; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) { - return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse other) { - if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse.getDefaultInstance()) return this; - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) - private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse(); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RecordAlertResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponseOrBuilder.java deleted file mode 100644 index 4ae50c1aa6..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordAlertResponseOrBuilder.java +++ /dev/null @@ -1,11 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -public interface RecordAlertResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.RecordAlertResponse) - com.google.protobuf.MessageOrBuilder { -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEvent.java deleted file mode 100644 index a375dc0fe3..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEvent.java +++ /dev/null @@ -1,1042 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -/** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent} - */ -public final class SampleMaliciousEvent extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) - SampleMaliciousEventOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SampleMaliciousEvent.class.getName()); - } - // Use SampleMaliciousEvent.newBuilder() to construct. - private SampleMaliciousEvent(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SampleMaliciousEvent() { - ip_ = ""; - url_ = ""; - method_ = ""; - payload_ = ""; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder.class); - } - - public static final int IP_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object ip_ = ""; - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - @java.lang.Override - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TIMESTAMP_FIELD_NUMBER = 2; - private long timestamp_ = 0L; - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - - public static final int URL_FIELD_NUMBER = 3; - @SuppressWarnings("serial") - private volatile java.lang.Object url_ = ""; - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - @java.lang.Override - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } - } - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int METHOD_FIELD_NUMBER = 4; - @SuppressWarnings("serial") - private volatile java.lang.Object method_ = ""; - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - @java.lang.Override - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } - } - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int API_COLLECTION_ID_FIELD_NUMBER = 5; - private int apiCollectionId_ = 0; - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - - public static final int PAYLOAD_FIELD_NUMBER = 6; - @SuppressWarnings("serial") - private volatile java.lang.Object payload_ = ""; - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - @java.lang.Override - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, ip_); - } - if (timestamp_ != 0L) { - output.writeInt64(2, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 3, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 4, method_); - } - if (apiCollectionId_ != 0) { - output.writeInt32(5, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 6, payload_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, ip_); - } - if (timestamp_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(2, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(3, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(4, method_); - } - if (apiCollectionId_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(5, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(6, payload_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) obj; - - if (!getIp() - .equals(other.getIp())) return false; - if (getTimestamp() - != other.getTimestamp()) return false; - if (!getUrl() - .equals(other.getUrl())) return false; - if (!getMethod() - .equals(other.getMethod())) return false; - if (getApiCollectionId() - != other.getApiCollectionId()) return false; - if (!getPayload() - .equals(other.getPayload())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + IP_FIELD_NUMBER; - hash = (53 * hash) + getIp().hashCode(); - hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getTimestamp()); - hash = (37 * hash) + URL_FIELD_NUMBER; - hash = (53 * hash) + getUrl().hashCode(); - hash = (37 * hash) + METHOD_FIELD_NUMBER; - hash = (53 * hash) + getMethod().hashCode(); - hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; - hash = (53 * hash) + getApiCollectionId(); - hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; - hash = (53 * hash) + getPayload().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - ip_ = ""; - timestamp_ = 0L; - url_ = ""; - method_ = ""; - apiCollectionId_ = 0; - payload_ = ""; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ConsumerServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_SampleMaliciousEvent_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent build() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent buildPartial() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.ip_ = ip_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.timestamp_ = timestamp_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.url_ = url_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.method_ = method_; - } - if (((from_bitField0_ & 0x00000010) != 0)) { - result.apiCollectionId_ = apiCollectionId_; - } - if (((from_bitField0_ & 0x00000020) != 0)) { - result.payload_ = payload_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) { - return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent other) { - if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent.getDefaultInstance()) return this; - if (!other.getIp().isEmpty()) { - ip_ = other.ip_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (other.getTimestamp() != 0L) { - setTimestamp(other.getTimestamp()); - } - if (!other.getUrl().isEmpty()) { - url_ = other.url_; - bitField0_ |= 0x00000004; - onChanged(); - } - if (!other.getMethod().isEmpty()) { - method_ = other.method_; - bitField0_ |= 0x00000008; - onChanged(); - } - if (other.getApiCollectionId() != 0) { - setApiCollectionId(other.getApiCollectionId()); - } - if (!other.getPayload().isEmpty()) { - payload_ = other.payload_; - bitField0_ |= 0x00000020; - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - ip_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 16: { - timestamp_ = input.readInt64(); - bitField0_ |= 0x00000002; - break; - } // case 16 - case 26: { - url_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000004; - break; - } // case 26 - case 34: { - method_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000008; - break; - } // case 34 - case 40: { - apiCollectionId_ = input.readInt32(); - bitField0_ |= 0x00000010; - break; - } // case 40 - case 50: { - payload_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000020; - break; - } // case 50 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object ip_ = ""; - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @param value The ip to set. - * @return This builder for chaining. - */ - public Builder setIp( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ip_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string ip = 1 [json_name = "ip"]; - * @return This builder for chaining. - */ - public Builder clearIp() { - ip_ = getDefaultInstance().getIp(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string ip = 1 [json_name = "ip"]; - * @param value The bytes for ip to set. - * @return This builder for chaining. - */ - public Builder setIpBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ip_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private long timestamp_ ; - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @param value The timestamp to set. - * @return This builder for chaining. - */ - public Builder setTimestamp(long value) { - - timestamp_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return This builder for chaining. - */ - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000002); - timestamp_ = 0L; - onChanged(); - return this; - } - - private java.lang.Object url_ = ""; - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string url = 3 [json_name = "url"]; - * @param value The url to set. - * @return This builder for chaining. - */ - public Builder setUrl( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - url_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * string url = 3 [json_name = "url"]; - * @return This builder for chaining. - */ - public Builder clearUrl() { - url_ = getDefaultInstance().getUrl(); - bitField0_ = (bitField0_ & ~0x00000004); - onChanged(); - return this; - } - /** - * string url = 3 [json_name = "url"]; - * @param value The bytes for url to set. - * @return This builder for chaining. - */ - public Builder setUrlBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - url_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - - private java.lang.Object method_ = ""; - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string method = 4 [json_name = "method"]; - * @param value The method to set. - * @return This builder for chaining. - */ - public Builder setMethod( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - method_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * string method = 4 [json_name = "method"]; - * @return This builder for chaining. - */ - public Builder clearMethod() { - method_ = getDefaultInstance().getMethod(); - bitField0_ = (bitField0_ & ~0x00000008); - onChanged(); - return this; - } - /** - * string method = 4 [json_name = "method"]; - * @param value The bytes for method to set. - * @return This builder for chaining. - */ - public Builder setMethodBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - method_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - - private int apiCollectionId_ ; - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @param value The apiCollectionId to set. - * @return This builder for chaining. - */ - public Builder setApiCollectionId(int value) { - - apiCollectionId_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return This builder for chaining. - */ - public Builder clearApiCollectionId() { - bitField0_ = (bitField0_ & ~0x00000010); - apiCollectionId_ = 0; - onChanged(); - return this; - } - - private java.lang.Object payload_ = ""; - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @param value The payload to set. - * @return This builder for chaining. - */ - public Builder setPayload( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - payload_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - /** - * string payload = 6 [json_name = "payload"]; - * @return This builder for chaining. - */ - public Builder clearPayload() { - payload_ = getDefaultInstance().getPayload(); - bitField0_ = (bitField0_ & ~0x00000020); - onChanged(); - return this; - } - /** - * string payload = 6 [json_name = "payload"]; - * @param value The bytes for payload to set. - * @return This builder for chaining. - */ - public Builder setPayloadBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - payload_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) - private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent(); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SampleMaliciousEvent parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEventOrBuilder.java deleted file mode 100644 index a1628943a1..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/SampleMaliciousEventOrBuilder.java +++ /dev/null @@ -1,71 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/consumer_service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -public interface SampleMaliciousEventOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent) - com.google.protobuf.MessageOrBuilder { - - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - java.lang.String getIp(); - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - com.google.protobuf.ByteString - getIpBytes(); - - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - long getTimestamp(); - - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - java.lang.String getUrl(); - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - com.google.protobuf.ByteString - getUrlBytes(); - - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - java.lang.String getMethod(); - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - com.google.protobuf.ByteString - getMethodBytes(); - - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - int getApiCollectionId(); - - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - java.lang.String getPayload(); - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - com.google.protobuf.ByteString - getPayloadBytes(); -} From 861835c70ccebc594e5d5bac148f1940fb282c52 Mon Sep 17 00:00:00 2001 From: Ajinkya <109141486+ag060@users.noreply.github.com> Date: Tue, 10 Dec 2024 12:25:02 +0530 Subject: [PATCH 42/73] added dashboard rpc service (#1808) --- apps/dashboard/pom.xml | 13 +- .../DashboardMaliciousRequest.java | 100 ++ .../SuspectSampleDataAction.java | 324 ++-- .../main/java/com/akto/grpc/AuthToken.java | 34 + apps/dashboard/src/main/resources/struts.xml | 1311 ++++++++--------- .../dashboard/pages/threat_detection/api.js | 4 +- .../akto/threat/protection/BackendServer.java | 6 +- .../protection/db/MaliciousEventModel.java | 160 +- .../protection/service/DashboardService.java | 104 ++ .../{ => service}/MaliciousAlertService.java | 39 +- .../dashboard_service/v1/service.proto | 43 + 11 files changed, 1186 insertions(+), 952 deletions(-) create mode 100644 apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousRequest.java create mode 100644 apps/dashboard/src/main/java/com/akto/grpc/AuthToken.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java rename apps/threat-protection-backend/src/main/java/com/akto/threat/protection/{ => service}/MaliciousAlertService.java (64%) create mode 100644 protobuf/threat_protection/service/dashboard_service/v1/service.proto diff --git a/apps/dashboard/pom.xml b/apps/dashboard/pom.xml index c7966302b5..0825d40a19 100644 --- a/apps/dashboard/pom.xml +++ b/apps/dashboard/pom.xml @@ -25,7 +25,7 @@ - + com.amazonaws aws-java-sdk-lambda 1.12.405 @@ -109,6 +109,11 @@ dao ${project.version} + + com.akto.libs.protobuf + protobuf + ${project.version} + com.akto.libs.utils utils @@ -242,6 +247,12 @@ commons-csv 1.9.0 + + grpc-api + io.grpc + 1.68.1 + compile + src/main/java diff --git a/apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousRequest.java b/apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousRequest.java new file mode 100644 index 0000000000..6b691e8190 --- /dev/null +++ b/apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousRequest.java @@ -0,0 +1,100 @@ +package com.akto.action.threat_detection; + +import com.akto.dto.type.URLMethods; +import com.akto.dto.type.URLMethods.Method; + +public class DashboardMaliciousRequest { + private String id; + private String actor; + private String filter_id; + private String url; + private URLMethods.Method method; + private String ip; + private String country; + private long timestamp; + + public DashboardMaliciousRequest() {} + + public DashboardMaliciousRequest( + String id, + String actor, + String filter, + String url, + Method method, + String ip, + String country, + long timestamp) { + this.id = id; + this.actor = actor; + this.filter_id = filter; + this.url = url; + this.method = method; + this.ip = ip; + this.country = country; + this.timestamp = timestamp; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getActor() { + return actor; + } + + public void setActor(String actor) { + this.actor = actor; + } + + public String getFilterId() { + return filter_id; + } + + public void setFilterId(String filter) { + this.filter_id = filter; + } + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public URLMethods.Method getMethod() { + return method; + } + + public void setMethod(URLMethods.Method method) { + this.method = method; + } + + public String getIp() { + return ip; + } + + public void setIp(String ip) { + this.ip = ip; + } + + public String getCountry() { + return country; + } + + public void setCountry(String country) { + this.country = country; + } + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } +} diff --git a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java index d271463da8..ab221922b7 100644 --- a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java +++ b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java @@ -1,184 +1,156 @@ package com.akto.action.threat_detection; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.bson.conversions.Bson; - import com.akto.action.UserAction; -import com.akto.dao.SuspectSampleDataDao; -import com.akto.dao.context.Context; import com.akto.dto.traffic.SuspectSampleData; -import com.akto.util.Constants; -import com.mongodb.client.model.Filters; -import com.mongodb.client.model.Sorts; +import com.akto.dto.type.URLMethods; +import com.akto.grpc.AuthToken; +import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc; +import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceBlockingStub; +import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest; +import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse; +import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; +import com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; public class SuspectSampleDataAction extends UserAction { - List sampleData; - int skip; - static final int LIMIT = 50; - List ips; - List urls; - List apiCollectionIds; - long total; - Map sort; - int startTimestamp, endTimestamp; - - public String fetchSuspectSampleData() { - - List filterList = new ArrayList<>(); - - /* - * In case time filters are empty, - * using default filter as 2 months. - */ - - if (startTimestamp <= 0) { - startTimestamp = Context.now() - 2 * 30 * 24 * 60 * 60; - } - if (endTimestamp <= 0) { - endTimestamp = Context.now() + 10 * 60; - } - - filterList.add(Filters.gte(SuspectSampleData._DISCOVERED, startTimestamp)); - filterList.add(Filters.lte(SuspectSampleData._DISCOVERED, endTimestamp)); - - if (ips != null && !ips.isEmpty()) { - filterList.add(Filters.in(SuspectSampleData.SOURCE_IPS, ips)); - } - if (urls != null && !urls.isEmpty()) { - filterList.add(Filters.in(SuspectSampleData.MATCHING_URL, urls)); - } - if (apiCollectionIds != null && !apiCollectionIds.isEmpty()) { - filterList.add(Filters.in(SuspectSampleData.API_COLLECTION_ID, apiCollectionIds)); - } - - Bson finalFilter = Filters.empty(); - - if (!filterList.isEmpty()) { - finalFilter = Filters.and(filterList); - } - - String sortKey = SuspectSampleData._DISCOVERED; - int sortDirection = -1; - /* - * add any new sort key here, - * for validation and sanity. - */ - Set sortKeys = new HashSet<>(); - sortKeys.add(SuspectSampleData._DISCOVERED); - - if (sort != null && !sort.isEmpty()) { - Entry sortEntry = sort.entrySet().iterator().next(); - sortKey = sortEntry.getKey(); - if (!sortKeys.contains(sortKey)) { - sortKey = SuspectSampleData._DISCOVERED; - } - sortDirection = sortEntry.getValue(); - if (!(sortDirection == -1 || sortDirection == 1)) { - sortDirection = -1; - } - } - - /* - * In case timestamp is same, then id acts as tie-breaker, - * to avoid repeating the same documents again. - */ - Bson sort = sortDirection == -1 ? Sorts.descending(sortKey, Constants.ID) - : Sorts.ascending(sortKey, Constants.ID); - sampleData = SuspectSampleDataDao.instance.findAll(finalFilter, skip, LIMIT, sort); - total = SuspectSampleDataDao.instance.count(finalFilter); - - return SUCCESS.toUpperCase(); - } - - public String fetchFilters() { - ips = new ArrayList<>( - SuspectSampleDataDao.instance.findDistinctFields(SuspectSampleData.SOURCE_IPS, String.class, Filters.empty())); - urls = new ArrayList<>(SuspectSampleDataDao.instance.findDistinctFields(SuspectSampleData.MATCHING_URL, String.class, - Filters.empty())); - return SUCCESS.toUpperCase(); - } - - public List getSampleData() { - return sampleData; - } - - public void setSampleData(List sampleData) { - this.sampleData = sampleData; - } - - public int getSkip() { - return skip; - } - - public void setSkip(int skip) { - this.skip = skip; - } - - public static int getLimit() { - return LIMIT; - } - - public List getIps() { - return ips; - } - - public void setIps(List ips) { - this.ips = ips; - } - - public List getUrls() { - return urls; - } - - public void setUrls(List urls) { - this.urls = urls; - } - - public List getApiCollectionIds() { - return apiCollectionIds; - } - - public void setApiCollectionIds(List apiCollectionIds) { - this.apiCollectionIds = apiCollectionIds; - } - - public long getTotal() { - return total; - } - - public void setTotal(long total) { - this.total = total; - } - - public Map getSort() { - return sort; - } - - public void setSort(Map sort) { - this.sort = sort; - } - - public int getStartTimestamp() { - return startTimestamp; - } - - public void setStartTimestamp(int startTimestamp) { - this.startTimestamp = startTimestamp; - } - - public int getEndTimestamp() { - return endTimestamp; - } - - public void setEndTimestamp(int endTimestamp) { - this.endTimestamp = endTimestamp; - } - + List sampleData; + List maliciousRequests; + int skip; + static final int LIMIT = 50; + List ips; + List urls; + List apiCollectionIds; + long total; + Map sort; + int startTimestamp, endTimestamp; + + private final DashboardServiceBlockingStub dsServiceStub; + + public SuspectSampleDataAction() { + super(); + + String target = "localhost:8980"; + ManagedChannel channel = + Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); + this.dsServiceStub = + DashboardServiceGrpc.newBlockingStub(channel) + .withCallCredentials( + new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); + } + + public String fetchSampleDataV2() { + List maliciousRequests = + this.dsServiceStub + .listMaliciousRequests( + ListMaliciousRequestsRequest.newBuilder().setPage(0).setLimit(500).build()) + .getMaliciousRequestsList(); + + this.maliciousRequests = + maliciousRequests.stream() + .map( + mr -> + new DashboardMaliciousRequest( + mr.getId(), + mr.getActor(), + mr.getFilterId(), + mr.getUrl(), + URLMethods.Method.fromString(mr.getMethod()), + mr.getIp(), + mr.getCountry(), + mr.getTimestamp())) + .collect(Collectors.toList()); + + return SUCCESS.toUpperCase(); + } + + public String fetchFiltersV2() { + FetchAlertFiltersResponse filters = + this.dsServiceStub.fetchAlertFilters(FetchAlertFiltersRequest.newBuilder().build()); + ips = filters.getActorsList(); + urls = filters.getUrlsList(); + + return SUCCESS.toUpperCase(); + } + + public List getSampleData() { + return sampleData; + } + + public void setSampleData(List sampleData) { + this.sampleData = sampleData; + } + + public int getSkip() { + return skip; + } + + public void setSkip(int skip) { + this.skip = skip; + } + + public static int getLimit() { + return LIMIT; + } + + public List getIps() { + return ips; + } + + public void setIps(List ips) { + this.ips = ips; + } + + public List getUrls() { + return urls; + } + + public void setUrls(List urls) { + this.urls = urls; + } + + public List getApiCollectionIds() { + return apiCollectionIds; + } + + public void setApiCollectionIds(List apiCollectionIds) { + this.apiCollectionIds = apiCollectionIds; + } + + public long getTotal() { + return total; + } + + public void setTotal(long total) { + this.total = total; + } + + public Map getSort() { + return sort; + } + + public void setSort(Map sort) { + this.sort = sort; + } + + public int getStartTimestamp() { + return startTimestamp; + } + + public void setStartTimestamp(int startTimestamp) { + this.startTimestamp = startTimestamp; + } + + public int getEndTimestamp() { + return endTimestamp; + } + + public void setEndTimestamp(int endTimestamp) { + this.endTimestamp = endTimestamp; + } } diff --git a/apps/dashboard/src/main/java/com/akto/grpc/AuthToken.java b/apps/dashboard/src/main/java/com/akto/grpc/AuthToken.java new file mode 100644 index 0000000000..c10bf4f224 --- /dev/null +++ b/apps/dashboard/src/main/java/com/akto/grpc/AuthToken.java @@ -0,0 +1,34 @@ +package com.akto.grpc; + +import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; + +import io.grpc.CallCredentials; +import io.grpc.Metadata; +import io.grpc.Status; +import java.util.concurrent.Executor; + +public class AuthToken extends CallCredentials { + + private final String token; + public static final Metadata.Key AUTHORIZATION_METADATA_KEY = + Metadata.Key.of("Authorization", ASCII_STRING_MARSHALLER); + + public AuthToken(String token) { + this.token = token; + } + + @Override + public void applyRequestMetadata( + RequestInfo requestInfo, Executor appExecutor, MetadataApplier applier) { + appExecutor.execute( + () -> { + try { + Metadata headers = new Metadata(); + headers.put(AUTHORIZATION_METADATA_KEY, token); + applier.apply(headers); + } catch (Throwable e) { + applier.fail(Status.UNAUTHENTICATED.withCause(e)); + } + }); + } +} diff --git a/apps/dashboard/src/main/resources/struts.xml b/apps/dashboard/src/main/resources/struts.xml index 0381d57e7b..c3065f4c62 100644 --- a/apps/dashboard/src/main/resources/struts.xml +++ b/apps/dashboard/src/main/resources/struts.xml @@ -15,8 +15,10 @@ - - + + @@ -44,7 +46,7 @@ - + akto_health @@ -56,7 +58,7 @@ - + @@ -69,7 +71,7 @@ - + @@ -79,16 +81,16 @@ - + - + 401 - + subscription @@ -99,9 +101,9 @@ - + - + 477 @@ -112,16 +114,16 @@ - + - + 401 - + ACTIVE_ACCOUNTS @@ -136,7 +138,7 @@ false ^actionErrors.* - + 401 @@ -148,7 +150,7 @@ - + USER_ACTIONS @@ -169,7 +171,7 @@ - + INTEGRATIONS @@ -190,7 +192,7 @@ - + INTEGRATIONS @@ -202,14 +204,14 @@ false ^actionErrors.* - + 401 - + INTEGRATIONS @@ -220,7 +222,7 @@ false ^actionErrors.* - + driveNamesToThirdPartyId 401 @@ -228,7 +230,7 @@ - + INTEGRATIONS @@ -239,14 +241,14 @@ false ^actionErrors.* - + 401 - + INTEGRATIONS @@ -257,14 +259,14 @@ false ^actionErrors.* - + 401 - + INTEGRATIONS @@ -275,7 +277,7 @@ false ^actionErrors.* - + 401 @@ -284,7 +286,7 @@ - + INVITE_MEMBERS @@ -295,18 +297,14 @@ false ^actionErrors.* - - users - - - 422 - false - ^actionErrors.* + + + 401 - + ADMIN_ACTIONS @@ -318,16 +316,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + USER_ACTIONS @@ -339,7 +335,7 @@ false ^actionErrors.* - + 422 false @@ -348,7 +344,7 @@ - + USER_ACTIONS @@ -370,18 +366,18 @@ - + - + code - + - + code @@ -389,52 +385,52 @@ - + - + code - + - + code - + - + code - + - + code - + - + code - + AKTO_SSO @@ -450,7 +446,7 @@ false ^actionErrors.* - + 422 false @@ -464,7 +460,7 @@ - + AKTO_SSO @@ -480,7 +476,7 @@ false ^actionErrors.* - + 422 false @@ -494,7 +490,7 @@ - + ADMIN_ACTIONS @@ -506,7 +502,7 @@ false ^actionErrors.* - + 422 false @@ -515,7 +511,7 @@ - + AKTO_SSO @@ -531,7 +527,7 @@ false ^actionErrors.* - + 422 false @@ -545,7 +541,7 @@ - + AKTO_SSO @@ -561,7 +557,7 @@ false ^actionErrors.* - + 422 false @@ -575,7 +571,7 @@ - + ADMIN_ACTIONS @@ -587,7 +583,7 @@ false ^actionErrors.* - + 422 false @@ -596,7 +592,7 @@ - + AKTO_SSO @@ -612,7 +608,7 @@ false ^actionErrors.* - + 422 false @@ -626,7 +622,7 @@ - + AKTO_SSO @@ -642,7 +638,7 @@ false ^actionErrors.* - + 422 false @@ -656,7 +652,7 @@ - + ADMIN_ACTIONS @@ -668,7 +664,7 @@ false ^actionErrors.* - + 422 false @@ -677,7 +673,7 @@ - + ADMIN_ACTIONS @@ -689,7 +685,7 @@ false ^actionErrors.* - + 422 false @@ -698,7 +694,7 @@ - + ADMIN_ACTIONS @@ -711,7 +707,7 @@ false ^actionErrors.* - + 422 false @@ -720,7 +716,7 @@ - + ADMIN_ACTIONS @@ -733,7 +729,7 @@ false ^actionErrors.* - + 422 false @@ -742,7 +738,7 @@ - + USER_ACTIONS @@ -766,9 +762,9 @@ - + - + API_COLLECTIONS READ @@ -795,9 +791,9 @@ - + - + API_COLLECTIONS READ @@ -824,9 +820,9 @@ - + - + API_COLLECTIONS READ @@ -853,9 +849,9 @@ - + - + API_COLLECTIONS READ @@ -884,7 +880,7 @@ - + API_COLLECTIONS @@ -907,7 +903,7 @@ - + API_COLLECTIONS @@ -930,7 +926,7 @@ - + API_COLLECTIONS @@ -953,7 +949,7 @@ - + API_COLLECTIONS @@ -976,7 +972,7 @@ - + API_COLLECTIONS @@ -999,7 +995,7 @@ - + API_COLLECTIONS @@ -1011,12 +1007,10 @@ false ^actionErrors.* - - - - 422 - false - ^actionErrors.* + + + + 401 403 @@ -1026,7 +1020,7 @@ - + API_COLLECTIONS @@ -1038,7 +1032,7 @@ false ^actionErrors.* - + response @@ -1056,7 +1050,7 @@ - + SENSITIVE_DATA @@ -1068,7 +1062,7 @@ false ^actionErrors.* - + response @@ -1085,7 +1079,7 @@ - + SENSITIVE_DATA @@ -1109,7 +1103,7 @@ - + API_COLLECTIONS @@ -1121,7 +1115,7 @@ false ^actionErrors.* - + response @@ -1138,7 +1132,7 @@ - + ISSUES @@ -1160,7 +1154,7 @@ - + USER_ACTIONS @@ -1183,7 +1177,7 @@ - + response @@ -1194,7 +1188,7 @@ - + SENSITIVE_DATA @@ -1217,7 +1211,7 @@ - + SENSITIVE_DATA @@ -1240,7 +1234,7 @@ - + SENSITIVE_DATA @@ -1261,7 +1255,7 @@ - + USER_CONFIG @@ -1273,7 +1267,7 @@ false ^actionErrors.* - + 422 false @@ -1282,7 +1276,7 @@ - + USER_CONFIG @@ -1295,7 +1289,7 @@ false ^actionErrors.* - + 422 false @@ -1304,7 +1298,7 @@ - + TEST_ROLES @@ -1316,7 +1310,7 @@ false ^actionErrors.* - + 422 false @@ -1325,7 +1319,7 @@ - + USER_CONFIG @@ -1338,7 +1332,7 @@ false ^actionErrors.* - + 422 false @@ -1347,7 +1341,7 @@ - + USER_CONFIG @@ -1360,7 +1354,7 @@ false ^actionErrors.* - + 422 false @@ -1368,7 +1362,7 @@ - + USER_CONFIG @@ -1381,7 +1375,7 @@ false ^actionErrors.* - + 422 false @@ -1390,7 +1384,7 @@ - + USER_CONFIG @@ -1403,7 +1397,7 @@ false ^actionErrors.* - + 422 false @@ -1412,7 +1406,7 @@ - + API_COLLECTIONS @@ -1435,7 +1429,7 @@ - + API_COLLECTIONS @@ -1448,7 +1442,7 @@ false ^actionErrors.* - + 422 false @@ -1457,7 +1451,7 @@ - + ACTIVE_ENDPOINTS @@ -1488,7 +1482,7 @@ - + API_COLLECTIONS @@ -1501,7 +1495,7 @@ false ^actionErrors.* - + 422 false @@ -1514,7 +1508,7 @@ - + INTEGRATIONS @@ -1544,7 +1538,7 @@ - + INTEGRATIONS @@ -1575,7 +1569,7 @@ - + API_COLLECTIONS @@ -1596,7 +1590,7 @@ - + API_COLLECTIONS @@ -1617,7 +1611,7 @@ - + API_COLLECTIONS @@ -1639,7 +1633,7 @@ - + API_COLLECTIONS @@ -1660,7 +1654,7 @@ - + API_COLLECTIONS @@ -1682,7 +1676,7 @@ - + API_COLLECTIONS @@ -1704,7 +1698,7 @@ - + API_COLLECTIONS @@ -1726,7 +1720,7 @@ - + API_COLLECTIONS @@ -1747,7 +1741,7 @@ - + API_COLLECTIONS @@ -1769,7 +1763,7 @@ - + API_COLLECTIONS @@ -1791,7 +1785,7 @@ - + API_COLLECTIONS @@ -1813,7 +1807,7 @@ - + API_COLLECTIONS @@ -1835,7 +1829,7 @@ - + API_COLLECTIONS @@ -1857,7 +1851,7 @@ - + API_COLLECTIONS @@ -1879,7 +1873,7 @@ - + API_COLLECTIONS @@ -1901,7 +1895,7 @@ - + API_COLLECTIONS @@ -1913,7 +1907,7 @@ false ^actionErrors.* - + 422 false @@ -1922,7 +1916,7 @@ - + API_COLLECTIONS @@ -1945,7 +1939,7 @@ - + API_COLLECTIONS @@ -1975,8 +1969,8 @@ - - + + API_COLLECTIONS READ_WRITE @@ -1995,7 +1989,7 @@ - + API_COLLECTIONS @@ -2024,7 +2018,7 @@ - + API_COLLECTIONS @@ -2046,7 +2040,7 @@ - + API_COLLECTIONS @@ -2068,7 +2062,7 @@ - + API_COLLECTIONS @@ -2090,7 +2084,7 @@ - + API_COLLECTIONS @@ -2111,7 +2105,7 @@ - + 403 @@ -2128,7 +2122,7 @@ - + INTEGRATIONS @@ -2148,7 +2142,7 @@ - + INTEGRATIONS @@ -2170,7 +2164,7 @@ - + INTEGRATIONS @@ -2192,7 +2186,7 @@ - + INTEGRATIONS @@ -2213,7 +2207,7 @@ - + INTEGRATIONS @@ -2234,21 +2228,21 @@ - + - + - + INTEGRATIONS @@ -2269,7 +2263,7 @@ - + INTEGRATIONS @@ -2299,7 +2293,7 @@ - + INTEGRATIONS @@ -2329,7 +2323,7 @@ - + INTEGRATIONS @@ -2351,7 +2345,7 @@ - + AKTO_EXTERNAL_API CI_CD_INTEGRATION @@ -2379,7 +2373,7 @@ - + INTEGRATIONS @@ -2401,7 +2395,7 @@ - + INTEGRATIONS @@ -2422,7 +2416,7 @@ - + INTEGRATIONS @@ -2444,7 +2438,7 @@ - + TRAFFIC_FILTERS @@ -2455,12 +2449,10 @@ false ^actionErrors.* - - - - 422 - false - ^actionErrors.* + + + + 401 403 @@ -2470,7 +2462,7 @@ - + SAMPLE_DATA @@ -2481,12 +2473,10 @@ false ^actionErrors.* - - - - 422 - false - ^actionErrors.* + + + + 401 403 @@ -2496,8 +2486,8 @@ - - + + API_COLLECTIONS READ_WRITE @@ -2518,8 +2508,8 @@ - - + + API_COLLECTIONS READ_WRITE @@ -2591,7 +2581,7 @@ - + API_COLLECTIONS @@ -2612,7 +2602,7 @@ - + API_COLLECTIONS @@ -2634,7 +2624,7 @@ - + TRAFFIC_FILTERS @@ -2655,7 +2645,7 @@ - + SAMPLE_DATA @@ -2676,7 +2666,7 @@ - + SAMPLE_DATA @@ -2687,12 +2677,10 @@ false ^actionErrors.* - - - - 422 - false - ^actionErrors.* + + + + 401 403 @@ -2702,7 +2690,7 @@ - + SENSITIVE_DATA @@ -2713,16 +2701,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + SENSITIVE_DATA @@ -2744,7 +2730,7 @@ - + TAGS @@ -2755,16 +2741,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + API_COLLECTIONS @@ -2775,16 +2759,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + API_COLLECTIONS @@ -2806,7 +2788,7 @@ - + SENSITIVE_DATA @@ -2821,7 +2803,7 @@ CUSTOM_DATA_TYPES - + 422 false @@ -2835,7 +2817,7 @@ - + SENSITIVE_DATA @@ -2847,7 +2829,7 @@ false ^actionErrors.* - + 422 false @@ -2856,7 +2838,7 @@ - + SENSITIVE_DATA @@ -2868,7 +2850,7 @@ false ^actionErrors.* - + 422 false @@ -2877,7 +2859,7 @@ - + SENSITIVE_DATA @@ -2889,7 +2871,7 @@ false ^actionErrors.* - + 422 false @@ -2898,7 +2880,7 @@ - + SENSITIVE_DATA @@ -2909,7 +2891,7 @@ false ^actionErrors.* - + 422 false @@ -2918,7 +2900,7 @@ - + SENSITIVE_DATA @@ -2930,7 +2912,7 @@ false ^actionErrors.* - + 422 false @@ -2939,7 +2921,7 @@ - + SENSITIVE_DATA @@ -2951,7 +2933,7 @@ false ^actionErrors.* - + 422 false @@ -2960,7 +2942,7 @@ - + SENSITIVE_DATA @@ -2971,16 +2953,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + AUTH_TYPE @@ -2991,7 +2971,7 @@ false ^actionErrors.* - + 422 false @@ -3000,7 +2980,7 @@ - + AUTH_TYPE @@ -3012,7 +2992,7 @@ false ^actionErrors.* - + 422 false @@ -3021,7 +3001,7 @@ - + AUTH_TYPE @@ -3033,7 +3013,7 @@ false ^actionErrors.* - + 422 false @@ -3043,7 +3023,7 @@ - + AUTH_TYPE @@ -3055,7 +3035,7 @@ false ^actionErrors.* - + 422 false @@ -3064,7 +3044,7 @@ - + AUTH_TYPE @@ -3076,7 +3056,7 @@ false ^actionErrors.* - + 422 false @@ -3085,7 +3065,7 @@ - + API_DATA_REDACTION @@ -3101,11 +3081,9 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 403 @@ -3115,7 +3093,7 @@ - + ADMIN_ACTIONS @@ -3127,16 +3105,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + ADMIN_ACTIONS @@ -3149,16 +3125,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + ADMIN_ACTIONS @@ -3171,16 +3145,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + ADMIN_ACTIONS @@ -3193,16 +3165,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + ADMIN_ACTIONS @@ -3214,11 +3184,11 @@ false ^actionErrors.* - + - + ADMIN_ACTIONS @@ -3242,7 +3212,7 @@ - + ADMIN_ACTIONS @@ -3254,7 +3224,7 @@ false ^actionErrors.* - + 422 false @@ -3263,7 +3233,7 @@ - + ADMIN_ACTIONS @@ -3287,7 +3257,7 @@ - + TAGS @@ -3300,7 +3270,7 @@ false ^actionErrors.* - + 422 false @@ -3309,7 +3279,7 @@ - + TAGS @@ -3322,7 +3292,7 @@ false ^actionErrors.* - + 422 false @@ -3331,7 +3301,7 @@ - + TAGS @@ -3344,7 +3314,7 @@ false ^actionErrors.* - + 422 false @@ -3353,7 +3323,7 @@ - + ADMIN_ACTIONS @@ -3366,7 +3336,7 @@ false ^actionErrors.* - + 401 @@ -3374,7 +3344,7 @@ - + TEST_RUNS @@ -3390,7 +3360,7 @@ false ^actionErrors.* - + 422 false @@ -3404,7 +3374,7 @@ - + TEST_RESULTS @@ -3416,7 +3386,7 @@ false ^actionErrors.* - + 422 false @@ -3425,7 +3395,7 @@ - + TEST_RESULTS @@ -3448,7 +3418,7 @@ - + TEST_RESULTS @@ -3470,7 +3440,7 @@ - + TEST_RESULTS @@ -3493,7 +3463,7 @@ - + TEST_RESULTS @@ -3505,7 +3475,7 @@ false ^actionErrors.* - + 422 false @@ -3514,7 +3484,7 @@ - + TEST_RESULTS @@ -3526,7 +3496,7 @@ false ^actionErrors.* - + 422 false @@ -3535,7 +3505,7 @@ - + START_TEST_RUN @@ -3548,7 +3518,7 @@ false ^actionErrors.* - + 422 false @@ -3557,7 +3527,7 @@ - + USER_CONFIG @@ -3573,7 +3543,7 @@ AUTOMATED_AUTH_TOKEN - + 422 false @@ -3587,7 +3557,7 @@ - + USER_CONFIG @@ -3600,7 +3570,7 @@ false ^actionErrors.* - + 422 false @@ -3609,7 +3579,7 @@ - + USER_CONFIG @@ -3621,7 +3591,7 @@ false ^actionErrors.* - + 422 false @@ -3630,7 +3600,7 @@ - + USER_CONFIG @@ -3642,7 +3612,7 @@ false ^actionErrors.* - + 422 false @@ -3651,9 +3621,9 @@ - + - + 422 false @@ -3662,7 +3632,7 @@ - + true @@ -3673,7 +3643,7 @@ - + true @@ -3684,7 +3654,7 @@ - + true @@ -3695,16 +3665,16 @@ - + - + 401 - + SAMPLE_DATA @@ -3716,12 +3686,10 @@ false ^actionErrors.* - - - - 422 - false - ^actionErrors.* + + + + 401 403 @@ -3731,7 +3699,7 @@ - + TEST_RESULTS @@ -3743,7 +3711,7 @@ false ^actionErrors.* - + 422 false @@ -3752,7 +3720,7 @@ - + TEST_RESULTS @@ -3764,7 +3732,7 @@ false ^actionErrors.* - + 422 false @@ -3773,7 +3741,7 @@ - + TEST_RESULTS @@ -3785,7 +3753,7 @@ false ^actionErrors.* - + 422 false @@ -3794,7 +3762,7 @@ - + ADMIN_ACTIONS @@ -3807,7 +3775,7 @@ false ^actionErrors.* - + 422 false @@ -3816,7 +3784,7 @@ - + ADMIN_ACTIONS @@ -3829,7 +3797,7 @@ false ^actionErrors.* - + 422 false @@ -3838,7 +3806,7 @@ - + ADMIN_ACTIONS @@ -3851,7 +3819,7 @@ false ^actionErrors.* - + 422 false @@ -3860,7 +3828,7 @@ - + DEFAULT_PAYLOADS @@ -3885,9 +3853,9 @@ - + - + 422 false @@ -3896,7 +3864,7 @@ - + LOGS @@ -3908,7 +3876,7 @@ false ^actionErrors.* - + 422 @@ -3918,7 +3886,7 @@ - + LOGS @@ -3930,7 +3898,7 @@ false ^actionErrors.* - + 422 false @@ -3939,51 +3907,7 @@ - - - - ISSUES - READ - - - - 403 - false - ^actionErrors.* - - - - 422 - false - ^actionErrors.* - - - - - - - - ISSUES - READ - - - - 403 - false - ^actionErrors.* - - - generatedReportId - - - 422 - false - ^actionErrors.* - - - - - + ISSUES @@ -3995,18 +3919,14 @@ false ^actionErrors.* - - response - - - 422 - false - ^actionErrors.* + + + 401 - + ISSUES @@ -4018,11 +3938,9 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 @@ -4050,7 +3968,7 @@ - + ISSUES @@ -4062,16 +3980,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + ISSUES @@ -4094,7 +4010,7 @@ - + TEST_ROLES @@ -4106,7 +4022,7 @@ false ^actionErrors.* - + 422 false @@ -4115,7 +4031,7 @@ - + TEST_ROLES @@ -4128,7 +4044,7 @@ false ^actionErrors.* - + 422 false @@ -4137,7 +4053,7 @@ - + TEST_ROLES @@ -4150,7 +4066,7 @@ false ^actionErrors.* - + 422 false @@ -4159,7 +4075,7 @@ - + TEST_ROLES @@ -4172,7 +4088,7 @@ false ^actionErrors.* - + 422 false @@ -4181,7 +4097,7 @@ - + TEST_ROLES @@ -4194,7 +4110,7 @@ false ^actionErrors.* - + 422 false @@ -4203,7 +4119,7 @@ - + TEST_ROLES @@ -4216,7 +4132,7 @@ false ^actionErrors.* - + 422 false @@ -4225,7 +4141,7 @@ - + TEST_ROLES @@ -4238,7 +4154,7 @@ false ^actionErrors.* - + 422 false @@ -4247,7 +4163,7 @@ - + TEST_RESULTS @@ -4259,16 +4175,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + TEST_RESULTS @@ -4280,16 +4194,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + TEST_RESULTS @@ -4301,16 +4213,14 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + 401 - + ISSUES @@ -4323,7 +4233,7 @@ false ^actionErrors.* - + 422 false @@ -4332,7 +4242,7 @@ - + ISSUES @@ -4345,7 +4255,7 @@ false ^actionErrors.* - + 422 false @@ -4354,7 +4264,7 @@ - + START_TEST_RUN @@ -4367,7 +4277,7 @@ false ^actionErrors.* - + 422 @@ -4377,7 +4287,7 @@ - + START_TEST_RUN @@ -4390,7 +4300,7 @@ false ^actionErrors.* - + 422 @@ -4400,7 +4310,7 @@ - + TEST_RESULTS @@ -4413,16 +4323,15 @@ false ^actionErrors.* - - - 422 - false - ^actionErrors.* + + + + 401 - + TEST_RESULTS @@ -4435,7 +4344,7 @@ false ^actionErrors.* - + 422 @@ -4445,7 +4354,7 @@ - + TEST_RESULTS @@ -4458,7 +4367,7 @@ false ^actionErrors.* - + 422 @@ -4468,7 +4377,7 @@ - + TEST_RESULTS @@ -4480,7 +4389,7 @@ false ^actionErrors.* - + 422 @@ -4490,7 +4399,7 @@ - + TEST_RESULTS @@ -4502,7 +4411,7 @@ false ^actionErrors.* - + 422 false @@ -4511,7 +4420,7 @@ - + ADMIN_ACTIONS @@ -4524,7 +4433,7 @@ false ^actionErrors.* - + 422 false @@ -4533,7 +4442,7 @@ - + ADMIN_ACTIONS @@ -4546,7 +4455,7 @@ false ^actionErrors.* - + 422 false @@ -4555,7 +4464,7 @@ - + SAMPLE_DATA @@ -4567,7 +4476,7 @@ false ^actionErrors.* - + 422 false @@ -4576,7 +4485,7 @@ - + ADMIN_ACTIONS @@ -4588,7 +4497,7 @@ false ^actionErrors.* - + 422 false @@ -4597,7 +4506,7 @@ - + ADMIN_ACTIONS @@ -4609,7 +4518,7 @@ false ^actionErrors.* - + 422 false @@ -4618,7 +4527,7 @@ - + INTEGRATIONS @@ -4631,7 +4540,7 @@ false ^actionErrors.* - + 422 false @@ -4640,7 +4549,7 @@ - + AKTO_ALERTS @@ -4656,7 +4565,7 @@ false ^actionErrors.* - + 422 false @@ -4670,7 +4579,7 @@ - + TEST_RESULTS @@ -4683,7 +4592,7 @@ false ^actionErrors.* - + 422 false @@ -4692,7 +4601,7 @@ - + TEST_RESULTS @@ -4704,7 +4613,7 @@ false ^actionErrors.* - + 422 false @@ -4712,7 +4621,7 @@ - + AKTO_ALERTS @@ -4728,7 +4637,7 @@ false ^actionErrors.* - + 422 false @@ -4742,7 +4651,7 @@ - + TEST_RESULTS @@ -4754,7 +4663,7 @@ false ^actionErrors.* - + 422 false @@ -4763,7 +4672,7 @@ - + TEST_RESULTS @@ -4776,7 +4685,7 @@ false ^actionErrors.* - + 422 false @@ -4785,7 +4694,7 @@ - + TEST_RESULTS @@ -4798,7 +4707,7 @@ false ^actionErrors.* - + 422 false @@ -4807,7 +4716,7 @@ - + INTEGRATIONS @@ -4819,7 +4728,7 @@ false ^actionErrors.* - + 422 false @@ -4828,7 +4737,7 @@ - + TEST_RESULTS @@ -4841,7 +4750,7 @@ false ^actionErrors.* - + 422 false @@ -4850,7 +4759,7 @@ - + INTEGRATIONS @@ -4863,7 +4772,7 @@ false ^actionErrors.* - + 422 false @@ -4872,7 +4781,7 @@ - + AKTO_ALERTS @@ -4888,7 +4797,7 @@ false ^actionErrors.* - + 422 false @@ -4903,9 +4812,9 @@ - + - + 422 false @@ -4914,7 +4823,7 @@ - + START_TEST_RUN @@ -4927,7 +4836,7 @@ false ^actionErrors.* - + 422 false @@ -4936,7 +4845,7 @@ - + START_TEST_RUN @@ -4949,7 +4858,7 @@ false ^actionErrors.* - + 422 false @@ -4958,7 +4867,7 @@ - + USER_ACTIONS @@ -4970,7 +4879,7 @@ false ^actionErrors.* - + 422 false @@ -4979,7 +4888,7 @@ - + INTEGRATIONS @@ -4991,7 +4900,7 @@ false ^actionErrors.* - + 422 false @@ -5000,7 +4909,7 @@ - + TEST_RESULTS @@ -5023,7 +4932,7 @@ - + ISSUES @@ -5045,7 +4954,7 @@ - + BILLING @@ -5068,7 +4977,7 @@ - + currentTestsStatus @@ -5082,7 +4991,7 @@ - + INTEGRATIONS @@ -5106,7 +5015,7 @@ - + INTEGRATIONS @@ -5118,7 +5027,7 @@ false ^actionErrors.* - + 422 false @@ -5126,7 +5035,7 @@ - + INTEGRATIONS @@ -5158,7 +5067,7 @@ - + INTEGRATIONS @@ -5181,7 +5090,7 @@ - + INTEGRATIONS @@ -5193,7 +5102,7 @@ false ^actionErrors.* - + 422 false @@ -5202,7 +5111,7 @@ - + INTEGRATIONS @@ -5218,7 +5127,7 @@ ADVANCED_TRAFFIC_CONNECTORS - + 422 false @@ -5232,7 +5141,7 @@ - + API_COLLECTIONS @@ -5255,7 +5164,7 @@ - + ADMIN_ACTIONS @@ -5268,7 +5177,7 @@ false ^actionErrors.* - + 422 false @@ -5277,7 +5186,7 @@ - + ADMIN_ACTIONS @@ -5300,7 +5209,7 @@ - + USER_CONFIG @@ -5313,7 +5222,7 @@ false ^actionErrors.* - + 422 false @@ -5322,7 +5231,7 @@ - + INTEGRATIONS @@ -5334,7 +5243,7 @@ false ^actionErrors.* - + 422 false @@ -5343,7 +5252,7 @@ - + INTEGRATIONS @@ -5355,7 +5264,7 @@ false ^actionErrors.* - + 422 false @@ -5364,7 +5273,7 @@ - + INTEGRATIONS @@ -5376,7 +5285,7 @@ false ^actionErrors.* - + 422 false @@ -5385,7 +5294,7 @@ - + INTEGRATIONS @@ -5397,7 +5306,7 @@ false ^actionErrors.* - + 422 false @@ -5406,7 +5315,7 @@ - + USER_CONFIG @@ -5427,7 +5336,7 @@ - + USER_CONFIG @@ -5450,7 +5359,7 @@ - + INTEGRATIONS @@ -5471,7 +5380,7 @@ - + ADMIN_ACTIONS @@ -5493,7 +5402,7 @@ - + ADMIN_ACTIONS @@ -5516,7 +5425,7 @@ - + ISSUES @@ -5538,7 +5447,7 @@ - + LOGS @@ -5561,7 +5470,7 @@ - + USER_ACTIONS @@ -5574,7 +5483,7 @@ false ^actionErrors.* - + 422 false @@ -5583,7 +5492,7 @@ - + TEST_EDITOR @@ -5606,7 +5515,7 @@ - + USER_ACTIONS @@ -5629,7 +5538,7 @@ - + ASK_GPT @@ -5645,7 +5554,7 @@ AKTO_GPT_AI - + 422 false @@ -5659,7 +5568,7 @@ - + ASK_GPT @@ -5671,7 +5580,7 @@ false ^actionErrors.* - + 422 false @@ -5680,7 +5589,7 @@ - + ASK_GPT @@ -5693,7 +5602,7 @@ false ^actionErrors.* - + 422 false @@ -5702,7 +5611,7 @@ - + METRICS @@ -5714,7 +5623,7 @@ false ^actionErrors.* - + 422 false @@ -5723,9 +5632,9 @@ - + - + 422 false @@ -5734,9 +5643,9 @@ - + - + 422 false @@ -5745,7 +5654,7 @@ - + METRICS @@ -5768,7 +5677,7 @@ - + INTEGRATIONS @@ -5784,7 +5693,7 @@ ACTIVE_ENDPOINTS - + 422 false @@ -5798,7 +5707,7 @@ - + INTEGRATIONS @@ -5814,7 +5723,7 @@ ACTIVE_ENDPOINTS - + 422 false @@ -5828,7 +5737,7 @@ - + INTEGRATIONS @@ -5843,7 +5752,7 @@ ACTIVE_ENDPOINTS - + 422 false @@ -5857,7 +5766,7 @@ - + INTEGRATIONS @@ -5873,7 +5782,7 @@ ACTIVE_ENDPOINTS - + 422 false @@ -5887,7 +5796,7 @@ - + INTEGRATIONS @@ -5910,7 +5819,7 @@ - + INTEGRATIONS @@ -5923,7 +5832,7 @@ false ^actionErrors.* - + 422 false @@ -5932,7 +5841,7 @@ - + TEST_EDITOR @@ -5945,7 +5854,7 @@ false ^actionErrors.* - + 422 false @@ -5954,7 +5863,7 @@ - + EXTERNAL_TEST_LIBRARY @@ -5967,7 +5876,7 @@ false ^actionErrors.* - + 422 false @@ -5976,7 +5885,7 @@ - + EXTERNAL_TEST_LIBRARY @@ -5989,7 +5898,7 @@ false ^actionErrors.* - + 422 false @@ -5998,7 +5907,7 @@ - + EXTERNAL_TEST_LIBRARY @@ -6011,7 +5920,7 @@ false ^actionErrors.* - + 422 false @@ -6020,7 +5929,7 @@ - + EXTERNAL_TEST_LIBRARY @@ -6043,7 +5952,7 @@ - + TEST_EDITOR @@ -6056,7 +5965,7 @@ false ^actionErrors.* - + 422 false @@ -6066,7 +5975,7 @@ - + ADMIN_ACTIONS @@ -6079,7 +5988,7 @@ false ^actionErrors.* - + 422 false @@ -6088,7 +5997,7 @@ - + ADMIN_ACTIONS @@ -6101,7 +6010,7 @@ false ^actionErrors.* - + 422 false @@ -6110,7 +6019,7 @@ - + ADMIN_ACTIONS @@ -6123,7 +6032,7 @@ false ^actionErrors.* - + 422 false @@ -6131,7 +6040,7 @@ - + ADMIN_ACTIONS @@ -6144,7 +6053,7 @@ false ^actionErrors.* - + 422 false @@ -6153,7 +6062,7 @@ - + TEST_EDITOR @@ -6166,7 +6075,7 @@ false ^actionErrors.* - + 422 false @@ -6175,9 +6084,9 @@ - + - + 422 false @@ -6186,9 +6095,9 @@ - + - + 422 false @@ -6200,16 +6109,16 @@ - + - + code - + USER_ACTIONS @@ -6221,7 +6130,7 @@ false ^actionErrors.* - + code @@ -6229,7 +6138,7 @@ - + ADMIN_ACTIONS @@ -6242,7 +6151,7 @@ false ^actionErrors.* - + 422 false @@ -6251,7 +6160,7 @@ - + API_COLLECTIONS @@ -6273,7 +6182,7 @@ - + @@ -6285,7 +6194,7 @@ - + INTEGRATIONS @@ -6317,7 +6226,7 @@ - + INTEGRATIONS @@ -6333,7 +6242,7 @@ JIRA_INTEGRATION - + 422 false @@ -6347,7 +6256,7 @@ - + INTEGRATIONS @@ -6359,7 +6268,7 @@ false ^actionErrors.* - + 422 false @@ -6368,7 +6277,7 @@ - + ISSUES @@ -6381,7 +6290,7 @@ false ^actionErrors.* - + 422 false @@ -6412,7 +6321,7 @@ - + ISSUES @@ -6425,7 +6334,7 @@ false ^actionErrors.* - + 422 false @@ -6433,7 +6342,7 @@ - + API_COLLECTIONS @@ -6445,16 +6354,29 @@ false ^actionErrors.* - - - 422 + + + 401 + + + + + + + + API_COLLECTIONS + READ + + + + 403 false ^actionErrors.* - + ISSUES @@ -6477,7 +6399,7 @@ - + USER_ACTIONS @@ -6499,7 +6421,7 @@ - + USER_ACTIONS @@ -6522,7 +6444,7 @@ - + INTEGRATIONS @@ -6544,7 +6466,7 @@ - + USER_ACTIONS @@ -6564,7 +6486,7 @@ - + BILLING @@ -6588,7 +6510,7 @@ - + BILLING @@ -6612,7 +6534,7 @@ - + BILLING @@ -6636,7 +6558,7 @@ - + BILLING @@ -6648,7 +6570,7 @@ false ^actionErrors.* - + 422 false @@ -6657,7 +6579,7 @@ - + BILLING @@ -6681,7 +6603,7 @@ - + BILLING @@ -6693,7 +6615,7 @@ false ^actionErrors.* - + 422 false @@ -6702,7 +6624,7 @@ - + DEFAULT_PAYLOADS @@ -6714,7 +6636,7 @@ false ^actionErrors.* - + 422 false @@ -6723,7 +6645,7 @@ - + DEFAULT_PAYLOADS @@ -6736,7 +6658,7 @@ false ^actionErrors.* - + 422 false @@ -6745,7 +6667,7 @@ - + API_COLLECTIONS @@ -6758,7 +6680,7 @@ false ^actionErrors.* - + 422 false @@ -6768,7 +6690,7 @@ - + API_COLLECTIONS @@ -6781,7 +6703,7 @@ false ^actionErrors.* - + 422 false @@ -6790,7 +6712,7 @@ - + API_COLLECTIONS @@ -6803,7 +6725,7 @@ false ^actionErrors.* - + 422 false @@ -6812,7 +6734,7 @@ - + API_COLLECTIONS @@ -6825,7 +6747,7 @@ false ^actionErrors.* - + 422 false @@ -6834,7 +6756,7 @@ - + API_COLLECTIONS @@ -6846,7 +6768,7 @@ false ^actionErrors.* - + 422 false @@ -6855,7 +6777,7 @@ - + USER_CONFIG @@ -6867,7 +6789,7 @@ false ^actionErrors.* - + 422 false @@ -6876,7 +6798,7 @@ - + USER_CONFIG @@ -6888,7 +6810,7 @@ false ^actionErrors.* - + 422 false @@ -6897,7 +6819,7 @@ - + USER_CONFIG @@ -6909,7 +6831,7 @@ false ^actionErrors.* - + 422 false @@ -6918,7 +6840,7 @@ - + API_COLLECTIONS @@ -6934,7 +6856,7 @@ CODE_ANALYSIS - + 422 false @@ -6948,9 +6870,9 @@ - + - + 422 false @@ -6959,7 +6881,7 @@ - + @@ -6971,7 +6893,7 @@ - + @@ -6983,7 +6905,7 @@ - + USER_ACTIONS @@ -7006,7 +6928,7 @@ - + TRAFFIC_FILTERS @@ -7029,7 +6951,7 @@ - + TRAFFIC_FILTERS @@ -7052,7 +6974,7 @@ - + CODE_ANALYSIS @@ -7082,7 +7004,7 @@ - + CODE_ANALYSIS @@ -7112,7 +7034,7 @@ - + CODE_ANALYSIS @@ -7141,7 +7063,7 @@ - + CODE_ANALYSIS @@ -7170,7 +7092,7 @@ - + 403 @@ -7187,7 +7109,7 @@ - + THREAT_DETECTION @@ -7216,6 +7138,45 @@ + + + + + 403 + false + ^actionErrors.* + + + + 422 + false + ^actionErrors.* + + + + + + + + 403 + false + ^actionErrors.* + + + + 422 + false + ^actionErrors.* + + + diff --git a/apps/dashboard/web/polaris_web/web/src/apps/dashboard/pages/threat_detection/api.js b/apps/dashboard/web/polaris_web/web/src/apps/dashboard/pages/threat_detection/api.js index d85a5597ab..d977b0e809 100644 --- a/apps/dashboard/web/polaris_web/web/src/apps/dashboard/pages/threat_detection/api.js +++ b/apps/dashboard/web/polaris_web/web/src/apps/dashboard/pages/threat_detection/api.js @@ -18,7 +18,7 @@ const threatDetectionRequests = { fetchSuspectSampleData(skip, ips, apiCollectionIds, urls, sort, startTimestamp, endTimestamp) { return request({ - url: '/api/fetchSuspectSampleData', + url: '/api/fetchSuspectSampleDataV2', method: 'post', data: { skip: skip, @@ -33,7 +33,7 @@ const threatDetectionRequests = { }, fetchFiltersThreatTable() { return request({ - url: '/api/fetchFiltersThreatTable', + url: '/api/fetchFiltersThreatTableV2', method: 'post', data: {} }) diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java index 462cbbfe7f..c025067079 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java @@ -1,6 +1,9 @@ package com.akto.threat.protection; import com.akto.threat.protection.interceptors.AuthenticationInterceptor; +import com.akto.threat.protection.service.DashboardService; +import com.akto.threat.protection.service.MaliciousAlertService; + import io.grpc.Server; import io.grpc.ServerBuilder; @@ -17,7 +20,8 @@ public BackendServer(int port, MongoClient mongoClient) { this.port = port; this.server = ServerBuilder.forPort(port) - .addService(new MaliciousAlertService(mongoClient)) + .addService(new MaliciousAlertService()) + .addService(new DashboardService(mongoClient)) .intercept(new AuthenticationInterceptor()) .build(); } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java index 60d319d27d..888118264f 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java @@ -1,116 +1,126 @@ package com.akto.threat.protection.db; import com.akto.dto.type.URLMethods.Method; - import java.util.UUID; public class MaliciousEventModel { - private String id; + private String id; + private String filterId; + private String actor; + private String ip; + private String url; + private String country; + private Method method; + private String orig; + private long requestTime; + + public MaliciousEventModel() {} + + private MaliciousEventModel(Builder builder) { + this.id = UUID.randomUUID().toString(); + this.filterId = builder.filterId; + this.actor = builder.actor; + this.ip = builder.ip; + this.country = builder.country; + this.method = builder.method; + this.orig = builder.orig; + this.requestTime = builder.requestTime; + this.url = builder.url; + } + + public static class Builder { private String filterId; private String actor; private String ip; + private String country; private String url; private Method method; - private String data; - private int binId; - private int expiry; + private String orig; + private long requestTime; - public MaliciousEventModel() { + public Builder setFilterId(String filterId) { + this.filterId = filterId; + return this; } - public MaliciousEventModel( - String filterId, - String actor, - String ip, - String url, - String method, - String data, - long requestTime) { - int now = (int) (System.currentTimeMillis() / 1000L); - this.id = UUID.randomUUID().toString(); - this.ip = ip; - this.filterId = filterId; - this.actor = actor; - this.data = data; - this.binId = (int) requestTime / 60; - this.url = url; - this.method = Method.fromString(method); - - // For now, we are hardcoding it to 3 hrs. - // But later we will read it through FilterConfig - this.expiry = now + (3 * 60 * 60); + public Builder setActor(String actor) { + this.actor = actor; + return this; } - public String getId() { - return id; + public Builder setIp(String ip) { + this.ip = ip; + return this; } - public String getFilterId() { - return filterId; + public Builder setCountry(String country) { + this.country = country; + return this; } - public String getActor() { - return actor; + public Builder setUrl(String url) { + this.url = url; + return this; } - public String getData() { - return data; + public Builder setMethod(Method method) { + this.method = method; + return this; } - public int getBinId() { - return binId; + public Builder setOrig(String orig) { + this.orig = orig; + return this; } - public int getExpiry() { - return expiry; + public Builder setRequestTime(long requestTime) { + this.requestTime = requestTime; + return this; } - public void setId(String id) { - this.id = id; + public MaliciousEventModel build() { + return new MaliciousEventModel(this); } + } - public void setFilterId(String filterId) { - this.filterId = filterId; - } + public String getId() { + return id; + } - public void setActor(String actor) { - this.actor = actor; - } + public String getFilterId() { + return filterId; + } - public void setData(String data) { - this.data = data; - } + public String getActor() { + return actor; + } - public void setBinId(int binId) { - this.binId = binId; - } + public String getIp() { + return ip; + } - public void setExpiry(int expiry) { - this.expiry = expiry; - } + public String getUrl() { + return url; + } - public String getUrl() { - return url; - } + public String getCountry() { + return country; + } - public void setUrl(String url) { - this.url = url; - } - - public Method getMethod() { - return method; - } + public Method getMethod() { + return method; + } - public void setMethod(Method method) { - this.method = method; - } + public String getOrig() { + return orig; + } - public String getIp() { - return ip; - } + public long getRequestTime() { + return requestTime; + } - public void setIp(String ip) { - this.ip = ip; - } + public static Builder newBuilder() { + return new Builder(); + } } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java new file mode 100644 index 0000000000..37671b844b --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java @@ -0,0 +1,104 @@ +package com.akto.threat.protection.service; + +import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceImplBase; +import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest; +import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse; +import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; +import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse; +import com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest; +import com.akto.threat.protection.db.MaliciousEventModel; +import com.akto.threat.protection.interceptors.Constants; +import com.mongodb.BasicDBObject; +import com.mongodb.client.DistinctIterable; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.model.Filters; +import io.grpc.stub.StreamObserver; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.bson.conversions.Bson; + +public class DashboardService extends DashboardServiceImplBase { + private final MongoClient mongoClient; + + public DashboardService(MongoClient mongoClient) { + this.mongoClient = mongoClient; + } + + private static Set findDistinctFields( + MongoCollection coll, String fieldName, Class tClass, Bson filters) { + DistinctIterable r = coll.distinct(fieldName, filters, tClass); + Set result = new HashSet<>(); + MongoCursor cursor = r.cursor(); + while (cursor.hasNext()) { + result.add(cursor.next()); + } + return result; + } + + @Override + public void fetchAlertFilters( + FetchAlertFiltersRequest request, + StreamObserver responseObserver) { + int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); + MongoCollection coll = + this.mongoClient + .getDatabase(accountId + "") + .getCollection("malicious_events", MaliciousEventModel.class); + + Set actors = + DashboardService.findDistinctFields(coll, "actor", String.class, Filters.empty()); + Set urls = + DashboardService.findDistinctFields(coll, "url", String.class, Filters.empty()); + + FetchAlertFiltersResponse response = + FetchAlertFiltersResponse.newBuilder().addAllActors(actors).addAllUrls(urls).build(); + responseObserver.onNext(response); + responseObserver.onCompleted(); + } + + @Override + public void listMaliciousRequests( + ListMaliciousRequestsRequest request, + StreamObserver responseObserver) { + int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); + + int page = request.hasPage() ? request.getPage() : 1; + int limit = request.getLimit(); + int skip = (page - 1) * limit; + + MongoCollection coll = + this.mongoClient + .getDatabase(accountId + "") + .getCollection("malicious_events", MaliciousEventModel.class); + + BasicDBObject query = new BasicDBObject(); + try (MongoCursor cursor = + coll.find(query).skip(skip).limit(limit).cursor()) { + List alerts = new ArrayList<>(); + while (cursor.hasNext()) { + MaliciousEventModel evt = cursor.next(); + alerts.add( + MaliciousRequest.newBuilder() + .setActor(evt.getActor()) + .setFilterId(evt.getFilterId()) + .setFilterId(evt.getFilterId()) + .setId(evt.getId()) + .setIp(evt.getIp()) + .setCountry(evt.getCountry()) + .setOrig(evt.getOrig()) + .setUrl(evt.getUrl()) + .setMethod(evt.getMethod().name()) + .setTimestamp(evt.getRequestTime()) + .build()); + } + ListMaliciousRequestsResponse response = + ListMaliciousRequestsResponse.newBuilder().setPage(page).setTotal(alerts.size()).build(); + responseObserver.onNext(response); + responseObserver.onCompleted(); + } + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousAlertService.java similarity index 64% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java rename to apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousAlertService.java index c4664b92e7..1145253732 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/MaliciousAlertService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousAlertService.java @@ -1,9 +1,9 @@ -package com.akto.threat.protection; +package com.akto.threat.protection.service; import java.util.ArrayList; import java.util.List; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.dto.type.URLMethods; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousAlertServiceGrpc.MaliciousAlertServiceImplBase; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse; @@ -12,21 +12,12 @@ import com.akto.threat.protection.db.SmartEventModel; import com.akto.threat.protection.interceptors.Constants; import com.akto.threat.protection.utils.KafkaUtils; -import com.mongodb.client.MongoClient; -import com.mongodb.client.model.BulkWriteOptions; -import com.mongodb.client.model.InsertOneModel; import com.mongodb.client.model.WriteModel; - import io.grpc.stub.StreamObserver; -import javassist.tools.rmi.Sample; public class MaliciousAlertService extends MaliciousAlertServiceImplBase { - private final MongoClient mongoClient; - - public MaliciousAlertService(MongoClient mongoClient) { - this.mongoClient = mongoClient; - } + public MaliciousAlertService() {} @Override public void recordAlert( @@ -40,21 +31,25 @@ public void recordAlert( List maliciousEvents = request.getSampleDataList(); List events = new ArrayList<>(); for (SampleMaliciousEvent maliciousEvent : maliciousEvents) { - events.add(new MaliciousEventModel(filterId, - actor, - maliciousEvent.getIp(), - maliciousEvent.getUrl(), - maliciousEvent.getMethod(), - maliciousEvent.getPayload(), - maliciousEvent.getTimestamp())); + events.add( + MaliciousEventModel.newBuilder() + .setActor(actor) + .setIp(maliciousEvent.getIp()) + .setUrl(maliciousEvent.getUrl()) + .setMethod(URLMethods.Method.fromString(maliciousEvent.getMethod())) + .setOrig(maliciousEvent.getPayload()) + .setRequestTime(maliciousEvent.getTimestamp()) + .setFilterId(filterId) + .build()); } KafkaUtils.insertData(events, "maliciousEvents", accountId); - KafkaUtils.insertData(new SmartEventModel(filterId, actor, request.getTotalEvents(), request.getDetectedAt()), "smartEvent", accountId); - + KafkaUtils.insertData( + new SmartEventModel(filterId, actor, request.getTotalEvents(), request.getDetectedAt()), + "smartEvent", + accountId); responseObserver.onNext(RecordAlertResponse.newBuilder().build()); responseObserver.onCompleted(); } - } diff --git a/protobuf/threat_protection/service/dashboard_service/v1/service.proto b/protobuf/threat_protection/service/dashboard_service/v1/service.proto new file mode 100644 index 0000000000..ecb286836b --- /dev/null +++ b/protobuf/threat_protection/service/dashboard_service/v1/service.proto @@ -0,0 +1,43 @@ +syntax = "proto3"; + +package threat_protection.service.dashboard_service.v1; + +// Dashboard service which the dashboard actions will call instead of directly calling DB +option java_outer_classname = "DashboardServiceProto"; +option java_package = "threat_protection.service.dashboard_service.v1"; + +message MaliciousRequest { + string id = 1; + string actor = 2; + string filter_id = 3; + string url = 4; + string method = 5; + string orig = 6; + string ip = 7; + string country = 8; + int64 timestamp = 9; +} + +message ListMaliciousRequestsResponse { + repeated MaliciousRequest malicious_requests = 1; + int32 total = 2; + int32 page = 3; +} + +message ListMaliciousRequestsRequest { + // The number of alerts to return + int32 limit = 3; + optional int32 page = 4; +} + +message FetchAlertFiltersRequest {} + +message FetchAlertFiltersResponse { + repeated string actors = 1; + repeated string urls = 2; +} + +service DashboardService { + rpc ListMaliciousRequests(ListMaliciousRequestsRequest) returns (ListMaliciousRequestsResponse); + rpc FetchAlertFilters(FetchAlertFiltersRequest) returns (FetchAlertFiltersResponse) {} +} \ No newline at end of file From 74bafd6a30a29a95626c06250d280a1abc71eb91 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 10 Dec 2024 14:57:10 +0530 Subject: [PATCH 43/73] updated proto definition to include malicious event type (SINGLE or AGGREGATED) --- .../java/com/akto/threat/detection/Main.java | 68 +++-- .../db/entity/MaliciousEventEntity.java | 284 +++++++++--------- .../WindowBasedThresholdNotifier.java | 4 +- .../detection/tasks/FlushSampleDataTask.java | 8 +- .../tasks/MaliciousTrafficDetectorTask.java | 80 +++-- .../detection/tasks/SendAlertsToBackend.java | 144 --------- .../tasks/SendMaliciousRequestsToBackend.java | 138 +++++++++ .../akto/threat/protection/BackendServer.java | 4 +- .../AggregateSampleMaliciousEventModel.java | 138 +++++++++ .../protection/db/MaliciousEventModel.java | 90 ++++-- .../threat/protection/db/SmartEventModel.java | 66 ---- .../protection/service/DashboardService.java | 19 +- .../service/MaliciousAlertService.java | 55 ---- .../service/MaliciousEventService.java | 77 +++++ .../threat/protection/utils/KafkaUtils.java | 283 ++++++++--------- .../message/malicious_event/v1/message.proto | 21 +- .../message/sample_request/v1/message.proto | 16 + .../message/smart_event/v1/message.proto | 13 - .../malicious_alert_service/v1/service.proto | 27 +- 19 files changed, 857 insertions(+), 678 deletions(-) delete mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/AggregateSampleMaliciousEventModel.java delete mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java delete mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousAlertService.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java create mode 100644 protobuf/threat_protection/message/sample_request/v1/message.proto delete mode 100644 protobuf/threat_protection/message/smart_event/v1/message.proto diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index d49b27672d..bb676ca9b1 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -9,7 +9,7 @@ import com.akto.threat.detection.tasks.CleanupTask; import com.akto.threat.detection.tasks.FlushSampleDataTask; import com.akto.threat.detection.tasks.MaliciousTrafficDetectorTask; -import com.akto.threat.detection.tasks.SendAlertsToBackend; +import com.akto.threat.detection.tasks.SendMaliciousRequestsToBackend; import com.mongodb.ConnectionString; import io.lettuce.core.RedisClient; @@ -26,33 +26,39 @@ public static void main(String[] args) throws Exception { SessionFactory sessionFactory = SessionFactoryUtils.createFactory(); DaoInit.init(new ConnectionString(System.getenv("AKTO_MONGO_CONN"))); - KafkaConfig trafficKafka = KafkaConfig.newBuilder() - .setGroupId(CONSUMER_GROUP_ID) - .setBootstrapServers(System.getenv("AKTO_TRAFFIC_KAFKA_BOOTSTRAP_SERVER")) - .setConsumerConfig( - KafkaConsumerConfig.newBuilder() - .setMaxPollRecords(100) - .setPollDurationMilli(100) - .build()) - .setProducerConfig( - KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) - .build(); + KafkaConfig trafficKafka = + KafkaConfig.newBuilder() + .setGroupId(CONSUMER_GROUP_ID) + .setBootstrapServers(System.getenv("AKTO_TRAFFIC_KAFKA_BOOTSTRAP_SERVER")) + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) + .build(); - KafkaConfig internalKafka = KafkaConfig.newBuilder() - .setGroupId(CONSUMER_GROUP_ID) - .setBootstrapServers(System.getenv("AKTO_INTERNAL_KAFKA_BOOTSTRAP_SERVER")) - .setConsumerConfig( - KafkaConsumerConfig.newBuilder() - .setMaxPollRecords(100) - .setPollDurationMilli(100) - .build()) - .setProducerConfig( - KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) - .build(); + KafkaConfig internalKafka = + KafkaConfig.newBuilder() + .setGroupId(CONSUMER_GROUP_ID) + .setBootstrapServers(System.getenv("AKTO_INTERNAL_KAFKA_BOOTSTRAP_SERVER")) + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(100).build()) + .build(); new MaliciousTrafficDetectorTask(trafficKafka, internalKafka, createRedisClient()).run(); - new FlushSampleDataTask(sessionFactory, internalKafka, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS).run(); - new SendAlertsToBackend(sessionFactory, internalKafka, KafkaTopic.ThreatDetection.ALERTS).run(); + new FlushSampleDataTask( + sessionFactory, internalKafka, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS) + .run(); + new SendMaliciousRequestsToBackend( + sessionFactory, internalKafka, KafkaTopic.ThreatDetection.ALERTS) + .run(); new CleanupTask(sessionFactory).run(); } @@ -64,12 +70,12 @@ public static void runMigrations() { String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); String user = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_USER"); String password = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_PASSWORD"); - Flyway flyway = Flyway - .configure() - .dataSource(url, user, password) - .locations("classpath:db/migration") - .schemas("flyway") - .load(); + Flyway flyway = + Flyway.configure() + .dataSource(url, user, password) + .locations("classpath:db/migration") + .schemas("flyway") + .load(); flyway.migrate(); } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java index aff882819c..6ec18ea559 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java @@ -21,169 +21,185 @@ @Table(name = "malicious_event", schema = "threat_detection") public class MaliciousEventEntity { - @Id - @GeneratedValue(generator = "UUID") - @GenericGenerator(name = "UUID", strategy = "org.hibernate.id.UUIDGenerator") - private UUID id; + @Id + @GeneratedValue(generator = "UUID") + @GenericGenerator(name = "UUID", strategy = "org.hibernate.id.UUIDGenerator") + private UUID id; - @Column(name = "actor") - private String actor; + @Column(name = "actor") + private String actor; - @Column(name = "filter_id") - private String filterId; + @Column(name = "filter_id") + private String filterId; - @Column(name = "url") - private String url; + @Column(name = "url") + private String url; - @Column(name = "method") - @Enumerated(EnumType.STRING) - private URLMethods.Method method; + @Column(name = "method") + @Enumerated(EnumType.STRING) + private URLMethods.Method method; - @Column(name = "timestamp") - private long timestamp; + @Column(name = "timestamp") + private long timestamp; - @Column(name = "orig") - private String orig; + @Column(name = "orig") + private String orig; - // Geo location data - @Column(name = "ip") - private String ip; + // Geo location data + @Column(name = "ip") + private String ip; - @Column(name = "created_at", updatable = false) - private LocalDateTime createdAt; + @Column(name = "api_collection_id") + private int apiCollectionId; - public MaliciousEventEntity() { - } + @Column(name = "created_at", updatable = false) + private LocalDateTime createdAt; - @PrePersist - protected void onCreate() { - this.createdAt = LocalDateTime.now(ZoneOffset.UTC); - } + public MaliciousEventEntity() {} - public MaliciousEventEntity(Builder builder) { - this.actor = builder.actorId; - this.filterId = builder.filterId; - this.url = builder.url; - this.method = builder.method; - this.timestamp = builder.timestamp; - this.orig = builder.orig; - this.ip = builder.ip; - } + @PrePersist + protected void onCreate() { + this.createdAt = LocalDateTime.now(ZoneOffset.UTC); + } - public static class Builder { - private String actorId; - private String filterId; - private String url; - private URLMethods.Method method; - private long timestamp; - private String orig; - private String ip; - - public Builder setActor(String actorId) { - this.actorId = actorId; - return this; - } - - public Builder setFilterId(String filterId) { - this.filterId = filterId; - return this; - } - - public Builder setUrl(String url) { - this.url = url; - return this; - } - - public Builder setMethod(URLMethods.Method method) { - this.method = method; - return this; - } - - public Builder setTimestamp(long timestamp) { - this.timestamp = timestamp; - return this; - } - - public Builder setOrig(String orig) { - this.orig = orig; - return this; - } - - public Builder setIp(String ip) { - this.ip = ip; - return this; - } - - public MaliciousEventEntity build() { - return new MaliciousEventEntity(this); - } - } + public MaliciousEventEntity(Builder builder) { + this.actor = builder.actorId; + this.filterId = builder.filterId; + this.url = builder.url; + this.method = builder.method; + this.timestamp = builder.timestamp; + this.orig = builder.orig; + this.ip = builder.ip; + this.apiCollectionId = builder.apiCollectionId; + } - public static Builder newBuilder() { - return new Builder(); - } - - public UUID getId() { - return id; - } + public static class Builder { + private String actorId; + private String filterId; + private String url; + private URLMethods.Method method; + private long timestamp; + private String orig; + private String ip; + private int apiCollectionId; - public String getActor() { - return actor; + public Builder setActor(String actorId) { + this.actorId = actorId; + return this; } - public String getFilterId() { - return filterId; + public Builder setFilterId(String filterId) { + this.filterId = filterId; + return this; } - public String getUrl() { - return url; + public Builder setUrl(String url) { + this.url = url; + return this; } - public URLMethods.Method getMethod() { - return method; + public Builder setMethod(URLMethods.Method method) { + this.method = method; + return this; } - public long getTimestamp() { - return timestamp; + public Builder setTimestamp(long timestamp) { + this.timestamp = timestamp; + return this; } - public String getOrig() { - return orig; + public Builder setOrig(String orig) { + this.orig = orig; + return this; } - public String getIp() { - return ip; + public Builder setIp(String ip) { + this.ip = ip; + return this; } - public LocalDateTime getCreatedAt() { - return createdAt; + public Builder setApiCollectionId(int apiCollectionId) { + this.apiCollectionId = apiCollectionId; + return this; } - @Override - public String toString() { - return "MaliciousEventModel{" - + "id='" - + id - + '\'' - + ", actorId='" - + actor - + '\'' - + ", filterId='" - + filterId - + '\'' - + ", url='" - + url - + '\'' - + ", method=" - + method - + ", timestamp=" - + timestamp - + ", orig='" - + orig - + '\'' - + ", ip='" - + ip - + '\'' - + '}'; + public MaliciousEventEntity build() { + return new MaliciousEventEntity(this); } + } + + public static Builder newBuilder() { + return new Builder(); + } + + public UUID getId() { + return id; + } + + public String getActor() { + return actor; + } + + public String getFilterId() { + return filterId; + } + + public String getUrl() { + return url; + } + + public URLMethods.Method getMethod() { + return method; + } + + public long getTimestamp() { + return timestamp; + } + + public String getOrig() { + return orig; + } + + public String getIp() { + return ip; + } + + public int getApiCollectionId() { + return apiCollectionId; + } + + public LocalDateTime getCreatedAt() { + return createdAt; + } + + @Override + public String toString() { + return "MaliciousEventEntity{" + + "createdAt=" + + createdAt + + ", apiCollectionId=" + + apiCollectionId + + ", ip='" + + ip + + '\'' + + ", orig='" + + orig + + '\'' + + ", timestamp=" + + timestamp + + ", method=" + + method + + ", url='" + + url + + '\'' + + ", filterId='" + + filterId + + '\'' + + ", actor='" + + actor + + '\'' + + ", id=" + + id + + '}'; + } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java index 218e5ac165..24c87f9b4c 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java @@ -1,6 +1,6 @@ package com.akto.threat.detection.smart_event_detector.window_based; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.threat.detection.cache.CounterCache; import com.akto.dto.api_protection_parse_layer.Rule; import java.util.ArrayList; @@ -66,7 +66,7 @@ public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.notifiedMap = new ConcurrentHashMap<>(); } - public Result shouldNotify(String aggKey, MaliciousEvent maliciousEvent, Rule rule) { + public Result shouldNotify(String aggKey, SampleMaliciousRequest maliciousEvent, Rule rule) { int binId = (int) maliciousEvent.getTimestamp() / 60; String cacheKey = aggKey + "|" + binId; this.cache.increment(cacheKey); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 974885e379..7fad234199 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -2,6 +2,7 @@ import com.akto.dto.type.URLMethods; import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; @@ -24,7 +25,8 @@ public class FlushSampleDataTask extends AbstractKafkaConsumerTask { private final SessionFactory sessionFactory; - public FlushSampleDataTask(SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { + public FlushSampleDataTask( + SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { super(trafficConfig, topic); this.sessionFactory = sessionFactory; } @@ -39,7 +41,7 @@ protected void processRecords(ConsumerRecords records) { records.forEach( r -> { String message = r.value(); - MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); + SampleMaliciousRequest.Builder builder = SampleMaliciousRequest.newBuilder(); MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); if (m == null) { return; @@ -52,7 +54,7 @@ protected void processRecords(ConsumerRecords records) { return; } - MaliciousEvent evt = builder.build(); + SampleMaliciousRequest evt = builder.build(); events.add( MaliciousEventEntity.newBuilder() diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index a9f1702b02..1cb8468c5f 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -2,7 +2,7 @@ import com.akto.dao.context.Context; import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; -import com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent; +import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.threat.detection.actor.SourceIPActorGenerator; import com.akto.threat.detection.cache.RedisBackedCounterCache; import com.akto.threat.detection.config.kafka.KafkaConfig; @@ -172,6 +172,8 @@ private void processRecord(ConsumerRecord record) { AggregationRules aggRules = new AggregationRules(); aggRules.setRule(rules); + boolean isAggFilter = aggRules != null && !aggRules.getRule().isEmpty(); + SourceIPActorGenerator.instance .generate(responseParam) .ifPresent( @@ -179,49 +181,47 @@ private void processRecord(ConsumerRecord record) { String groupKey = apiFilter.getId(); String aggKey = actor + "|" + groupKey; - MaliciousEvent maliciousEvent = - MaliciousEvent.newBuilder() - .setActor(actor) - .setFilterId(apiFilter.getId()) + SampleMaliciousRequest maliciousReq = + SampleMaliciousRequest.newBuilder() .setUrl(responseParam.getRequestParams().getURL()) .setMethod(responseParam.getRequestParams().getMethod()) .setPayload(responseParam.getOrig()) .setIp(actor) // For now using actor as IP .setApiCollectionId(responseParam.getRequestParams().getApiCollectionId()) .setTimestamp(responseParam.getTime()) + .setFilterId(apiFilter.getId()) .build(); try { maliciousMessages.add( MessageEnvelope.generateEnvelope( - responseParam.getAccountId(), maliciousEvent)); + responseParam.getAccountId(), maliciousReq)); } catch (InvalidProtocolBufferException e) { return; } + if (!isAggFilter) { + generateAndPushMaliciousEventRequest( + apiFilter, + actor, + responseParam, + maliciousReq, + MaliciousEvent.EventType.EVENT_TYPE_SINGLE); + return; + } + + // Aggregation rules for (Rule rule : aggRules.getRule()) { WindowBasedThresholdNotifier.Result result = - this.windowBasedThresholdNotifier.shouldNotify( - aggKey, maliciousEvent, rule); + this.windowBasedThresholdNotifier.shouldNotify(aggKey, maliciousReq, rule); if (result.shouldNotify()) { - SmartEvent smartEvent = - SmartEvent.newBuilder() - .setFilterId(apiFilter.getId()) - .setActor(actor) - .setDetectedAt(responseParam.getTime()) - .setRuleId(rule.getName()) - .build(); - try { - MessageEnvelope.generateEnvelope(responseParam.getAccountId(), smartEvent) - .marshal() - .ifPresent( - data -> { - internalKafka.send(data, KafkaTopic.ThreatDetection.ALERTS); - }); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); - } + generateAndPushMaliciousEventRequest( + apiFilter, + actor, + responseParam, + maliciousReq, + MaliciousEvent.EventType.EVENT_TYPE_AGGREGATED); } } }); @@ -244,4 +244,34 @@ private void processRecord(ConsumerRecord record) { e.printStackTrace(); } } + + private void generateAndPushMaliciousEventRequest( + FilterConfig apiFilter, + String actor, + HttpResponseParams responseParam, + SampleMaliciousRequest maliciousReq, + MaliciousEvent.EventType eventType) { + MaliciousEvent maliciousEvent = + MaliciousEvent.newBuilder() + .setFilterId(apiFilter.getId()) + .setActor(actor) + .setDetectedAt(responseParam.getTime()) + .setEventType(eventType) + .setLatestApiCollectionId(maliciousReq.getApiCollectionId()) + .setLatestApiIp(maliciousReq.getIp()) + .setLatestApiPayload(maliciousReq.getPayload()) + .setLatestApiMethod(maliciousReq.getMethod()) + .setDetectedAt(responseParam.getTime()) + .build(); + try { + MessageEnvelope.generateEnvelope(responseParam.getAccountId(), maliciousEvent) + .marshal() + .ifPresent( + data -> { + internalKafka.send(data, KafkaTopic.ThreatDetection.ALERTS); + }); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } + } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java deleted file mode 100644 index cb1c3fe0a6..0000000000 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendAlertsToBackend.java +++ /dev/null @@ -1,144 +0,0 @@ -package com.akto.threat.detection.tasks; - -import com.akto.proto.threat_protection.message.smart_event.v1.SmartEvent; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousAlertServiceGrpc; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousAlertServiceGrpc.MaliciousAlertServiceStub; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent; -import com.akto.threat.detection.config.kafka.KafkaConfig; -import com.akto.threat.detection.db.entity.MaliciousEventEntity; -import com.akto.threat.detection.dto.MessageEnvelope; -import com.akto.threat.detection.grpc.AuthToken; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.util.JsonFormat; -import io.grpc.Grpc; -import io.grpc.InsecureChannelCredentials; -import io.grpc.ManagedChannel; -import io.grpc.stub.StreamObserver; - -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.hibernate.Session; -import org.hibernate.SessionFactory; -import org.hibernate.Transaction; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.stream.Collectors; - -/* -This will send alerts to threat detection backend - */ -public class SendAlertsToBackend extends AbstractKafkaConsumerTask { - - private final SessionFactory sessionFactory; - - private final MaliciousAlertServiceStub consumerServiceStub; - - public SendAlertsToBackend(SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { - super(trafficConfig, topic); - this.sessionFactory = sessionFactory; - - String target = "localhost:8980"; - ManagedChannel channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - this.consumerServiceStub = MaliciousAlertServiceGrpc.newStub(channel) - .withCallCredentials( - new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); - } - - ExecutorService getPollingExecutor() { - return Executors.newSingleThreadExecutor(); - } - - private List getSampleMaliciousEvents(String actor, String filterId) { - Session session = this.sessionFactory.openSession(); - Transaction txn = session.beginTransaction(); - try { - return session - .createQuery("from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId order by m.createdAt desc", MaliciousEventEntity.class) - .setParameter("actor", actor).setParameter("filterId", filterId) - .setMaxResults(50) - .getResultList(); - } finally { - txn.commit(); - session.close(); - } - } - - private long getTotalEvents(String actor, String filterId) { - Session session = this.sessionFactory.openSession(); - Transaction txn = session.beginTransaction(); - try { - return session - .createQuery("select count(m) from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId", Long.class) - .setParameter("actor", actor).setParameter("filterId", filterId).uniqueResult(); - } finally { - txn.commit(); - session.close(); - } - } - - protected void processRecords(ConsumerRecords records) { - records.forEach( - r -> { - String message = r.value(); - SmartEvent.Builder builder = SmartEvent.newBuilder(); - MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); - if (m == null) { - return; - } - - try { - JsonFormat.parser().merge(m.getData(), builder); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); - return; - } - - SmartEvent evt = builder.build(); - - // Get sample data from postgres for this alert - try { - List sampleData = this.getSampleMaliciousEvents(evt.getActor(), evt.getFilterId()); - - long totalEvents = this.getTotalEvents(evt.getActor(), evt.getFilterId()); - - this.consumerServiceStub.recordAlert( - RecordAlertRequest.newBuilder() - .setActor(evt.getActor()) - .setFilterId(evt.getFilterId()) - .setTotalEvents(totalEvents) - .addAllSampleData( - sampleData.stream() - .map( - d -> SampleMaliciousEvent.newBuilder() - .setUrl(d.getUrl()) - .setMethod(d.getMethod().name()) - .setTimestamp(d.getTimestamp()) - .setPayload(d.getOrig()) - .setIp(d.getIp()) - .build()) - .collect(Collectors.toList())) - .build(), - new StreamObserver() { - @Override - public void onNext(RecordAlertResponse value) { - // Do nothing - } - - @Override - public void onError(Throwable t) { - t.printStackTrace(); - } - - @Override - public void onCompleted() { - // Do nothing - } - }); - } catch (Exception e) { - e.printStackTrace(); - } - }); - } -} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java new file mode 100644 index 0000000000..cf1a74a19f --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java @@ -0,0 +1,138 @@ +package com.akto.threat.detection.tasks; + +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; +import com.akto.threat.detection.config.kafka.KafkaConfig; +import com.akto.threat.detection.db.entity.MaliciousEventEntity; +import com.akto.threat.detection.dto.MessageEnvelope; +import com.akto.threat.detection.grpc.AuthToken; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.util.JsonFormat; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import io.grpc.stub.StreamObserver; + +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.hibernate.Session; +import org.hibernate.SessionFactory; +import org.hibernate.Transaction; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.stream.Collectors; + +/* +This will send alerts to threat detection backend + */ +public class SendMaliciousRequestsToBackend extends AbstractKafkaConsumerTask { + + private final SessionFactory sessionFactory; + + private final MaliciousEventServiceGrpc.MaliciousEventServiceStub consumerServiceStub; + + public SendMaliciousRequestsToBackend( + SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { + super(trafficConfig, topic); + this.sessionFactory = sessionFactory; + + String target = "localhost:8980"; + ManagedChannel channel = + Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); + this.consumerServiceStub = + MaliciousEventServiceGrpc.newStub(channel) + .withCallCredentials( + new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); + } + + ExecutorService getPollingExecutor() { + return Executors.newSingleThreadExecutor(); + } + + private List getSampleMaliciousRequests(String actor, String filterId) { + Session session = this.sessionFactory.openSession(); + Transaction txn = session.beginTransaction(); + try { + return session + .createQuery( + "from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId order by m.createdAt desc", + MaliciousEventEntity.class) + .setParameter("actor", actor) + .setParameter("filterId", filterId) + .setMaxResults(50) + .getResultList(); + } finally { + txn.commit(); + session.close(); + } + } + + protected void processRecords(ConsumerRecords records) { + records.forEach( + r -> { + String message = r.value(); + MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); + MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); + if (m == null) { + return; + } + + try { + JsonFormat.parser().merge(m.getData(), builder); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + return; + } + + MaliciousEvent evt = builder.build(); + + // Get sample data from postgres for this alert + try { + RecordMaliciousEventRequest.Builder reqBuilder = + RecordMaliciousEventRequest.newBuilder().setMaliciousEvent(evt); + if (MaliciousEvent.EventType.EVENT_TYPE_AGGREGATED.equals(evt.getEventType())) { + List sampleData = + this.getSampleMaliciousRequests(evt.getActor(), evt.getFilterId()); + + reqBuilder.addAllSampleRequests( + sampleData.stream() + .map( + d -> + SampleMaliciousRequest.newBuilder() + .setUrl(d.getUrl()) + .setMethod(d.getMethod().name()) + .setTimestamp(d.getTimestamp()) + .setPayload(d.getOrig()) + .setIp(d.getIp()) + .setApiCollectionId(d.getApiCollectionId()) + .build()) + .collect(Collectors.toList())); + } + + this.consumerServiceStub.recordMaliciousEvent( + reqBuilder.build(), + new StreamObserver() { + @Override + public void onNext(RecordMaliciousEventResponse value) { + // Do nothing + } + + @Override + public void onError(Throwable t) { + t.printStackTrace(); + } + + @Override + public void onCompleted() { + // Do nothing + } + }); + } catch (Exception e) { + e.printStackTrace(); + } + }); + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java index c025067079..ae2138d715 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java @@ -2,7 +2,7 @@ import com.akto.threat.protection.interceptors.AuthenticationInterceptor; import com.akto.threat.protection.service.DashboardService; -import com.akto.threat.protection.service.MaliciousAlertService; +import com.akto.threat.protection.service.MaliciousEventService; import io.grpc.Server; import io.grpc.ServerBuilder; @@ -20,7 +20,7 @@ public BackendServer(int port, MongoClient mongoClient) { this.port = port; this.server = ServerBuilder.forPort(port) - .addService(new MaliciousAlertService()) + .addService(new MaliciousEventService()) .addService(new DashboardService(mongoClient)) .intercept(new AuthenticationInterceptor()) .build(); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/AggregateSampleMaliciousEventModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/AggregateSampleMaliciousEventModel.java new file mode 100644 index 0000000000..bbc8338bc2 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/AggregateSampleMaliciousEventModel.java @@ -0,0 +1,138 @@ +package com.akto.threat.protection.db; + +import com.akto.dto.type.URLMethods.Method; +import java.util.UUID; + +public class AggregateSampleMaliciousEventModel { + + private String id; + private String filterId; + private String actor; + private String ip; + private String url; + private String country; + private Method method; + private String orig; + private int apiCollectionId; + private long requestTime; + + public AggregateSampleMaliciousEventModel() {} + + private AggregateSampleMaliciousEventModel(Builder builder) { + this.id = UUID.randomUUID().toString(); + this.filterId = builder.filterId; + this.actor = builder.actor; + this.ip = builder.ip; + this.country = builder.country; + this.method = builder.method; + this.orig = builder.orig; + this.requestTime = builder.requestTime; + this.url = builder.url; + this.apiCollectionId = builder.apiCollectionId; + } + + public static class Builder { + public int apiCollectionId; + private String filterId; + private String actor; + private String ip; + private String country; + private String url; + private Method method; + private String orig; + private long requestTime; + + public Builder setFilterId(String filterId) { + this.filterId = filterId; + return this; + } + + public Builder setActor(String actor) { + this.actor = actor; + return this; + } + + public Builder setIp(String ip) { + this.ip = ip; + return this; + } + + public Builder setCountry(String country) { + this.country = country; + return this; + } + + public Builder setUrl(String url) { + this.url = url; + return this; + } + + public Builder setMethod(Method method) { + this.method = method; + return this; + } + + public Builder setOrig(String orig) { + this.orig = orig; + return this; + } + + public Builder setRequestTime(long requestTime) { + this.requestTime = requestTime; + return this; + } + + public Builder setApiCollectionId(int apiCollectionId) { + this.apiCollectionId = apiCollectionId; + return this; + } + + public AggregateSampleMaliciousEventModel build() { + return new AggregateSampleMaliciousEventModel(this); + } + } + + public String getId() { + return id; + } + + public String getFilterId() { + return filterId; + } + + public String getActor() { + return actor; + } + + public String getIp() { + return ip; + } + + public String getUrl() { + return url; + } + + public String getCountry() { + return country; + } + + public Method getMethod() { + return method; + } + + public String getOrig() { + return orig; + } + + public long getRequestTime() { + return requestTime; + } + + public int getApiCollectionId() { + return apiCollectionId; + } + + public static Builder newBuilder() { + return new Builder(); + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java index 888118264f..6d71db936d 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java @@ -1,6 +1,7 @@ package com.akto.threat.protection.db; -import com.akto.dto.type.URLMethods.Method; +import com.akto.dto.type.URLMethods; + import java.util.UUID; public class MaliciousEventModel { @@ -9,35 +10,46 @@ public class MaliciousEventModel { private String filterId; private String actor; private String ip; - private String url; + private String latestApiEndpoint; private String country; - private Method method; - private String orig; - private long requestTime; + private URLMethods.Method latestApiMethod; + private String latestApiOrig; + private long detectedAt; + private int latestApiCollectionId; + private EventType eventType; + + public enum EventType { + SINGLE, + AGGREGATED + } public MaliciousEventModel() {} - private MaliciousEventModel(Builder builder) { + public MaliciousEventModel(Builder builder) { this.id = UUID.randomUUID().toString(); this.filterId = builder.filterId; this.actor = builder.actor; this.ip = builder.ip; this.country = builder.country; - this.method = builder.method; - this.orig = builder.orig; - this.requestTime = builder.requestTime; - this.url = builder.url; + this.latestApiEndpoint = builder.latestApiEndpoint; + this.latestApiMethod = builder.latestApiMethod; + this.latestApiOrig = builder.latestApiOrig; + this.latestApiCollectionId = builder.latestApiCollectionId; + this.detectedAt = builder.detectedAt; + this.eventType = builder.eventType; } public static class Builder { + public EventType eventType; private String filterId; private String actor; private String ip; private String country; - private String url; - private Method method; - private String orig; - private long requestTime; + private String latestApiEndpoint; + private URLMethods.Method latestApiMethod; + private String latestApiOrig; + private int latestApiCollectionId; + private long detectedAt; public Builder setFilterId(String filterId) { this.filterId = filterId; @@ -59,23 +71,33 @@ public Builder setCountry(String country) { return this; } - public Builder setUrl(String url) { - this.url = url; + public Builder setLatestApiEndpoint(String latestApiEndpoint) { + this.latestApiEndpoint = latestApiEndpoint; + return this; + } + + public Builder setLatestApiMethod(URLMethods.Method latestApiMethod) { + this.latestApiMethod = latestApiMethod; + return this; + } + + public Builder setLatestApiOrig(String latestApiOrig) { + this.latestApiOrig = latestApiOrig; return this; } - public Builder setMethod(Method method) { - this.method = method; + public Builder setDetectedAt(long detectedAt) { + this.detectedAt = detectedAt; return this; } - public Builder setOrig(String orig) { - this.orig = orig; + public Builder setLatestApiCollectionId(int latestApiCollectionId) { + this.latestApiCollectionId = latestApiCollectionId; return this; } - public Builder setRequestTime(long requestTime) { - this.requestTime = requestTime; + public Builder setEventType(EventType eventType) { + this.eventType = eventType; return this; } @@ -100,24 +122,32 @@ public String getIp() { return ip; } - public String getUrl() { - return url; + public String getLatestApiEndpoint() { + return latestApiEndpoint; } public String getCountry() { return country; } - public Method getMethod() { - return method; + public URLMethods.Method getLatestApiMethod() { + return latestApiMethod; + } + + public String getLatestApiOrig() { + return latestApiOrig; + } + + public long getDetectedAt() { + return detectedAt; } - public String getOrig() { - return orig; + public int getLatestApiCollectionId() { + return latestApiCollectionId; } - public long getRequestTime() { - return requestTime; + public EventType getEventType() { + return eventType; } public static Builder newBuilder() { diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java deleted file mode 100644 index e28aeb9f30..0000000000 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/SmartEventModel.java +++ /dev/null @@ -1,66 +0,0 @@ -package com.akto.threat.protection.db; - -import java.util.UUID; - -public class SmartEventModel { - - private String id; - - private String filterId; - - private String actor; - - private long totalHits; - - private long detectedAt; - - public SmartEventModel() {} - - public SmartEventModel(String filterId, String actor, long totalHits, long detectedAt) { - this.id = UUID.randomUUID().toString(); - this.filterId = filterId; - this.detectedAt = detectedAt; - this.totalHits = totalHits; - this.actor = actor; - } - - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - - public String getFilterId() { - return filterId; - } - - public void setFilterId(String filterId) { - this.filterId = filterId; - } - - public long getDetectedAt() { - return detectedAt; - } - - public void setDetectedAt(long detectedAt) { - this.detectedAt = detectedAt; - } - - public String getActor() { - return actor; - } - - public void setActor(String actor) { - this.actor = actor; - } - - public long getTotalHits() { - return totalHits; - } - - public void setTotalHits(long totalHits) { - this.totalHits = totalHits; - } -} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java index 37671b844b..a236ab3a4e 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java @@ -6,7 +6,7 @@ import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse; import com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest; -import com.akto.threat.protection.db.MaliciousEventModel; +import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; import com.akto.threat.protection.interceptors.Constants; import com.mongodb.BasicDBObject; import com.mongodb.client.DistinctIterable; @@ -29,7 +29,10 @@ public DashboardService(MongoClient mongoClient) { } private static Set findDistinctFields( - MongoCollection coll, String fieldName, Class tClass, Bson filters) { + MongoCollection coll, + String fieldName, + Class tClass, + Bson filters) { DistinctIterable r = coll.distinct(fieldName, filters, tClass); Set result = new HashSet<>(); MongoCursor cursor = r.cursor(); @@ -44,10 +47,10 @@ public void fetchAlertFilters( FetchAlertFiltersRequest request, StreamObserver responseObserver) { int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); - MongoCollection coll = + MongoCollection coll = this.mongoClient .getDatabase(accountId + "") - .getCollection("malicious_events", MaliciousEventModel.class); + .getCollection("malicious_events", AggregateSampleMaliciousEventModel.class); Set actors = DashboardService.findDistinctFields(coll, "actor", String.class, Filters.empty()); @@ -70,17 +73,17 @@ public void listMaliciousRequests( int limit = request.getLimit(); int skip = (page - 1) * limit; - MongoCollection coll = + MongoCollection coll = this.mongoClient .getDatabase(accountId + "") - .getCollection("malicious_events", MaliciousEventModel.class); + .getCollection("malicious_events", AggregateSampleMaliciousEventModel.class); BasicDBObject query = new BasicDBObject(); - try (MongoCursor cursor = + try (MongoCursor cursor = coll.find(query).skip(skip).limit(limit).cursor()) { List alerts = new ArrayList<>(); while (cursor.hasNext()) { - MaliciousEventModel evt = cursor.next(); + AggregateSampleMaliciousEventModel evt = cursor.next(); alerts.add( MaliciousRequest.newBuilder() .setActor(evt.getActor()) diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousAlertService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousAlertService.java deleted file mode 100644 index 1145253732..0000000000 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousAlertService.java +++ /dev/null @@ -1,55 +0,0 @@ -package com.akto.threat.protection.service; - -import java.util.ArrayList; -import java.util.List; - -import com.akto.dto.type.URLMethods; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousAlertServiceGrpc.MaliciousAlertServiceImplBase; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertRequest; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordAlertResponse; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent; -import com.akto.threat.protection.db.MaliciousEventModel; -import com.akto.threat.protection.db.SmartEventModel; -import com.akto.threat.protection.interceptors.Constants; -import com.akto.threat.protection.utils.KafkaUtils; -import com.mongodb.client.model.WriteModel; -import io.grpc.stub.StreamObserver; - -public class MaliciousAlertService extends MaliciousAlertServiceImplBase { - - public MaliciousAlertService() {} - - @Override - public void recordAlert( - RecordAlertRequest request, StreamObserver responseObserver) { - - String actor = request.getActor(); - String filterId = request.getFilterId(); - List> bulkUpdates = new ArrayList<>(); - int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); - - List maliciousEvents = request.getSampleDataList(); - List events = new ArrayList<>(); - for (SampleMaliciousEvent maliciousEvent : maliciousEvents) { - events.add( - MaliciousEventModel.newBuilder() - .setActor(actor) - .setIp(maliciousEvent.getIp()) - .setUrl(maliciousEvent.getUrl()) - .setMethod(URLMethods.Method.fromString(maliciousEvent.getMethod())) - .setOrig(maliciousEvent.getPayload()) - .setRequestTime(maliciousEvent.getTimestamp()) - .setFilterId(filterId) - .build()); - } - - KafkaUtils.insertData(events, "maliciousEvents", accountId); - KafkaUtils.insertData( - new SmartEventModel(filterId, actor, request.getTotalEvents(), request.getDetectedAt()), - "smartEvent", - accountId); - - responseObserver.onNext(RecordAlertResponse.newBuilder().build()); - responseObserver.onCompleted(); - } -} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java new file mode 100644 index 0000000000..df0fe9f690 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java @@ -0,0 +1,77 @@ +package com.akto.threat.protection.service; + +import java.util.ArrayList; +import java.util.List; + +import com.akto.dto.type.URLMethods; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; +import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; +import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; +import com.akto.threat.protection.db.MaliciousEventModel; +import com.akto.threat.protection.interceptors.Constants; +import com.akto.threat.protection.utils.KafkaUtils; +import com.mongodb.client.model.WriteModel; +import io.grpc.stub.StreamObserver; + +public class MaliciousEventService extends MaliciousEventServiceGrpc.MaliciousEventServiceImplBase { + + public MaliciousEventService() {} + + @Override + public void recordMaliciousEvent( + RecordMaliciousEventRequest request, + StreamObserver responseObserver) { + + MaliciousEvent evt = request.getMaliciousEvent(); + String actor = evt.getActor(); + String filterId = evt.getFilterId(); + List> bulkUpdates = new ArrayList<>(); + int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); + + MaliciousEvent.EventType eventType = evt.getEventType(); + + MaliciousEventModel.EventType maliciousEventType = + MaliciousEvent.EventType.EVENT_TYPE_AGGREGATED.equals(eventType) + ? MaliciousEventModel.EventType.AGGREGATED + : MaliciousEventModel.EventType.SINGLE; + + MaliciousEventModel maliciousEventModel = + MaliciousEventModel.newBuilder() + .setDetectedAt(evt.getDetectedAt()) + .setActor(actor) + .setFilterId(filterId) + .setLatestApiEndpoint(evt.getLatestApiEndpoint()) + .setLatestApiMethod(URLMethods.Method.fromString(evt.getLatestApiMethod())) + .setLatestApiOrig(evt.getLatestApiPayload()) + .setLatestApiCollectionId(evt.getLatestApiCollectionId()) + .setEventType(maliciousEventType) + .build(); + + if (MaliciousEventModel.EventType.AGGREGATED.equals(maliciousEventType)) { + List events = new ArrayList<>(); + for (SampleMaliciousRequest sampleReq : request.getSampleRequestsList()) { + events.add( + AggregateSampleMaliciousEventModel.newBuilder() + .setActor(actor) + .setIp(sampleReq.getIp()) + .setUrl(sampleReq.getUrl()) + .setMethod(URLMethods.Method.fromString(sampleReq.getMethod())) + .setOrig(sampleReq.getPayload()) + .setRequestTime(sampleReq.getTimestamp()) + .setApiCollectionId(sampleReq.getApiCollectionId()) + .setFilterId(filterId) + .build()); + } + + KafkaUtils.insertData(events, "maliciousEvents", accountId); + } + + KafkaUtils.insertData(maliciousEventModel, "smartEvent", accountId); + + responseObserver.onNext(RecordMaliciousEventResponse.newBuilder().build()); + responseObserver.onCompleted(); + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java index f772288a91..50f98a38cf 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java @@ -8,7 +8,6 @@ import java.util.Properties; import java.util.concurrent.atomic.AtomicBoolean; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.SampleMaliciousEvent; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -23,8 +22,8 @@ import com.akto.kafka.Kafka; import com.akto.log.LoggerMaker; import com.akto.log.LoggerMaker.LogDb; +import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; import com.akto.threat.protection.db.MaliciousEventModel; -import com.akto.threat.protection.db.SmartEventModel; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; @@ -36,155 +35,159 @@ public class KafkaUtils { - private static Kafka kafkaProducer; - private static Consumer consumer; - private static final Logger logger = LoggerFactory.getLogger(KafkaUtils.class); - private static final LoggerMaker loggerMaker = new LoggerMaker(KafkaUtils.class); - private static final Gson gson = new Gson(); - private static MongoClient mClient; - private final static ObjectMapper mapper = new ObjectMapper(); - private static long lastSyncOffset = 0; - - public static void initKafkaProducer() { - String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); - int batchSize = Integer.parseInt(System.getenv("AKTO_KAFKA_PRODUCER_BATCH_SIZE")); - int kafkaLingerMS = Integer.parseInt(System.getenv("AKTO_KAFKA_PRODUCER_LINGER_MS")); - kafkaProducer = new Kafka(kafkaBrokerUrl, kafkaLingerMS, batchSize); - logger.info("Kafka Producer Init " + Context.now()); + private static Kafka kafkaProducer; + private static Consumer consumer; + private static final Logger logger = LoggerFactory.getLogger(KafkaUtils.class); + private static final LoggerMaker loggerMaker = new LoggerMaker(KafkaUtils.class); + private static final Gson gson = new Gson(); + private static MongoClient mClient; + private static final ObjectMapper mapper = new ObjectMapper(); + private static long lastSyncOffset = 0; + + public static void initKafkaProducer() { + String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); + int batchSize = Integer.parseInt(System.getenv("AKTO_KAFKA_PRODUCER_BATCH_SIZE")); + int kafkaLingerMS = Integer.parseInt(System.getenv("AKTO_KAFKA_PRODUCER_LINGER_MS")); + kafkaProducer = new Kafka(kafkaBrokerUrl, kafkaLingerMS, batchSize); + logger.info("Kafka Producer Init " + Context.now()); + } + + public static void insertData(Object writes, String eventType, int accountId) { + String topicName = System.getenv("THREAT_EVENTS_KAFKA_TOPIC"); + BasicDBObject obj = new BasicDBObject(); + obj.put("eventType", eventType); + String payloadStr = gson.toJson(writes); + obj.put("payload", payloadStr); + obj.put("accountId", accountId); + kafkaProducer.send(obj.toString(), topicName); + } + + public static void initMongoClient(MongoClient mongoClient) { + mClient = mongoClient; + } + + public static void initKafkaConsumer() { + System.out.println("Kafka Init consumer called"); + String topicName = System.getenv("THREAT_EVENTS_KAFKA_TOPIC"); + String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); // kafka1:19092 + String isKubernetes = System.getenv("IS_KUBERNETES"); + if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { + kafkaBrokerUrl = "127.0.0.1:29092"; } - - public static void insertData(Object writes, String eventType, int accountId) { - String topicName = System.getenv("THREAT_EVENTS_KAFKA_TOPIC"); - BasicDBObject obj = new BasicDBObject(); - obj.put("eventType", eventType); - String payloadStr = gson.toJson(writes); - obj.put("payload", payloadStr); - obj.put("accountId", accountId); - kafkaProducer.send(obj.toString(), topicName); - } - - public static void initMongoClient(MongoClient mongoClient) { - mClient = mongoClient; - } - - public static void initKafkaConsumer() { - System.out.println("Kafka Init consumer called"); - String topicName = System.getenv("THREAT_EVENTS_KAFKA_TOPIC"); - String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); // kafka1:19092 - String isKubernetes = System.getenv("IS_KUBERNETES"); - if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { - kafkaBrokerUrl = "127.0.0.1:29092"; - } - String groupIdConfig = System.getenv("THREAT_EVENTS_KAFKA_GROUP_ID_CONFIG"); - int maxPollRecordsConfig = Integer.parseInt(System.getenv("THREAT_EVENTS_KAFKA_MAX_POLL_RECORDS_CONFIG")); - - Properties properties = configProperties(kafkaBrokerUrl, groupIdConfig, maxPollRecordsConfig); - consumer = new KafkaConsumer<>(properties); - final Thread mainThread = Thread.currentThread(); - final AtomicBoolean exceptionOnCommitSync = new AtomicBoolean(false); - - - Runtime.getRuntime().addShutdownHook(new Thread() { - public void run() { + String groupIdConfig = System.getenv("THREAT_EVENTS_KAFKA_GROUP_ID_CONFIG"); + int maxPollRecordsConfig = + Integer.parseInt(System.getenv("THREAT_EVENTS_KAFKA_MAX_POLL_RECORDS_CONFIG")); + + Properties properties = configProperties(kafkaBrokerUrl, groupIdConfig, maxPollRecordsConfig); + consumer = new KafkaConsumer<>(properties); + final Thread mainThread = Thread.currentThread(); + final AtomicBoolean exceptionOnCommitSync = new AtomicBoolean(false); + + Runtime.getRuntime() + .addShutdownHook( + new Thread() { + public void run() { consumer.wakeup(); try { - if (!exceptionOnCommitSync.get()) { - mainThread.join(); - } + if (!exceptionOnCommitSync.get()) { + mainThread.join(); + } } catch (InterruptedException e) { - e.printStackTrace(); - } catch (Error e){ - loggerMaker.errorAndAddToDb("Error in add shut down hook: "+ e.getMessage(), LogDb.DASHBOARD); + e.printStackTrace(); + } catch (Error e) { + loggerMaker.errorAndAddToDb( + "Error in add shut down hook: " + e.getMessage(), LogDb.DASHBOARD); } - } - }); - + } + }); + + try { + consumer.subscribe(Arrays.asList(topicName)); + loggerMaker.infoAndAddToDb("Kafka Consumer subscribed", LogDb.DASHBOARD); + while (true) { + ConsumerRecords records = consumer.poll(Duration.ofMillis(10000)); try { - consumer.subscribe(Arrays.asList(topicName)); - loggerMaker.infoAndAddToDb("Kafka Consumer subscribed", LogDb.DASHBOARD); - while (true) { - ConsumerRecords records = consumer.poll(Duration.ofMillis(10000)); - try { - consumer.commitSync(); - } catch (Exception e) { - throw e; - } - - for (ConsumerRecord r: records) { - try { - lastSyncOffset++; - if (lastSyncOffset % 100 == 0) { - logger.info("Committing offset at position: " + lastSyncOffset); - } - - parseAndTriggerWrites(r.value()); - } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, "Error in parseAndTriggerWrites " + e, LogDb.DASHBOARD); - continue; - } - } - } - } catch (WakeupException ignored) { - // nothing to catch. This exception is called from the shutdown hook. + consumer.commitSync(); } catch (Exception e) { - exceptionOnCommitSync.set(true); - loggerMaker.errorAndAddToDb("Exception in init kafka consumer " + e.getMessage(),LogDb.DASHBOARD); - e.printStackTrace(); - System.exit(0); - } finally { - consumer.close(); + throw e; } - } + for (ConsumerRecord r : records) { + try { + lastSyncOffset++; + if (lastSyncOffset % 100 == 0) { + logger.info("Committing offset at position: " + lastSyncOffset); + } - private static void parseAndTriggerWrites(String message) throws Exception { - Map json = gson.fromJson(message, Map.class); - String eventType = (String) json.get("eventType"); - String payload = (String) json.get("payload"); - Double accIdDouble = (Double) json.get("accountId"); - int accountId = accIdDouble.intValue(); - Context.accountId.set(accountId); - - switch (eventType) { - case "maliciousEvents": - List> bulkUpdates = new ArrayList<>(); - List events = mapper.readValue(payload, new TypeReference>(){}); - events - .forEach( - event -> { - bulkUpdates.add( - new InsertOneModel<>( - event)); - }); - - mClient.getDatabase(accountId + "") - .getCollection("malicious_events", MaliciousEventModel.class) - .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); - break; - - case "smartEvent": - SmartEventModel event = mapper.readValue(payload, new TypeReference(){}); - mClient - .getDatabase(accountId + "") - .getCollection("smart_events", SmartEventModel.class) - .insertOne(event); - break; - default: - break; + parseAndTriggerWrites(r.value()); + } catch (Exception e) { + loggerMaker.errorAndAddToDb(e, "Error in parseAndTriggerWrites " + e, LogDb.DASHBOARD); + continue; + } } + } + } catch (WakeupException ignored) { + // nothing to catch. This exception is called from the shutdown hook. + } catch (Exception e) { + exceptionOnCommitSync.set(true); + loggerMaker.errorAndAddToDb( + "Exception in init kafka consumer " + e.getMessage(), LogDb.DASHBOARD); + e.printStackTrace(); + System.exit(0); + } finally { + consumer.close(); } - - public static Properties configProperties(String kafkaBrokerUrl, String groupIdConfig, int maxPollRecordsConfig) { - Properties properties = new Properties(); - properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaBrokerUrl); - properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecordsConfig); - properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupIdConfig); - properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); - - return properties; + } + + private static void parseAndTriggerWrites(String message) throws Exception { + Map json = gson.fromJson(message, Map.class); + String eventType = (String) json.get("eventType"); + String payload = (String) json.get("payload"); + Double accIdDouble = (Double) json.get("accountId"); + int accountId = accIdDouble.intValue(); + Context.accountId.set(accountId); + + switch (eventType) { + case "maliciousEvents": + List> bulkUpdates = new ArrayList<>(); + List events = + mapper.readValue( + payload, new TypeReference>() {}); + events.forEach( + event -> { + bulkUpdates.add(new InsertOneModel<>(event)); + }); + + mClient + .getDatabase(accountId + "") + .getCollection("malicious_events", AggregateSampleMaliciousEventModel.class) + .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); + break; + + case "smartEvent": + MaliciousEventModel event = + mapper.readValue(payload, new TypeReference() {}); + mClient + .getDatabase(accountId + "") + .getCollection("smart_events", MaliciousEventModel.class) + .insertOne(event); + break; + default: + break; } - + } + + public static Properties configProperties( + String kafkaBrokerUrl, String groupIdConfig, int maxPollRecordsConfig) { + Properties properties = new Properties(); + properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaBrokerUrl); + properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecordsConfig); + properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupIdConfig); + properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + + return properties; + } } diff --git a/protobuf/threat_protection/message/malicious_event/v1/message.proto b/protobuf/threat_protection/message/malicious_event/v1/message.proto index 837d22b661..4b2ed7b0b8 100644 --- a/protobuf/threat_protection/message/malicious_event/v1/message.proto +++ b/protobuf/threat_protection/message/malicious_event/v1/message.proto @@ -6,12 +6,19 @@ option java_outer_classname = "MaliciousEventProto"; option java_package = "threat_protection.v1.message.malicious_event"; message MaliciousEvent { - string actor= 1; + string actor = 1; string filter_id = 2; - string ip = 3; - int64 timestamp = 4; - string url = 5; - string method = 6; - int32 api_collection_id = 7; - string payload = 8; + int64 detected_at = 3; + string latest_api_ip = 4; + string latest_api_endpoint = 5; + string latest_api_method = 6; + int32 latest_api_collection_id = 7; + string latest_api_payload = 8; + EventType event_type = 9; + + enum EventType { + EVENT_TYPE_UNSPECIFIED = 0; + EVENT_TYPE_SINGLE = 1; + EVENT_TYPE_AGGREGATED = 2; + } } diff --git a/protobuf/threat_protection/message/sample_request/v1/message.proto b/protobuf/threat_protection/message/sample_request/v1/message.proto new file mode 100644 index 0000000000..37a19b1411 --- /dev/null +++ b/protobuf/threat_protection/message/sample_request/v1/message.proto @@ -0,0 +1,16 @@ +syntax = "proto3"; + +package threat_protection.message.sample_request.v1; + +option java_outer_classname = "SampleRequestProto"; +option java_package = "threat_protection.v1.message.sample_request"; + +message SampleMaliciousRequest { + string ip = 1; + int64 timestamp = 2; + string url = 3; + string method = 4; + int32 api_collection_id = 5; + string payload = 6; + string filter_id = 7; +} diff --git a/protobuf/threat_protection/message/smart_event/v1/message.proto b/protobuf/threat_protection/message/smart_event/v1/message.proto deleted file mode 100644 index b7dacf6e53..0000000000 --- a/protobuf/threat_protection/message/smart_event/v1/message.proto +++ /dev/null @@ -1,13 +0,0 @@ -syntax = "proto3"; - -package threat_protection.message.smart_event.v1; - -option java_outer_classname = "SmartEventProto"; -option java_package = "threat_protection.v1.message.smart_event"; - -message SmartEvent { - string actor= 1; - string filter_id = 2; - int64 detected_at = 3; - string rule_id = 4; -} diff --git a/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto b/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto index 3aa4a6ee79..55b3322149 100644 --- a/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto +++ b/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto @@ -2,31 +2,22 @@ syntax = "proto3"; package threat_protection.service.malicious_alert_service.v1; +import "threat_protection/message/malicious_event/v1/message.proto"; +import "threat_protection/message/sample_request/v1/message.proto"; + // This is a consumer service for recording malicious alerts // For dashboard purposes we will have a separate service to retrieve these events. option java_outer_classname = "MaliciousAlertServiceProto"; option java_package = "threat_protection.service.malicious_alert_service.v1"; -message RecordAlertResponse { -} - -message SampleMaliciousEvent { - string ip = 1; - int64 timestamp = 2; - string url = 3; - string method = 4; - int32 api_collection_id = 5; - string payload = 6; +message RecordMaliciousEventResponse { } -message RecordAlertRequest { - string actor = 1; - string filter_id = 2; - int64 total_events = 3; - int64 detected_at = 4; - repeated SampleMaliciousEvent sample_data = 5; +message RecordMaliciousEventRequest { + threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1; + repeated threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2; } -service MaliciousAlertService { - rpc RecordAlert(RecordAlertRequest) returns (RecordAlertResponse) {} +service MaliciousEventService { + rpc RecordMaliciousEvent(RecordMaliciousEventRequest) returns (RecordMaliciousEventResponse) {} } \ No newline at end of file From 80b04e1de61c9985ed0df284a1a22322f17c591a Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 10 Dec 2024 16:17:43 +0530 Subject: [PATCH 44/73] storing api_collection_id in postgres --- .../session_factory/SessionFactoryUtils.java | 38 +++++++------- .../threat/detection/tasks/CleanupTask.java | 49 ++++++++++--------- .../detection/tasks/FlushSampleDataTask.java | 28 ++++++----- .../tasks/SendMaliciousRequestsToBackend.java | 8 +++ .../V2__create_malicious_event_table.sql | 1 + .../java/com/akto/threat/protection/Main.java | 5 +- .../threat/protection/utils/KafkaUtils.java | 19 ++++--- 7 files changed, 85 insertions(+), 63 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java index c725bd978c..556486182f 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java @@ -8,23 +8,23 @@ public class SessionFactoryUtils { - public static SessionFactory createFactory() { - final String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); - final String user = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_USER"); - final String password = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_PASSWORD"); - - final Configuration cfg = new Configuration(); - cfg.setProperty("hibernate.connection.url", url); - cfg.setProperty("hibernate.connection.user", user); - cfg.setProperty("hibernate.connection.password", password); - cfg.setProperty("dialect", "org.hibernate.dialect.PostgreSQL92Dialect"); - cfg.setProperty("connection.driver_class", "org.postgresql.Driver"); - cfg.setProperty("show_sql", "true"); - cfg.setProperty("format_sql", "true"); - - cfg.addAnnotatedClass(MaliciousEventEntity.class); - - return cfg.buildSessionFactory(new StandardServiceRegistryBuilder().applySettings(cfg.getProperties()).build()); - } - + public static SessionFactory createFactory() { + final String url = System.getenv("AKTO_THREAT_DETECTION_POSTGRES"); + final String user = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_USER"); + final String password = System.getenv("AKTO_THREAT_DETECTION_POSTGRES_PASSWORD"); + + final Configuration cfg = new Configuration(); + cfg.setProperty("hibernate.connection.url", url); + cfg.setProperty("hibernate.connection.user", user); + cfg.setProperty("hibernate.connection.password", password); + cfg.setProperty("dialect", "org.hibernate.dialect.PostgreSQL92Dialect"); + cfg.setProperty("connection.driver_class", "org.postgresql.Driver"); + cfg.setProperty("show_sql", "false"); + cfg.setProperty("format_sql", "false"); + + cfg.addAnnotatedClass(MaliciousEventEntity.class); + + return cfg.buildSessionFactory( + new StandardServiceRegistryBuilder().applySettings(cfg.getProperties()).build()); + } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java index 81914ef901..acfc6abbc8 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java @@ -12,29 +12,30 @@ public class CleanupTask implements Task { - private final SessionFactory sessionFactory; - - private final ScheduledExecutorService cronExecutorService = Executors.newScheduledThreadPool(1); - - public CleanupTask(SessionFactory sessionFactory) { - this.sessionFactory = sessionFactory; - } - - @Override - public void run() { - this.cronExecutorService.scheduleAtFixedRate(this::cleanup, 5, 10 * 60, TimeUnit.SECONDS); - } - - private void cleanup() { - Session session = this.sessionFactory.openSession(); - Transaction txn = session.beginTransaction(); - int deletedCount = session.createQuery("delete from MaliciousEventEntity m where m.createdAt < :startDate") - .setParameter("startDate", LocalDateTime.now(ZoneOffset.UTC).minusDays(7)) - .executeUpdate(); - - txn.commit(); - session.close(); - - System.out.println("Number of rows deleted: " + deletedCount); + private final SessionFactory sessionFactory; + + private final ScheduledExecutorService cronExecutorService = Executors.newScheduledThreadPool(1); + + public CleanupTask(SessionFactory sessionFactory) { + this.sessionFactory = sessionFactory; + } + + @Override + public void run() { + this.cronExecutorService.scheduleAtFixedRate(this::cleanup, 5, 10 * 60, TimeUnit.SECONDS); + } + + private void cleanup() { + try (Session session = this.sessionFactory.openSession()) { + Transaction txn = session.beginTransaction(); + int deletedCount = + session + .createQuery("delete from MaliciousEventEntity m where m.createdAt < :startDate") + .setParameter("startDate", LocalDateTime.now(ZoneOffset.UTC).minusDays(7)) + .executeUpdate(); + + txn.commit(); + System.out.println("Number of rows deleted: " + deletedCount); } + } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 7fad234199..dd5fd28bce 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -64,25 +64,29 @@ protected void processRecords(ConsumerRecords records) { .setMethod(URLMethods.Method.fromString(evt.getMethod())) .setTimestamp(evt.getTimestamp()) .setOrig(evt.getPayload()) + .setApiCollectionId(evt.getApiCollectionId()) .setIp(evt.getIp()) .build()); }); Session session = this.sessionFactory.openSession(); Transaction txn = session.beginTransaction(); - - txn.begin(); - - // Commit these events in 2 batches - for (int i = 0; i < events.size(); i += 2) { - session.persist(events.get(i)); - if (i % 50 == 0) { - session.flush(); - session.clear(); + try { + // Commit these events in 2 batches + for (int i = 0; i < events.size(); i += 2) { + session.persist(events.get(i)); + if (i % 50 == 0) { + session.flush(); + session.clear(); + } } - } - txn.commit(); - session.close(); + txn.commit(); + } catch (Exception e) { + e.printStackTrace(); + txn.rollback(); + } finally { + session.close(); + } } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java index cf1a74a19f..72bf1008f2 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java @@ -20,6 +20,8 @@ import org.hibernate.Session; import org.hibernate.SessionFactory; import org.hibernate.Transaction; + +import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -64,10 +66,15 @@ private List getSampleMaliciousRequests(String actor, Stri .setParameter("filterId", filterId) .setMaxResults(50) .getResultList(); + } catch (Exception ex) { + ex.printStackTrace(); + txn.rollback(); } finally { txn.commit(); session.close(); } + + return Collections.emptyList(); } protected void processRecords(ConsumerRecords records) { @@ -128,6 +135,7 @@ public void onError(Throwable t) { @Override public void onCompleted() { // Do nothing + System.out.println("Completed"); } }); } catch (Exception e) { diff --git a/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql b/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql index a346d0bab5..17e1b9d80c 100644 --- a/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql +++ b/apps/threat-detection/src/main/resources/db/migration/V2__create_malicious_event_table.sql @@ -9,6 +9,7 @@ create table if not exists threat_detection.malicious_event ( method varchar(255), timestamp bigint not null, orig text not null, + api_collection_id int not null, created_at timestamp default (timezone('utc', now())) ); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index dcc78a1290..4265fd918a 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -16,8 +16,9 @@ public static void main(String[] args) throws Exception { MongoClient threatProtectionMongo = DaoInit.createMongoClient( new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")), - ReadPreference.secondary(), WriteConcern.ACKNOWLEDGED); - String initProducer = System.getenv("INIT_KAFKA_PRODUCER"); + ReadPreference.secondary(), + WriteConcern.ACKNOWLEDGED); + String initProducer = System.getenv().getOrDefault("INIT_KAFKA_PRODUCER", "true"); if (initProducer != null && initProducer.equalsIgnoreCase("true")) { KafkaUtils.initKafkaProducer(); } else { diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java index 50f98a38cf..ac8b1b380d 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java @@ -46,14 +46,18 @@ public class KafkaUtils { public static void initKafkaProducer() { String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); - int batchSize = Integer.parseInt(System.getenv("AKTO_KAFKA_PRODUCER_BATCH_SIZE")); - int kafkaLingerMS = Integer.parseInt(System.getenv("AKTO_KAFKA_PRODUCER_LINGER_MS")); + int batchSize = + Integer.parseInt(System.getenv().getOrDefault("AKTO_KAFKA_PRODUCER_BATCH_SIZE", "100")); + int kafkaLingerMS = + Integer.parseInt(System.getenv().getOrDefault("AKTO_KAFKA_PRODUCER_LINGER_MS", "1000")); kafkaProducer = new Kafka(kafkaBrokerUrl, kafkaLingerMS, batchSize); logger.info("Kafka Producer Init " + Context.now()); } public static void insertData(Object writes, String eventType, int accountId) { - String topicName = System.getenv("THREAT_EVENTS_KAFKA_TOPIC"); + String topicName = + System.getenv() + .getOrDefault("THREAT_EVENTS_KAFKA_TOPIC", "akto.threat_protection.internal_events"); BasicDBObject obj = new BasicDBObject(); obj.put("eventType", eventType); String payloadStr = gson.toJson(writes); @@ -68,15 +72,18 @@ public static void initMongoClient(MongoClient mongoClient) { public static void initKafkaConsumer() { System.out.println("Kafka Init consumer called"); - String topicName = System.getenv("THREAT_EVENTS_KAFKA_TOPIC"); + String topicName = + System.getenv() + .getOrDefault("THREAT_EVENTS_KAFKA_TOPIC", "akto.threat_protection.internal_events"); String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); // kafka1:19092 - String isKubernetes = System.getenv("IS_KUBERNETES"); + String isKubernetes = System.getenv().getOrDefault("IS_KUBERNETES", "false"); if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { kafkaBrokerUrl = "127.0.0.1:29092"; } String groupIdConfig = System.getenv("THREAT_EVENTS_KAFKA_GROUP_ID_CONFIG"); int maxPollRecordsConfig = - Integer.parseInt(System.getenv("THREAT_EVENTS_KAFKA_MAX_POLL_RECORDS_CONFIG")); + Integer.parseInt( + System.getenv().getOrDefault("THREAT_EVENTS_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); Properties properties = configProperties(kafkaBrokerUrl, groupIdConfig, maxPollRecordsConfig); consumer = new KafkaConsumer<>(properties); From 2fe1e23bd1929f7f2f72c1fde354eb2a981dd3da Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 11 Dec 2024 11:11:57 +0530 Subject: [PATCH 45/73] fixed paths in ci for threat-detection module --- .github/workflows/prod.yml | 17 +- .github/workflows/staging.yml | 6 +- apps/threat-detection/Dockerfile | 4 +- .../malicious_event/v1/MaliciousEvent.java | 2766 ----------------- .../v1/MaliciousEventOrBuilder.java | 168 - .../malicious_event/v1/MessageProto.java | 110 - .../sample_request/v1/MessageProto.java | 73 - .../v1/SampleMaliciousRequest.java | 1178 ------- .../v1/SampleMaliciousRequestOrBuilder.java | 83 - .../v1/MaliciousEventServiceGrpc.java | 293 -- .../v1/RecordMaliciousEventRequest.java | 913 ------ .../RecordMaliciousEventRequestOrBuilder.java | 50 - .../v1/RecordMaliciousEventResponse.java | 358 --- ...RecordMaliciousEventResponseOrBuilder.java | 11 - .../v1/ServiceProto.java | 99 - 15 files changed, 16 insertions(+), 6113 deletions(-) delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/MessageProto.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousEventServiceGrpc.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequest.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequestOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponse.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponseOrBuilder.java delete mode 100644 libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ServiceProto.java diff --git a/.github/workflows/prod.yml b/.github/workflows/prod.yml index 269cb175e4..f9127da18d 100644 --- a/.github/workflows/prod.yml +++ b/.github/workflows/prod.yml @@ -6,8 +6,8 @@ on: workflow_dispatch: inputs: release_version: - required: true - + required: true + # A workflow run is made up of one or more jobs that can run sequentially or in parallel jobs: # This workflow contains a single job called "build" @@ -17,7 +17,7 @@ jobs: # Steps represent a sequence of tasks that will be executed as part of the job steps: - # Checks-out your repository under $GITHUB_WORKSPACE, so your job can access it + # Checks-out your repository under $GITHUB_WORKSPACE, so your job can access it - uses: actions/checkout@v2 - uses: actions/setup-java@v2 with: @@ -27,6 +27,11 @@ jobs: - uses: actions/setup-node@v2 with: node-version: '17' + - uses: bufbuild/buf-action@v1 + with: + setup_only: true + - name: Generate Proto files + run: make proto-gen - name: Download Akto templates zip and PII files working-directory: ./apps/dashboard/src/main/resources run: | @@ -82,8 +87,8 @@ jobs: cd ../internal docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-internal:$IMAGE_TAG_1 -t $ECR_REGISTRY/akto-internal:$IMAGE_TAG_2 . --push echo "::set-output name=image::$ECR_REGISTRY/akto-internal:$IMAGE_TAG" - cd ../api-threat-detection - docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-api-protection:$IMAGE_TAG_1 -t $ECR_REGISTRY/akto-api-protection:$IMAGE_TAG_2 -t $ECR_REGISTRY/akto-api-protection:$IMAGE_TAG_3 . --push + cd ../threat-detection + docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-threat-protection:$IMAGE_TAG_1 -t $ECR_REGISTRY/akto-threat-detection:$IMAGE_TAG_2 -t $ECR_REGISTRY/akto-threat-detection:$IMAGE_TAG_3 . --push - name: Configure AWS Credentials for ECR uses: aws-actions/configure-aws-credentials@v1 @@ -108,7 +113,7 @@ jobs: run: | docker buildx create --use # Build a docker container and push it to DockerHub - cd apps/api-threat-detection + cd apps/threat-detection docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/$REGISTRY_ALIAS/akto-api-protection:$IMAGE_TAG -t $ECR_REGISTRY/$REGISTRY_ALIAS/akto-api-protection:$IMAGE_TAG2 . --push echo "::set-output name=image::$ECR_REGISTRY/akto-api-protection:$IMAGE_TAG" diff --git a/.github/workflows/staging.yml b/.github/workflows/staging.yml index d2b40efa9f..afb68670e2 100644 --- a/.github/workflows/staging.yml +++ b/.github/workflows/staging.yml @@ -3,7 +3,7 @@ name: Staging # Controls when the workflow will run on: push: - branches: [master] + branches: [ master ] pull_request: workflow_dispatch: @@ -84,8 +84,8 @@ jobs: docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-billing:$IMAGE_TAG . --push cd ../internal docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-internal:$IMAGE_TAG . --push - cd ../api-threat-detection - docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-api-protection:$IMAGE_TAG . --push + cd ../threat-detection + docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-threat-detection:$IMAGE_TAG . --push cd ../source-code-analyser docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/source-code-analyser:$IMAGE_TAG . --push diff --git a/apps/threat-detection/Dockerfile b/apps/threat-detection/Dockerfile index 80e2dad6aa..41b66dbbac 100644 --- a/apps/threat-detection/Dockerfile +++ b/apps/threat-detection/Dockerfile @@ -1,4 +1,4 @@ FROM openjdk WORKDIR /app -COPY ./target/api-threat-detection-1.0-SNAPSHOT-jar-with-dependencies.jar /app/api-threat-detection-1.0-SNAPSHOT-jar-with-dependencies.jar -CMD "java" "-XX:+ExitOnOutOfMemoryError" "-jar" "/app/api-threat-detection-1.0-SNAPSHOT-jar-with-dependencies.jar" \ No newline at end of file +COPY ./target/threat-detection-1.0-SNAPSHOT-jar-with-dependencies.jar /app/threat-detection-1.0-SNAPSHOT-jar-with-dependencies.jar +CMD "java" "-XX:+ExitOnOutOfMemoryError" "-jar" "/app/threat-detection-1.0-SNAPSHOT-jar-with-dependencies.jar" \ No newline at end of file diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java deleted file mode 100644 index bc57008c58..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEvent.java +++ /dev/null @@ -1,2766 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/malicious_event/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.malicious_event.v1; - -/** - * Protobuf type {@code threat_protection.message.malicious_event.v1.MaliciousEvent} - */ -public final class MaliciousEvent extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.message.malicious_event.v1.MaliciousEvent) - MaliciousEventOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - MaliciousEvent.class.getName()); - } - // Use MaliciousEvent.newBuilder() to construct. - private MaliciousEvent(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private MaliciousEvent() { - actor_ = ""; - filterId_ = ""; -<<<<<<< HEAD - latestApiIp_ = ""; - latestApiEndpoint_ = ""; - latestApiMethod_ = ""; - latestApiPayload_ = ""; - eventType_ = 0; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.class, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder.class); - } - - /** - * Protobuf enum {@code threat_protection.message.malicious_event.v1.MaliciousEvent.EventType} - */ - public enum EventType - implements com.google.protobuf.ProtocolMessageEnum { - /** - * EVENT_TYPE_UNSPECIFIED = 0; - */ - EVENT_TYPE_UNSPECIFIED(0), - /** - * EVENT_TYPE_SINGLE = 1; - */ - EVENT_TYPE_SINGLE(1), - /** - * EVENT_TYPE_AGGREGATED = 2; - */ - EVENT_TYPE_AGGREGATED(2), - UNRECOGNIZED(-1), - ; - - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - EventType.class.getName()); - } - /** - * EVENT_TYPE_UNSPECIFIED = 0; - */ - public static final int EVENT_TYPE_UNSPECIFIED_VALUE = 0; - /** - * EVENT_TYPE_SINGLE = 1; - */ - public static final int EVENT_TYPE_SINGLE_VALUE = 1; - /** - * EVENT_TYPE_AGGREGATED = 2; - */ - public static final int EVENT_TYPE_AGGREGATED_VALUE = 2; - - - public final int getNumber() { - if (this == UNRECOGNIZED) { - throw new java.lang.IllegalArgumentException( - "Can't get the number of an unknown enum value."); - } - return value; - } - - /** - * @param value The numeric wire value of the corresponding enum entry. - * @return The enum associated with the given numeric wire value. - * @deprecated Use {@link #forNumber(int)} instead. - */ - @java.lang.Deprecated - public static EventType valueOf(int value) { - return forNumber(value); - } - - /** - * @param value The numeric wire value of the corresponding enum entry. - * @return The enum associated with the given numeric wire value. - */ - public static EventType forNumber(int value) { - switch (value) { - case 0: return EVENT_TYPE_UNSPECIFIED; - case 1: return EVENT_TYPE_SINGLE; - case 2: return EVENT_TYPE_AGGREGATED; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static final com.google.protobuf.Internal.EnumLiteMap< - EventType> internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public EventType findValueByNumber(int number) { - return EventType.forNumber(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - if (this == UNRECOGNIZED) { - throw new java.lang.IllegalStateException( - "Can't get the descriptor of an unrecognized enum value."); - } - return getDescriptor().getValues().get(ordinal()); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDescriptor().getEnumTypes().get(0); - } - - private static final EventType[] VALUES = values(); - - public static EventType valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - if (desc.getIndex() == -1) { - return UNRECOGNIZED; - } - return VALUES[desc.getIndex()]; - } - - private final int value; - - private EventType(int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:threat_protection.message.malicious_event.v1.MaliciousEvent.EventType) - } - - public static final int ACTOR_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - @java.lang.Override - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int DETECTED_AT_FIELD_NUMBER = 3; - private long detectedAt_ = 0L; - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - - public static final int LATEST_API_IP_FIELD_NUMBER = 4; - @SuppressWarnings("serial") - private volatile java.lang.Object latestApiIp_ = ""; - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @return The latestApiIp. - */ - @java.lang.Override - public java.lang.String getLatestApiIp() { - java.lang.Object ref = latestApiIp_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - latestApiIp_ = s; - return s; - } - } - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @return The bytes for latestApiIp. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getLatestApiIpBytes() { - java.lang.Object ref = latestApiIp_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - latestApiIp_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int LATEST_API_ENDPOINT_FIELD_NUMBER = 5; - @SuppressWarnings("serial") - private volatile java.lang.Object latestApiEndpoint_ = ""; - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @return The latestApiEndpoint. - */ - @java.lang.Override - public java.lang.String getLatestApiEndpoint() { - java.lang.Object ref = latestApiEndpoint_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - latestApiEndpoint_ = s; - return s; - } - } - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @return The bytes for latestApiEndpoint. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getLatestApiEndpointBytes() { - java.lang.Object ref = latestApiEndpoint_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - latestApiEndpoint_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int LATEST_API_METHOD_FIELD_NUMBER = 6; - @SuppressWarnings("serial") - private volatile java.lang.Object latestApiMethod_ = ""; - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @return The latestApiMethod. - */ - @java.lang.Override - public java.lang.String getLatestApiMethod() { - java.lang.Object ref = latestApiMethod_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - latestApiMethod_ = s; - return s; - } - } - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @return The bytes for latestApiMethod. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getLatestApiMethodBytes() { - java.lang.Object ref = latestApiMethod_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - latestApiMethod_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int LATEST_API_COLLECTION_ID_FIELD_NUMBER = 7; - private int latestApiCollectionId_ = 0; - /** - * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; - * @return The latestApiCollectionId. - */ - @java.lang.Override - public int getLatestApiCollectionId() { - return latestApiCollectionId_; - } - - public static final int LATEST_API_PAYLOAD_FIELD_NUMBER = 8; - @SuppressWarnings("serial") - private volatile java.lang.Object latestApiPayload_ = ""; - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @return The latestApiPayload. - */ - @java.lang.Override - public java.lang.String getLatestApiPayload() { - java.lang.Object ref = latestApiPayload_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - latestApiPayload_ = s; - return s; - } - } - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @return The bytes for latestApiPayload. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getLatestApiPayloadBytes() { - java.lang.Object ref = latestApiPayload_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - latestApiPayload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int EVENT_TYPE_FIELD_NUMBER = 9; - private int eventType_ = 0; - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @return The enum numeric value on the wire for eventType. - */ - @java.lang.Override public int getEventTypeValue() { - return eventType_; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @return The eventType. - */ - @java.lang.Override public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType getEventType() { - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType result = com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.forNumber(eventType_); - return result == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.UNRECOGNIZED : result; - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); - } - if (detectedAt_ != 0L) { - output.writeInt64(3, detectedAt_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiIp_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 4, latestApiIp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiEndpoint_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 5, latestApiEndpoint_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiMethod_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 6, latestApiMethod_); - } - if (latestApiCollectionId_ != 0) { - output.writeInt32(7, latestApiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiPayload_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 8, latestApiPayload_); - } - if (eventType_ != com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.EVENT_TYPE_UNSPECIFIED.getNumber()) { - output.writeEnum(9, eventType_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); - } - if (detectedAt_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(3, detectedAt_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiIp_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(4, latestApiIp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiEndpoint_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(5, latestApiEndpoint_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiMethod_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(6, latestApiMethod_); - } - if (latestApiCollectionId_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(7, latestApiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(latestApiPayload_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(8, latestApiPayload_); - } - if (eventType_ != com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.EVENT_TYPE_UNSPECIFIED.getNumber()) { - size += com.google.protobuf.CodedOutputStream - .computeEnumSize(9, eventType_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent other = (com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent) obj; - - if (!getActor() - .equals(other.getActor())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (getDetectedAt() - != other.getDetectedAt()) return false; - if (!getLatestApiIp() - .equals(other.getLatestApiIp())) return false; - if (!getLatestApiEndpoint() - .equals(other.getLatestApiEndpoint())) return false; - if (!getLatestApiMethod() - .equals(other.getLatestApiMethod())) return false; - if (getLatestApiCollectionId() - != other.getLatestApiCollectionId()) return false; - if (!getLatestApiPayload() - .equals(other.getLatestApiPayload())) return false; - if (eventType_ != other.eventType_) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACTOR_FIELD_NUMBER; - hash = (53 * hash) + getActor().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (37 * hash) + DETECTED_AT_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getDetectedAt()); - hash = (37 * hash) + LATEST_API_IP_FIELD_NUMBER; - hash = (53 * hash) + getLatestApiIp().hashCode(); - hash = (37 * hash) + LATEST_API_ENDPOINT_FIELD_NUMBER; - hash = (53 * hash) + getLatestApiEndpoint().hashCode(); - hash = (37 * hash) + LATEST_API_METHOD_FIELD_NUMBER; - hash = (53 * hash) + getLatestApiMethod().hashCode(); - hash = (37 * hash) + LATEST_API_COLLECTION_ID_FIELD_NUMBER; - hash = (53 * hash) + getLatestApiCollectionId(); - hash = (37 * hash) + LATEST_API_PAYLOAD_FIELD_NUMBER; - hash = (53 * hash) + getLatestApiPayload().hashCode(); - hash = (37 * hash) + EVENT_TYPE_FIELD_NUMBER; - hash = (53 * hash) + eventType_; - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.message.malicious_event.v1.MaliciousEvent} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.message.malicious_event.v1.MaliciousEvent) - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.class, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - actor_ = ""; - filterId_ = ""; - detectedAt_ = 0L; - latestApiIp_ = ""; - latestApiEndpoint_ = ""; - latestApiMethod_ = ""; - latestApiCollectionId_ = 0; - latestApiPayload_ = ""; - eventType_ = 0; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getDefaultInstanceForType() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent build() { - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent buildPartial() { - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result = new com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.actor_ = actor_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.filterId_ = filterId_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.detectedAt_ = detectedAt_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.latestApiIp_ = latestApiIp_; - } - if (((from_bitField0_ & 0x00000010) != 0)) { - result.latestApiEndpoint_ = latestApiEndpoint_; - } - if (((from_bitField0_ & 0x00000020) != 0)) { - result.latestApiMethod_ = latestApiMethod_; - } - if (((from_bitField0_ & 0x00000040) != 0)) { - result.latestApiCollectionId_ = latestApiCollectionId_; - } - if (((from_bitField0_ & 0x00000080) != 0)) { - result.latestApiPayload_ = latestApiPayload_; - } - if (((from_bitField0_ & 0x00000100) != 0)) { - result.eventType_ = eventType_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent) { - return mergeFrom((com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent other) { - if (other == com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance()) return this; - if (!other.getActor().isEmpty()) { - actor_ = other.actor_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000002; - onChanged(); - } - if (other.getDetectedAt() != 0L) { - setDetectedAt(other.getDetectedAt()); - } - if (!other.getLatestApiIp().isEmpty()) { - latestApiIp_ = other.latestApiIp_; - bitField0_ |= 0x00000008; - onChanged(); - } - if (!other.getLatestApiEndpoint().isEmpty()) { - latestApiEndpoint_ = other.latestApiEndpoint_; - bitField0_ |= 0x00000010; - onChanged(); - } - if (!other.getLatestApiMethod().isEmpty()) { - latestApiMethod_ = other.latestApiMethod_; - bitField0_ |= 0x00000020; - onChanged(); - } - if (other.getLatestApiCollectionId() != 0) { - setLatestApiCollectionId(other.getLatestApiCollectionId()); - } - if (!other.getLatestApiPayload().isEmpty()) { - latestApiPayload_ = other.latestApiPayload_; - bitField0_ |= 0x00000080; - onChanged(); - } - if (other.eventType_ != 0) { - setEventTypeValue(other.getEventTypeValue()); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - actor_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000002; - break; - } // case 18 - case 24: { - detectedAt_ = input.readInt64(); - bitField0_ |= 0x00000004; - break; - } // case 24 - case 34: { - latestApiIp_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000008; - break; - } // case 34 - case 42: { - latestApiEndpoint_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000010; - break; - } // case 42 - case 50: { - latestApiMethod_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000020; - break; - } // case 50 - case 56: { - latestApiCollectionId_ = input.readInt32(); - bitField0_ |= 0x00000040; - break; - } // case 56 - case 66: { - latestApiPayload_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000080; - break; - } // case 66 - case 72: { - eventType_ = input.readEnum(); - bitField0_ |= 0x00000100; - break; - } // case 72 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The actor to set. - * @return This builder for chaining. - */ - public Builder setActor( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @return This builder for chaining. - */ - public Builder clearActor() { - actor_ = getDefaultInstance().getActor(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The bytes for actor to set. - * @return This builder for chaining. - */ - public Builder setActorBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - private long detectedAt_ ; - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - @java.lang.Override - public long getDetectedAt() { - return detectedAt_; - } - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @param value The detectedAt to set. - * @return This builder for chaining. - */ - public Builder setDetectedAt(long value) { - - detectedAt_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return This builder for chaining. - */ - public Builder clearDetectedAt() { - bitField0_ = (bitField0_ & ~0x00000004); - detectedAt_ = 0L; - onChanged(); - return this; - } - - private java.lang.Object latestApiIp_ = ""; - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @return The latestApiIp. - */ - public java.lang.String getLatestApiIp() { - java.lang.Object ref = latestApiIp_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - latestApiIp_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @return The bytes for latestApiIp. - */ - public com.google.protobuf.ByteString - getLatestApiIpBytes() { - java.lang.Object ref = latestApiIp_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - latestApiIp_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @param value The latestApiIp to set. - * @return This builder for chaining. - */ - public Builder setLatestApiIp( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - latestApiIp_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @return This builder for chaining. - */ - public Builder clearLatestApiIp() { - latestApiIp_ = getDefaultInstance().getLatestApiIp(); - bitField0_ = (bitField0_ & ~0x00000008); - onChanged(); - return this; - } - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @param value The bytes for latestApiIp to set. - * @return This builder for chaining. - */ - public Builder setLatestApiIpBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - latestApiIp_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - - private java.lang.Object latestApiEndpoint_ = ""; - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @return The latestApiEndpoint. - */ - public java.lang.String getLatestApiEndpoint() { - java.lang.Object ref = latestApiEndpoint_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - latestApiEndpoint_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @return The bytes for latestApiEndpoint. - */ - public com.google.protobuf.ByteString - getLatestApiEndpointBytes() { - java.lang.Object ref = latestApiEndpoint_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - latestApiEndpoint_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @param value The latestApiEndpoint to set. - * @return This builder for chaining. - */ - public Builder setLatestApiEndpoint( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - latestApiEndpoint_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @return This builder for chaining. - */ - public Builder clearLatestApiEndpoint() { - latestApiEndpoint_ = getDefaultInstance().getLatestApiEndpoint(); - bitField0_ = (bitField0_ & ~0x00000010); - onChanged(); - return this; - } - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @param value The bytes for latestApiEndpoint to set. - * @return This builder for chaining. - */ - public Builder setLatestApiEndpointBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - latestApiEndpoint_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - - private java.lang.Object latestApiMethod_ = ""; - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @return The latestApiMethod. - */ - public java.lang.String getLatestApiMethod() { - java.lang.Object ref = latestApiMethod_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - latestApiMethod_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @return The bytes for latestApiMethod. - */ - public com.google.protobuf.ByteString - getLatestApiMethodBytes() { - java.lang.Object ref = latestApiMethod_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - latestApiMethod_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @param value The latestApiMethod to set. - * @return This builder for chaining. - */ - public Builder setLatestApiMethod( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - latestApiMethod_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @return This builder for chaining. - */ - public Builder clearLatestApiMethod() { - latestApiMethod_ = getDefaultInstance().getLatestApiMethod(); - bitField0_ = (bitField0_ & ~0x00000020); - onChanged(); - return this; - } - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @param value The bytes for latestApiMethod to set. - * @return This builder for chaining. - */ - public Builder setLatestApiMethodBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - latestApiMethod_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - - private int latestApiCollectionId_ ; - /** - * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; - * @return The latestApiCollectionId. - */ - @java.lang.Override - public int getLatestApiCollectionId() { - return latestApiCollectionId_; - } - /** - * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; - * @param value The latestApiCollectionId to set. - * @return This builder for chaining. - */ - public Builder setLatestApiCollectionId(int value) { - - latestApiCollectionId_ = value; - bitField0_ |= 0x00000040; - onChanged(); - return this; - } - /** - * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; - * @return This builder for chaining. - */ - public Builder clearLatestApiCollectionId() { - bitField0_ = (bitField0_ & ~0x00000040); - latestApiCollectionId_ = 0; - onChanged(); - return this; - } - - private java.lang.Object latestApiPayload_ = ""; - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @return The latestApiPayload. - */ - public java.lang.String getLatestApiPayload() { - java.lang.Object ref = latestApiPayload_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - latestApiPayload_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @return The bytes for latestApiPayload. - */ - public com.google.protobuf.ByteString - getLatestApiPayloadBytes() { - java.lang.Object ref = latestApiPayload_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - latestApiPayload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @param value The latestApiPayload to set. - * @return This builder for chaining. - */ - public Builder setLatestApiPayload( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - latestApiPayload_ = value; - bitField0_ |= 0x00000080; - onChanged(); - return this; - } - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @return This builder for chaining. - */ - public Builder clearLatestApiPayload() { - latestApiPayload_ = getDefaultInstance().getLatestApiPayload(); - bitField0_ = (bitField0_ & ~0x00000080); - onChanged(); - return this; - } - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @param value The bytes for latestApiPayload to set. - * @return This builder for chaining. - */ - public Builder setLatestApiPayloadBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - latestApiPayload_ = value; - bitField0_ |= 0x00000080; - onChanged(); - return this; - } - - private int eventType_ = 0; - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @return The enum numeric value on the wire for eventType. - */ - @java.lang.Override public int getEventTypeValue() { - return eventType_; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @param value The enum numeric value on the wire for eventType to set. - * @return This builder for chaining. - */ - public Builder setEventTypeValue(int value) { - eventType_ = value; - bitField0_ |= 0x00000100; - onChanged(); - return this; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @return The eventType. - */ - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType getEventType() { - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType result = com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.forNumber(eventType_); - return result == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType.UNRECOGNIZED : result; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @param value The eventType to set. - * @return This builder for chaining. - */ - public Builder setEventType(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000100; - eventType_ = value.getNumber(); - onChanged(); - return this; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @return This builder for chaining. - */ - public Builder clearEventType() { - bitField0_ = (bitField0_ & ~0x00000100); - eventType_ = 0; -||||||| parent of 36e13656a (refactor code) -======= - ip_ = ""; - url_ = ""; - method_ = ""; - payload_ = ""; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.class, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder.class); - } - - public static final int ACTOR_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - @java.lang.Override - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int IP_FIELD_NUMBER = 3; - @SuppressWarnings("serial") - private volatile java.lang.Object ip_ = ""; - /** - * string ip = 3 [json_name = "ip"]; - * @return The ip. - */ - @java.lang.Override - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } - } - /** - * string ip = 3 [json_name = "ip"]; - * @return The bytes for ip. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TIMESTAMP_FIELD_NUMBER = 4; - private long timestamp_ = 0L; - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - - public static final int URL_FIELD_NUMBER = 5; - @SuppressWarnings("serial") - private volatile java.lang.Object url_ = ""; - /** - * string url = 5 [json_name = "url"]; - * @return The url. - */ - @java.lang.Override - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } - } - /** - * string url = 5 [json_name = "url"]; - * @return The bytes for url. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int METHOD_FIELD_NUMBER = 6; - @SuppressWarnings("serial") - private volatile java.lang.Object method_ = ""; - /** - * string method = 6 [json_name = "method"]; - * @return The method. - */ - @java.lang.Override - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } - } - /** - * string method = 6 [json_name = "method"]; - * @return The bytes for method. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int API_COLLECTION_ID_FIELD_NUMBER = 7; - private int apiCollectionId_ = 0; - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - - public static final int PAYLOAD_FIELD_NUMBER = 8; - @SuppressWarnings("serial") - private volatile java.lang.Object payload_ = ""; - /** - * string payload = 8 [json_name = "payload"]; - * @return The payload. - */ - @java.lang.Override - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } - } - /** - * string payload = 8 [json_name = "payload"]; - * @return The bytes for payload. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 2, filterId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 3, ip_); - } - if (timestamp_ != 0L) { - output.writeInt64(4, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 5, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 6, method_); - } - if (apiCollectionId_ != 0) { - output.writeInt32(7, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 8, payload_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(actor_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, actor_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(2, filterId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(3, ip_); - } - if (timestamp_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(4, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(5, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(6, method_); - } - if (apiCollectionId_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(7, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(8, payload_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent other = (com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent) obj; - - if (!getActor() - .equals(other.getActor())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (!getIp() - .equals(other.getIp())) return false; - if (getTimestamp() - != other.getTimestamp()) return false; - if (!getUrl() - .equals(other.getUrl())) return false; - if (!getMethod() - .equals(other.getMethod())) return false; - if (getApiCollectionId() - != other.getApiCollectionId()) return false; - if (!getPayload() - .equals(other.getPayload())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + ACTOR_FIELD_NUMBER; - hash = (53 * hash) + getActor().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (37 * hash) + IP_FIELD_NUMBER; - hash = (53 * hash) + getIp().hashCode(); - hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getTimestamp()); - hash = (37 * hash) + URL_FIELD_NUMBER; - hash = (53 * hash) + getUrl().hashCode(); - hash = (37 * hash) + METHOD_FIELD_NUMBER; - hash = (53 * hash) + getMethod().hashCode(); - hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; - hash = (53 * hash) + getApiCollectionId(); - hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; - hash = (53 * hash) + getPayload().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.message.malicious_event.v1.MaliciousEvent} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.message.malicious_event.v1.MaliciousEvent) - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.class, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - actor_ = ""; - filterId_ = ""; - ip_ = ""; - timestamp_ = 0L; - url_ = ""; - method_ = ""; - apiCollectionId_ = 0; - payload_ = ""; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getDefaultInstanceForType() { - return com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent build() { - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent buildPartial() { - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result = new com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.actor_ = actor_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.filterId_ = filterId_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.ip_ = ip_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.timestamp_ = timestamp_; - } - if (((from_bitField0_ & 0x00000010) != 0)) { - result.url_ = url_; - } - if (((from_bitField0_ & 0x00000020) != 0)) { - result.method_ = method_; - } - if (((from_bitField0_ & 0x00000040) != 0)) { - result.apiCollectionId_ = apiCollectionId_; - } - if (((from_bitField0_ & 0x00000080) != 0)) { - result.payload_ = payload_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent) { - return mergeFrom((com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent other) { - if (other == com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance()) return this; - if (!other.getActor().isEmpty()) { - actor_ = other.actor_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000002; - onChanged(); - } - if (!other.getIp().isEmpty()) { - ip_ = other.ip_; - bitField0_ |= 0x00000004; - onChanged(); - } - if (other.getTimestamp() != 0L) { - setTimestamp(other.getTimestamp()); - } - if (!other.getUrl().isEmpty()) { - url_ = other.url_; - bitField0_ |= 0x00000010; - onChanged(); - } - if (!other.getMethod().isEmpty()) { - method_ = other.method_; - bitField0_ |= 0x00000020; - onChanged(); - } - if (other.getApiCollectionId() != 0) { - setApiCollectionId(other.getApiCollectionId()); - } - if (!other.getPayload().isEmpty()) { - payload_ = other.payload_; - bitField0_ |= 0x00000080; - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - actor_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000002; - break; - } // case 18 - case 26: { - ip_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000004; - break; - } // case 26 - case 32: { - timestamp_ = input.readInt64(); - bitField0_ |= 0x00000008; - break; - } // case 32 - case 42: { - url_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000010; - break; - } // case 42 - case 50: { - method_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000020; - break; - } // case 50 - case 56: { - apiCollectionId_ = input.readInt32(); - bitField0_ |= 0x00000040; - break; - } // case 56 - case 66: { - payload_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000080; - break; - } // case 66 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object actor_ = ""; - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - public java.lang.String getActor() { - java.lang.Object ref = actor_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - actor_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - public com.google.protobuf.ByteString - getActorBytes() { - java.lang.Object ref = actor_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - actor_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The actor to set. - * @return This builder for chaining. - */ - public Builder setActor( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @return This builder for chaining. - */ - public Builder clearActor() { - actor_ = getDefaultInstance().getActor(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string actor = 1 [json_name = "actor"]; - * @param value The bytes for actor to set. - * @return This builder for chaining. - */ - public Builder setActorBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - actor_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - /** - * string filter_id = 2 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - - private java.lang.Object ip_ = ""; - /** - * string ip = 3 [json_name = "ip"]; - * @return The ip. - */ - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string ip = 3 [json_name = "ip"]; - * @return The bytes for ip. - */ - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string ip = 3 [json_name = "ip"]; - * @param value The ip to set. - * @return This builder for chaining. - */ - public Builder setIp( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ip_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * string ip = 3 [json_name = "ip"]; - * @return This builder for chaining. - */ - public Builder clearIp() { - ip_ = getDefaultInstance().getIp(); - bitField0_ = (bitField0_ & ~0x00000004); - onChanged(); - return this; - } - /** - * string ip = 3 [json_name = "ip"]; - * @param value The bytes for ip to set. - * @return This builder for chaining. - */ - public Builder setIpBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ip_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - - private long timestamp_ ; - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @param value The timestamp to set. - * @return This builder for chaining. - */ - public Builder setTimestamp(long value) { - - timestamp_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @return This builder for chaining. - */ - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000008); - timestamp_ = 0L; - onChanged(); - return this; - } - - private java.lang.Object url_ = ""; - /** - * string url = 5 [json_name = "url"]; - * @return The url. - */ - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string url = 5 [json_name = "url"]; - * @return The bytes for url. - */ - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string url = 5 [json_name = "url"]; - * @param value The url to set. - * @return This builder for chaining. - */ - public Builder setUrl( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - url_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - /** - * string url = 5 [json_name = "url"]; - * @return This builder for chaining. - */ - public Builder clearUrl() { - url_ = getDefaultInstance().getUrl(); - bitField0_ = (bitField0_ & ~0x00000010); - onChanged(); - return this; - } - /** - * string url = 5 [json_name = "url"]; - * @param value The bytes for url to set. - * @return This builder for chaining. - */ - public Builder setUrlBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - url_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - - private java.lang.Object method_ = ""; - /** - * string method = 6 [json_name = "method"]; - * @return The method. - */ - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string method = 6 [json_name = "method"]; - * @return The bytes for method. - */ - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string method = 6 [json_name = "method"]; - * @param value The method to set. - * @return This builder for chaining. - */ - public Builder setMethod( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - method_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - /** - * string method = 6 [json_name = "method"]; - * @return This builder for chaining. - */ - public Builder clearMethod() { - method_ = getDefaultInstance().getMethod(); - bitField0_ = (bitField0_ & ~0x00000020); - onChanged(); - return this; - } - /** - * string method = 6 [json_name = "method"]; - * @param value The bytes for method to set. - * @return This builder for chaining. - */ - public Builder setMethodBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - method_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - - private int apiCollectionId_ ; - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @param value The apiCollectionId to set. - * @return This builder for chaining. - */ - public Builder setApiCollectionId(int value) { - - apiCollectionId_ = value; - bitField0_ |= 0x00000040; - onChanged(); - return this; - } - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @return This builder for chaining. - */ - public Builder clearApiCollectionId() { - bitField0_ = (bitField0_ & ~0x00000040); - apiCollectionId_ = 0; - onChanged(); - return this; - } - - private java.lang.Object payload_ = ""; - /** - * string payload = 8 [json_name = "payload"]; - * @return The payload. - */ - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string payload = 8 [json_name = "payload"]; - * @return The bytes for payload. - */ - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string payload = 8 [json_name = "payload"]; - * @param value The payload to set. - * @return This builder for chaining. - */ - public Builder setPayload( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - payload_ = value; - bitField0_ |= 0x00000080; - onChanged(); - return this; - } - /** - * string payload = 8 [json_name = "payload"]; - * @return This builder for chaining. - */ - public Builder clearPayload() { - payload_ = getDefaultInstance().getPayload(); - bitField0_ = (bitField0_ & ~0x00000080); - onChanged(); - return this; - } - /** - * string payload = 8 [json_name = "payload"]; - * @param value The bytes for payload to set. - * @return This builder for chaining. - */ - public Builder setPayloadBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - payload_ = value; - bitField0_ |= 0x00000080; ->>>>>>> 36e13656a (refactor code) - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.message.malicious_event.v1.MaliciousEvent) - } - - // @@protoc_insertion_point(class_scope:threat_protection.message.malicious_event.v1.MaliciousEvent) - private static final com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent(); - } - - public static com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public MaliciousEvent parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java deleted file mode 100644 index cd9548b517..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MaliciousEventOrBuilder.java +++ /dev/null @@ -1,168 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/malicious_event/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.malicious_event.v1; - -public interface MaliciousEventOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.message.malicious_event.v1.MaliciousEvent) - com.google.protobuf.MessageOrBuilder { - - /** - * string actor = 1 [json_name = "actor"]; - * @return The actor. - */ - java.lang.String getActor(); - /** - * string actor = 1 [json_name = "actor"]; - * @return The bytes for actor. - */ - com.google.protobuf.ByteString - getActorBytes(); - - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The filterId. - */ - java.lang.String getFilterId(); - /** - * string filter_id = 2 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - com.google.protobuf.ByteString - getFilterIdBytes(); - - /** -<<<<<<< HEAD - * int64 detected_at = 3 [json_name = "detectedAt"]; - * @return The detectedAt. - */ - long getDetectedAt(); - - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @return The latestApiIp. - */ - java.lang.String getLatestApiIp(); - /** - * string latest_api_ip = 4 [json_name = "latestApiIp"]; - * @return The bytes for latestApiIp. - */ - com.google.protobuf.ByteString - getLatestApiIpBytes(); - - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @return The latestApiEndpoint. - */ - java.lang.String getLatestApiEndpoint(); - /** - * string latest_api_endpoint = 5 [json_name = "latestApiEndpoint"]; - * @return The bytes for latestApiEndpoint. - */ - com.google.protobuf.ByteString - getLatestApiEndpointBytes(); - - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @return The latestApiMethod. - */ - java.lang.String getLatestApiMethod(); - /** - * string latest_api_method = 6 [json_name = "latestApiMethod"]; - * @return The bytes for latestApiMethod. - */ - com.google.protobuf.ByteString - getLatestApiMethodBytes(); - - /** - * int32 latest_api_collection_id = 7 [json_name = "latestApiCollectionId"]; - * @return The latestApiCollectionId. - */ - int getLatestApiCollectionId(); - - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @return The latestApiPayload. - */ - java.lang.String getLatestApiPayload(); - /** - * string latest_api_payload = 8 [json_name = "latestApiPayload"]; - * @return The bytes for latestApiPayload. - */ - com.google.protobuf.ByteString - getLatestApiPayloadBytes(); - - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @return The enum numeric value on the wire for eventType. - */ - int getEventTypeValue(); - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent.EventType event_type = 9 [json_name = "eventType"]; - * @return The eventType. - */ - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.EventType getEventType(); -||||||| parent of 36e13656a (refactor code) -======= - * string ip = 3 [json_name = "ip"]; - * @return The ip. - */ - java.lang.String getIp(); - /** - * string ip = 3 [json_name = "ip"]; - * @return The bytes for ip. - */ - com.google.protobuf.ByteString - getIpBytes(); - - /** - * int64 timestamp = 4 [json_name = "timestamp"]; - * @return The timestamp. - */ - long getTimestamp(); - - /** - * string url = 5 [json_name = "url"]; - * @return The url. - */ - java.lang.String getUrl(); - /** - * string url = 5 [json_name = "url"]; - * @return The bytes for url. - */ - com.google.protobuf.ByteString - getUrlBytes(); - - /** - * string method = 6 [json_name = "method"]; - * @return The method. - */ - java.lang.String getMethod(); - /** - * string method = 6 [json_name = "method"]; - * @return The bytes for method. - */ - com.google.protobuf.ByteString - getMethodBytes(); - - /** - * int32 api_collection_id = 7 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - int getApiCollectionId(); - - /** - * string payload = 8 [json_name = "payload"]; - * @return The payload. - */ - java.lang.String getPayload(); - /** - * string payload = 8 [json_name = "payload"]; - * @return The bytes for payload. - */ - com.google.protobuf.ByteString - getPayloadBytes(); ->>>>>>> 36e13656a (refactor code) -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java deleted file mode 100644 index 55a086f74c..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/malicious_event/v1/MessageProto.java +++ /dev/null @@ -1,110 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/malicious_event/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.malicious_event.v1; - -public final class MessageProto { - private MessageProto() {} - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - MessageProto.class.getName()); - } - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { - } - - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); - } - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n:threat_protection/message/malicious_ev" + - "ent/v1/message.proto\022,threat_protection." + -<<<<<<< HEAD - "message.malicious_event.v1\"\215\004\n\016Malicious" + - "Event\022\024\n\005actor\030\001 \001(\tR\005actor\022\033\n\tfilter_id" + - "\030\002 \001(\tR\010filterId\022\037\n\013detected_at\030\003 \001(\003R\nd" + - "etectedAt\022\"\n\rlatest_api_ip\030\004 \001(\tR\013latest" + - "ApiIp\022.\n\023latest_api_endpoint\030\005 \001(\tR\021late" + - "stApiEndpoint\022*\n\021latest_api_method\030\006 \001(\t" + - "R\017latestApiMethod\0227\n\030latest_api_collecti" + - "on_id\030\007 \001(\005R\025latestApiCollectionId\022,\n\022la" + - "test_api_payload\030\010 \001(\tR\020latestApiPayload" + - "\022e\n\nevent_type\030\t \001(\0162F.threat_protection" + - ".message.malicious_event.v1.MaliciousEve" + - "nt.EventTypeR\teventType\"Y\n\tEventType\022\032\n\026" + - "EVENT_TYPE_UNSPECIFIED\020\000\022\025\n\021EVENT_TYPE_S" + - "INGLE\020\001\022\031\n\025EVENT_TYPE_AGGREGATED\020\002B\226\002\n;c" + - "om.akto.proto.threat_protection.message." + - "malicious_event.v1B\014MessageProtoP\001\242\002\003TMM" + - "\252\002*ThreatProtection.Message.MaliciousEve" + - "nt.V1\312\002*ThreatProtection\\Message\\Malicio" + - "usEvent\\V1\342\0026ThreatProtection\\Message\\Ma" + - "liciousEvent\\V1\\GPBMetadata\352\002-ThreatProt" + - "ection::Message::MaliciousEvent::V1b\006pro" + - "to3" - }; - descriptor = com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - }); - internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor, - new java.lang.String[] { "Actor", "FilterId", "DetectedAt", "LatestApiIp", "LatestApiEndpoint", "LatestApiMethod", "LatestApiCollectionId", "LatestApiPayload", "EventType", }); -||||||| parent of 36e13656a (refactor code) -======= - "message.malicious_event.v1\"\341\001\n\016Malicious" + - "Event\022\024\n\005actor\030\001 \001(\tR\005actor\022\033\n\tfilter_id" + - "\030\002 \001(\tR\010filterId\022\016\n\002ip\030\003 \001(\tR\002ip\022\034\n\ttime" + - "stamp\030\004 \001(\003R\ttimestamp\022\020\n\003url\030\005 \001(\tR\003url" + - "\022\026\n\006method\030\006 \001(\tR\006method\022*\n\021api_collecti" + - "on_id\030\007 \001(\005R\017apiCollectionId\022\030\n\007payload\030" + - "\010 \001(\tR\007payloadB\226\002\n;com.akto.proto.threat" + - "_protection.message.malicious_event.v1B\014" + - "MessageProtoP\001\242\002\003TMM\252\002*ThreatProtection." + - "Message.MaliciousEvent.V1\312\002*ThreatProtec" + - "tion\\Message\\MaliciousEvent\\V1\342\0026ThreatP" + - "rotection\\Message\\MaliciousEvent\\V1\\GPBM" + - "etadata\352\002-ThreatProtection::Message::Mal" + - "iciousEvent::V1b\006proto3" - }; - descriptor = com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - }); - internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_threat_protection_message_malicious_event_v1_MaliciousEvent_descriptor, - new java.lang.String[] { "Actor", "FilterId", "Ip", "Timestamp", "Url", "Method", "ApiCollectionId", "Payload", }); ->>>>>>> 36e13656a (refactor code) - descriptor.resolveAllFeaturesImmutable(); - } - - // @@protoc_insertion_point(outer_class_scope) -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/MessageProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/MessageProto.java deleted file mode 100644 index 828d016696..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/MessageProto.java +++ /dev/null @@ -1,73 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/sample_request/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.sample_request.v1; - -public final class MessageProto { - private MessageProto() {} - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - MessageProto.class.getName()); - } - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { - } - - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); - } - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n9threat_protection/message/sample_reque" + - "st/v1/message.proto\022+threat_protection.m" + - "essage.sample_request.v1\"\323\001\n\026SampleMalic" + - "iousRequest\022\016\n\002ip\030\001 \001(\tR\002ip\022\034\n\ttimestamp" + - "\030\002 \001(\003R\ttimestamp\022\020\n\003url\030\003 \001(\tR\003url\022\026\n\006m" + - "ethod\030\004 \001(\tR\006method\022*\n\021api_collection_id" + - "\030\005 \001(\005R\017apiCollectionId\022\030\n\007payload\030\006 \001(\t" + - "R\007payload\022\033\n\tfilter_id\030\007 \001(\tR\010filterIdB\221" + - "\002\n:com.akto.proto.threat_protection.mess" + - "age.sample_request.v1B\014MessageProtoP\001\242\002\003" + - "TMS\252\002)ThreatProtection.Message.SampleReq" + - "uest.V1\312\002)ThreatProtection\\Message\\Sampl" + - "eRequest\\V1\342\0025ThreatProtection\\Message\\S" + - "ampleRequest\\V1\\GPBMetadata\352\002,ThreatProt" + - "ection::Message::SampleRequest::V1b\006prot" + - "o3" - }; - descriptor = com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - }); - internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor, - new java.lang.String[] { "Ip", "Timestamp", "Url", "Method", "ApiCollectionId", "Payload", "FilterId", }); - descriptor.resolveAllFeaturesImmutable(); - } - - // @@protoc_insertion_point(outer_class_scope) -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequest.java deleted file mode 100644 index c2cb45ef84..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequest.java +++ /dev/null @@ -1,1178 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/sample_request/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.sample_request.v1; - -/** - * Protobuf type {@code threat_protection.message.sample_request.v1.SampleMaliciousRequest} - */ -public final class SampleMaliciousRequest extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.message.sample_request.v1.SampleMaliciousRequest) - SampleMaliciousRequestOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - SampleMaliciousRequest.class.getName()); - } - // Use SampleMaliciousRequest.newBuilder() to construct. - private SampleMaliciousRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private SampleMaliciousRequest() { - ip_ = ""; - url_ = ""; - method_ = ""; - payload_ = ""; - filterId_ = ""; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.class, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder.class); - } - - public static final int IP_FIELD_NUMBER = 1; - @SuppressWarnings("serial") - private volatile java.lang.Object ip_ = ""; - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - @java.lang.Override - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TIMESTAMP_FIELD_NUMBER = 2; - private long timestamp_ = 0L; - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - - public static final int URL_FIELD_NUMBER = 3; - @SuppressWarnings("serial") - private volatile java.lang.Object url_ = ""; - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - @java.lang.Override - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } - } - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int METHOD_FIELD_NUMBER = 4; - @SuppressWarnings("serial") - private volatile java.lang.Object method_ = ""; - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - @java.lang.Override - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } - } - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int API_COLLECTION_ID_FIELD_NUMBER = 5; - private int apiCollectionId_ = 0; - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - - public static final int PAYLOAD_FIELD_NUMBER = 6; - @SuppressWarnings("serial") - private volatile java.lang.Object payload_ = ""; - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - @java.lang.Override - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int FILTER_ID_FIELD_NUMBER = 7; - @SuppressWarnings("serial") - private volatile java.lang.Object filterId_ = ""; - /** - * string filter_id = 7 [json_name = "filterId"]; - * @return The filterId. - */ - @java.lang.Override - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } - } - /** - * string filter_id = 7 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - @java.lang.Override - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 1, ip_); - } - if (timestamp_ != 0L) { - output.writeInt64(2, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 3, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 4, method_); - } - if (apiCollectionId_ != 0) { - output.writeInt32(5, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 6, payload_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - com.google.protobuf.GeneratedMessage.writeString(output, 7, filterId_); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(ip_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(1, ip_); - } - if (timestamp_ != 0L) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(2, timestamp_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(url_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(3, url_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(method_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(4, method_); - } - if (apiCollectionId_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(5, apiCollectionId_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(payload_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(6, payload_); - } - if (!com.google.protobuf.GeneratedMessage.isStringEmpty(filterId_)) { - size += com.google.protobuf.GeneratedMessage.computeStringSize(7, filterId_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest other = (com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest) obj; - - if (!getIp() - .equals(other.getIp())) return false; - if (getTimestamp() - != other.getTimestamp()) return false; - if (!getUrl() - .equals(other.getUrl())) return false; - if (!getMethod() - .equals(other.getMethod())) return false; - if (getApiCollectionId() - != other.getApiCollectionId()) return false; - if (!getPayload() - .equals(other.getPayload())) return false; - if (!getFilterId() - .equals(other.getFilterId())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (37 * hash) + IP_FIELD_NUMBER; - hash = (53 * hash) + getIp().hashCode(); - hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getTimestamp()); - hash = (37 * hash) + URL_FIELD_NUMBER; - hash = (53 * hash) + getUrl().hashCode(); - hash = (37 * hash) + METHOD_FIELD_NUMBER; - hash = (53 * hash) + getMethod().hashCode(); - hash = (37 * hash) + API_COLLECTION_ID_FIELD_NUMBER; - hash = (53 * hash) + getApiCollectionId(); - hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; - hash = (53 * hash) + getPayload().hashCode(); - hash = (37 * hash) + FILTER_ID_FIELD_NUMBER; - hash = (53 * hash) + getFilterId().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.message.sample_request.v1.SampleMaliciousRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.message.sample_request.v1.SampleMaliciousRequest) - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.class, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - ip_ = ""; - timestamp_ = 0L; - url_ = ""; - method_ = ""; - apiCollectionId_ = 0; - payload_ = ""; - filterId_ = ""; - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.internal_static_threat_protection_message_sample_request_v1_SampleMaliciousRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest build() { - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest buildPartial() { - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest result = new com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest(this); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartial0(com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest result) { - int from_bitField0_ = bitField0_; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.ip_ = ip_; - } - if (((from_bitField0_ & 0x00000002) != 0)) { - result.timestamp_ = timestamp_; - } - if (((from_bitField0_ & 0x00000004) != 0)) { - result.url_ = url_; - } - if (((from_bitField0_ & 0x00000008) != 0)) { - result.method_ = method_; - } - if (((from_bitField0_ & 0x00000010) != 0)) { - result.apiCollectionId_ = apiCollectionId_; - } - if (((from_bitField0_ & 0x00000020) != 0)) { - result.payload_ = payload_; - } - if (((from_bitField0_ & 0x00000040) != 0)) { - result.filterId_ = filterId_; - } - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest) { - return mergeFrom((com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest other) { - if (other == com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.getDefaultInstance()) return this; - if (!other.getIp().isEmpty()) { - ip_ = other.ip_; - bitField0_ |= 0x00000001; - onChanged(); - } - if (other.getTimestamp() != 0L) { - setTimestamp(other.getTimestamp()); - } - if (!other.getUrl().isEmpty()) { - url_ = other.url_; - bitField0_ |= 0x00000004; - onChanged(); - } - if (!other.getMethod().isEmpty()) { - method_ = other.method_; - bitField0_ |= 0x00000008; - onChanged(); - } - if (other.getApiCollectionId() != 0) { - setApiCollectionId(other.getApiCollectionId()); - } - if (!other.getPayload().isEmpty()) { - payload_ = other.payload_; - bitField0_ |= 0x00000020; - onChanged(); - } - if (!other.getFilterId().isEmpty()) { - filterId_ = other.filterId_; - bitField0_ |= 0x00000040; - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - ip_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 16: { - timestamp_ = input.readInt64(); - bitField0_ |= 0x00000002; - break; - } // case 16 - case 26: { - url_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000004; - break; - } // case 26 - case 34: { - method_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000008; - break; - } // case 34 - case 40: { - apiCollectionId_ = input.readInt32(); - bitField0_ |= 0x00000010; - break; - } // case 40 - case 50: { - payload_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000020; - break; - } // case 50 - case 58: { - filterId_ = input.readStringRequireUtf8(); - bitField0_ |= 0x00000040; - break; - } // case 58 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private java.lang.Object ip_ = ""; - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - public java.lang.String getIp() { - java.lang.Object ref = ip_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - ip_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - public com.google.protobuf.ByteString - getIpBytes() { - java.lang.Object ref = ip_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - ip_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string ip = 1 [json_name = "ip"]; - * @param value The ip to set. - * @return This builder for chaining. - */ - public Builder setIp( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - ip_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * string ip = 1 [json_name = "ip"]; - * @return This builder for chaining. - */ - public Builder clearIp() { - ip_ = getDefaultInstance().getIp(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - return this; - } - /** - * string ip = 1 [json_name = "ip"]; - * @param value The bytes for ip to set. - * @return This builder for chaining. - */ - public Builder setIpBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - ip_ = value; - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - - private long timestamp_ ; - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - @java.lang.Override - public long getTimestamp() { - return timestamp_; - } - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @param value The timestamp to set. - * @return This builder for chaining. - */ - public Builder setTimestamp(long value) { - - timestamp_ = value; - bitField0_ |= 0x00000002; - onChanged(); - return this; - } - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return This builder for chaining. - */ - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000002); - timestamp_ = 0L; - onChanged(); - return this; - } - - private java.lang.Object url_ = ""; - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - public java.lang.String getUrl() { - java.lang.Object ref = url_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - url_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - public com.google.protobuf.ByteString - getUrlBytes() { - java.lang.Object ref = url_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - url_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string url = 3 [json_name = "url"]; - * @param value The url to set. - * @return This builder for chaining. - */ - public Builder setUrl( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - url_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - /** - * string url = 3 [json_name = "url"]; - * @return This builder for chaining. - */ - public Builder clearUrl() { - url_ = getDefaultInstance().getUrl(); - bitField0_ = (bitField0_ & ~0x00000004); - onChanged(); - return this; - } - /** - * string url = 3 [json_name = "url"]; - * @param value The bytes for url to set. - * @return This builder for chaining. - */ - public Builder setUrlBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - url_ = value; - bitField0_ |= 0x00000004; - onChanged(); - return this; - } - - private java.lang.Object method_ = ""; - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - public java.lang.String getMethod() { - java.lang.Object ref = method_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - method_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - public com.google.protobuf.ByteString - getMethodBytes() { - java.lang.Object ref = method_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - method_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string method = 4 [json_name = "method"]; - * @param value The method to set. - * @return This builder for chaining. - */ - public Builder setMethod( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - method_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - /** - * string method = 4 [json_name = "method"]; - * @return This builder for chaining. - */ - public Builder clearMethod() { - method_ = getDefaultInstance().getMethod(); - bitField0_ = (bitField0_ & ~0x00000008); - onChanged(); - return this; - } - /** - * string method = 4 [json_name = "method"]; - * @param value The bytes for method to set. - * @return This builder for chaining. - */ - public Builder setMethodBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - method_ = value; - bitField0_ |= 0x00000008; - onChanged(); - return this; - } - - private int apiCollectionId_ ; - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - @java.lang.Override - public int getApiCollectionId() { - return apiCollectionId_; - } - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @param value The apiCollectionId to set. - * @return This builder for chaining. - */ - public Builder setApiCollectionId(int value) { - - apiCollectionId_ = value; - bitField0_ |= 0x00000010; - onChanged(); - return this; - } - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return This builder for chaining. - */ - public Builder clearApiCollectionId() { - bitField0_ = (bitField0_ & ~0x00000010); - apiCollectionId_ = 0; - onChanged(); - return this; - } - - private java.lang.Object payload_ = ""; - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - public java.lang.String getPayload() { - java.lang.Object ref = payload_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - payload_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - public com.google.protobuf.ByteString - getPayloadBytes() { - java.lang.Object ref = payload_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - payload_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string payload = 6 [json_name = "payload"]; - * @param value The payload to set. - * @return This builder for chaining. - */ - public Builder setPayload( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - payload_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - /** - * string payload = 6 [json_name = "payload"]; - * @return This builder for chaining. - */ - public Builder clearPayload() { - payload_ = getDefaultInstance().getPayload(); - bitField0_ = (bitField0_ & ~0x00000020); - onChanged(); - return this; - } - /** - * string payload = 6 [json_name = "payload"]; - * @param value The bytes for payload to set. - * @return This builder for chaining. - */ - public Builder setPayloadBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - payload_ = value; - bitField0_ |= 0x00000020; - onChanged(); - return this; - } - - private java.lang.Object filterId_ = ""; - /** - * string filter_id = 7 [json_name = "filterId"]; - * @return The filterId. - */ - public java.lang.String getFilterId() { - java.lang.Object ref = filterId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - filterId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string filter_id = 7 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - public com.google.protobuf.ByteString - getFilterIdBytes() { - java.lang.Object ref = filterId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - filterId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string filter_id = 7 [json_name = "filterId"]; - * @param value The filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterId( - java.lang.String value) { - if (value == null) { throw new NullPointerException(); } - filterId_ = value; - bitField0_ |= 0x00000040; - onChanged(); - return this; - } - /** - * string filter_id = 7 [json_name = "filterId"]; - * @return This builder for chaining. - */ - public Builder clearFilterId() { - filterId_ = getDefaultInstance().getFilterId(); - bitField0_ = (bitField0_ & ~0x00000040); - onChanged(); - return this; - } - /** - * string filter_id = 7 [json_name = "filterId"]; - * @param value The bytes for filterId to set. - * @return This builder for chaining. - */ - public Builder setFilterIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { throw new NullPointerException(); } - checkByteStringIsUtf8(value); - filterId_ = value; - bitField0_ |= 0x00000040; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.message.sample_request.v1.SampleMaliciousRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.message.sample_request.v1.SampleMaliciousRequest) - private static final com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest(); - } - - public static com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SampleMaliciousRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequestOrBuilder.java deleted file mode 100644 index 5080d4b4ae..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/message/sample_request/v1/SampleMaliciousRequestOrBuilder.java +++ /dev/null @@ -1,83 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/message/sample_request/v1/message.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.message.sample_request.v1; - -public interface SampleMaliciousRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.message.sample_request.v1.SampleMaliciousRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * string ip = 1 [json_name = "ip"]; - * @return The ip. - */ - java.lang.String getIp(); - /** - * string ip = 1 [json_name = "ip"]; - * @return The bytes for ip. - */ - com.google.protobuf.ByteString - getIpBytes(); - - /** - * int64 timestamp = 2 [json_name = "timestamp"]; - * @return The timestamp. - */ - long getTimestamp(); - - /** - * string url = 3 [json_name = "url"]; - * @return The url. - */ - java.lang.String getUrl(); - /** - * string url = 3 [json_name = "url"]; - * @return The bytes for url. - */ - com.google.protobuf.ByteString - getUrlBytes(); - - /** - * string method = 4 [json_name = "method"]; - * @return The method. - */ - java.lang.String getMethod(); - /** - * string method = 4 [json_name = "method"]; - * @return The bytes for method. - */ - com.google.protobuf.ByteString - getMethodBytes(); - - /** - * int32 api_collection_id = 5 [json_name = "apiCollectionId"]; - * @return The apiCollectionId. - */ - int getApiCollectionId(); - - /** - * string payload = 6 [json_name = "payload"]; - * @return The payload. - */ - java.lang.String getPayload(); - /** - * string payload = 6 [json_name = "payload"]; - * @return The bytes for payload. - */ - com.google.protobuf.ByteString - getPayloadBytes(); - - /** - * string filter_id = 7 [json_name = "filterId"]; - * @return The filterId. - */ - java.lang.String getFilterId(); - /** - * string filter_id = 7 [json_name = "filterId"]; - * @return The bytes for filterId. - */ - com.google.protobuf.ByteString - getFilterIdBytes(); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousEventServiceGrpc.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousEventServiceGrpc.java deleted file mode 100644 index 8056585eae..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/MaliciousEventServiceGrpc.java +++ /dev/null @@ -1,293 +0,0 @@ -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -import static io.grpc.MethodDescriptor.generateFullMethodName; - -/** - */ -@javax.annotation.Generated( - value = "by gRPC proto compiler (version 1.68.1)", - comments = "Source: threat_protection/service/malicious_alert_service/v1/service.proto") -@io.grpc.stub.annotations.GrpcGenerated -public final class MaliciousEventServiceGrpc { - - private MaliciousEventServiceGrpc() {} - - public static final java.lang.String SERVICE_NAME = "threat_protection.service.malicious_alert_service.v1.MaliciousEventService"; - - // Static method descriptors that strictly reflect the proto. - private static volatile io.grpc.MethodDescriptor getRecordMaliciousEventMethod; - - @io.grpc.stub.annotations.RpcMethod( - fullMethodName = SERVICE_NAME + '/' + "RecordMaliciousEvent", - requestType = com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.class, - responseType = com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.class, - methodType = io.grpc.MethodDescriptor.MethodType.UNARY) - public static io.grpc.MethodDescriptor getRecordMaliciousEventMethod() { - io.grpc.MethodDescriptor getRecordMaliciousEventMethod; - if ((getRecordMaliciousEventMethod = MaliciousEventServiceGrpc.getRecordMaliciousEventMethod) == null) { - synchronized (MaliciousEventServiceGrpc.class) { - if ((getRecordMaliciousEventMethod = MaliciousEventServiceGrpc.getRecordMaliciousEventMethod) == null) { - MaliciousEventServiceGrpc.getRecordMaliciousEventMethod = getRecordMaliciousEventMethod = - io.grpc.MethodDescriptor.newBuilder() - .setType(io.grpc.MethodDescriptor.MethodType.UNARY) - .setFullMethodName(generateFullMethodName(SERVICE_NAME, "RecordMaliciousEvent")) - .setSampledToLocalTracing(true) - .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.getDefaultInstance())) - .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.getDefaultInstance())) - .setSchemaDescriptor(new MaliciousEventServiceMethodDescriptorSupplier("RecordMaliciousEvent")) - .build(); - } - } - } - return getRecordMaliciousEventMethod; - } - - /** - * Creates a new async stub that supports all call types for the service - */ - public static MaliciousEventServiceStub newStub(io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public MaliciousEventServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousEventServiceStub(channel, callOptions); - } - }; - return MaliciousEventServiceStub.newStub(factory, channel); - } - - /** - * Creates a new blocking-style stub that supports unary and streaming output calls on the service - */ - public static MaliciousEventServiceBlockingStub newBlockingStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public MaliciousEventServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousEventServiceBlockingStub(channel, callOptions); - } - }; - return MaliciousEventServiceBlockingStub.newStub(factory, channel); - } - - /** - * Creates a new ListenableFuture-style stub that supports unary calls on the service - */ - public static MaliciousEventServiceFutureStub newFutureStub( - io.grpc.Channel channel) { - io.grpc.stub.AbstractStub.StubFactory factory = - new io.grpc.stub.AbstractStub.StubFactory() { - @java.lang.Override - public MaliciousEventServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousEventServiceFutureStub(channel, callOptions); - } - }; - return MaliciousEventServiceFutureStub.newStub(factory, channel); - } - - /** - */ - public interface AsyncService { - - /** - */ - default void recordMaliciousEvent(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getRecordMaliciousEventMethod(), responseObserver); - } - } - - /** - * Base class for the server implementation of the service MaliciousEventService. - */ - public static abstract class MaliciousEventServiceImplBase - implements io.grpc.BindableService, AsyncService { - - @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { - return MaliciousEventServiceGrpc.bindService(this); - } - } - - /** - * A stub to allow clients to do asynchronous rpc calls to service MaliciousEventService. - */ - public static final class MaliciousEventServiceStub - extends io.grpc.stub.AbstractAsyncStub { - private MaliciousEventServiceStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected MaliciousEventServiceStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousEventServiceStub(channel, callOptions); - } - - /** - */ - public void recordMaliciousEvent(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest request, - io.grpc.stub.StreamObserver responseObserver) { - io.grpc.stub.ClientCalls.asyncUnaryCall( - getChannel().newCall(getRecordMaliciousEventMethod(), getCallOptions()), request, responseObserver); - } - } - - /** - * A stub to allow clients to do synchronous rpc calls to service MaliciousEventService. - */ - public static final class MaliciousEventServiceBlockingStub - extends io.grpc.stub.AbstractBlockingStub { - private MaliciousEventServiceBlockingStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected MaliciousEventServiceBlockingStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousEventServiceBlockingStub(channel, callOptions); - } - - /** - */ - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse recordMaliciousEvent(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest request) { - return io.grpc.stub.ClientCalls.blockingUnaryCall( - getChannel(), getRecordMaliciousEventMethod(), getCallOptions(), request); - } - } - - /** - * A stub to allow clients to do ListenableFuture-style rpc calls to service MaliciousEventService. - */ - public static final class MaliciousEventServiceFutureStub - extends io.grpc.stub.AbstractFutureStub { - private MaliciousEventServiceFutureStub( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - super(channel, callOptions); - } - - @java.lang.Override - protected MaliciousEventServiceFutureStub build( - io.grpc.Channel channel, io.grpc.CallOptions callOptions) { - return new MaliciousEventServiceFutureStub(channel, callOptions); - } - - /** - */ - public com.google.common.util.concurrent.ListenableFuture recordMaliciousEvent( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest request) { - return io.grpc.stub.ClientCalls.futureUnaryCall( - getChannel().newCall(getRecordMaliciousEventMethod(), getCallOptions()), request); - } - } - - private static final int METHODID_RECORD_MALICIOUS_EVENT = 0; - - private static final class MethodHandlers implements - io.grpc.stub.ServerCalls.UnaryMethod, - io.grpc.stub.ServerCalls.ServerStreamingMethod, - io.grpc.stub.ServerCalls.ClientStreamingMethod, - io.grpc.stub.ServerCalls.BidiStreamingMethod { - private final AsyncService serviceImpl; - private final int methodId; - - MethodHandlers(AsyncService serviceImpl, int methodId) { - this.serviceImpl = serviceImpl; - this.methodId = methodId; - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - case METHODID_RECORD_MALICIOUS_EVENT: - serviceImpl.recordMaliciousEvent((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) request, - (io.grpc.stub.StreamObserver) responseObserver); - break; - default: - throw new AssertionError(); - } - } - - @java.lang.Override - @java.lang.SuppressWarnings("unchecked") - public io.grpc.stub.StreamObserver invoke( - io.grpc.stub.StreamObserver responseObserver) { - switch (methodId) { - default: - throw new AssertionError(); - } - } - } - - public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { - return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) - .addMethod( - getRecordMaliciousEventMethod(), - io.grpc.stub.ServerCalls.asyncUnaryCall( - new MethodHandlers< - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest, - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse>( - service, METHODID_RECORD_MALICIOUS_EVENT))) - .build(); - } - - private static abstract class MaliciousEventServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { - MaliciousEventServiceBaseDescriptorSupplier() {} - - @java.lang.Override - public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.getDescriptor(); - } - - @java.lang.Override - public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { - return getFileDescriptor().findServiceByName("MaliciousEventService"); - } - } - - private static final class MaliciousEventServiceFileDescriptorSupplier - extends MaliciousEventServiceBaseDescriptorSupplier { - MaliciousEventServiceFileDescriptorSupplier() {} - } - - private static final class MaliciousEventServiceMethodDescriptorSupplier - extends MaliciousEventServiceBaseDescriptorSupplier - implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { - private final java.lang.String methodName; - - MaliciousEventServiceMethodDescriptorSupplier(java.lang.String methodName) { - this.methodName = methodName; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { - return getServiceDescriptor().findMethodByName(methodName); - } - } - - private static volatile io.grpc.ServiceDescriptor serviceDescriptor; - - public static io.grpc.ServiceDescriptor getServiceDescriptor() { - io.grpc.ServiceDescriptor result = serviceDescriptor; - if (result == null) { - synchronized (MaliciousEventServiceGrpc.class) { - result = serviceDescriptor; - if (result == null) { - serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) - .setSchemaDescriptor(new MaliciousEventServiceFileDescriptorSupplier()) - .addMethod(getRecordMaliciousEventMethod()) - .build(); - } - } - } - return result; - } -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequest.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequest.java deleted file mode 100644 index 08df258112..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequest.java +++ /dev/null @@ -1,913 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -/** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest} - */ -public final class RecordMaliciousEventRequest extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) - RecordMaliciousEventRequestOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - RecordMaliciousEventRequest.class.getName()); - } - // Use RecordMaliciousEventRequest.newBuilder() to construct. - private RecordMaliciousEventRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private RecordMaliciousEventRequest() { - sampleRequests_ = java.util.Collections.emptyList(); - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.Builder.class); - } - - private int bitField0_; - public static final int MALICIOUS_EVENT_FIELD_NUMBER = 1; - private com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent maliciousEvent_; - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - * @return Whether the maliciousEvent field is set. - */ - @java.lang.Override - public boolean hasMaliciousEvent() { - return ((bitField0_ & 0x00000001) != 0); - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - * @return The maliciousEvent. - */ - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getMaliciousEvent() { - return maliciousEvent_ == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance() : maliciousEvent_; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder getMaliciousEventOrBuilder() { - return maliciousEvent_ == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance() : maliciousEvent_; - } - - public static final int SAMPLE_REQUESTS_FIELD_NUMBER = 2; - @SuppressWarnings("serial") - private java.util.List sampleRequests_; - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - @java.lang.Override - public java.util.List getSampleRequestsList() { - return sampleRequests_; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - @java.lang.Override - public java.util.List - getSampleRequestsOrBuilderList() { - return sampleRequests_; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - @java.lang.Override - public int getSampleRequestsCount() { - return sampleRequests_.size(); - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getSampleRequests(int index) { - return sampleRequests_.get(index); - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - @java.lang.Override - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder getSampleRequestsOrBuilder( - int index) { - return sampleRequests_.get(index); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - if (((bitField0_ & 0x00000001) != 0)) { - output.writeMessage(1, getMaliciousEvent()); - } - for (int i = 0; i < sampleRequests_.size(); i++) { - output.writeMessage(2, sampleRequests_.get(i)); - } - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (((bitField0_ & 0x00000001) != 0)) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getMaliciousEvent()); - } - for (int i = 0; i < sampleRequests_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, sampleRequests_.get(i)); - } - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) obj; - - if (hasMaliciousEvent() != other.hasMaliciousEvent()) return false; - if (hasMaliciousEvent()) { - if (!getMaliciousEvent() - .equals(other.getMaliciousEvent())) return false; - } - if (!getSampleRequestsList() - .equals(other.getSampleRequestsList())) return false; - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - if (hasMaliciousEvent()) { - hash = (37 * hash) + MALICIOUS_EVENT_FIELD_NUMBER; - hash = (53 * hash) + getMaliciousEvent().hashCode(); - } - if (getSampleRequestsCount() > 0) { - hash = (37 * hash) + SAMPLE_REQUESTS_FIELD_NUMBER; - hash = (53 * hash) + getSampleRequestsList().hashCode(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage - .alwaysUseFieldBuilders) { - getMaliciousEventFieldBuilder(); - getSampleRequestsFieldBuilder(); - } - } - @java.lang.Override - public Builder clear() { - super.clear(); - bitField0_ = 0; - maliciousEvent_ = null; - if (maliciousEventBuilder_ != null) { - maliciousEventBuilder_.dispose(); - maliciousEventBuilder_ = null; - } - if (sampleRequestsBuilder_ == null) { - sampleRequests_ = java.util.Collections.emptyList(); - } else { - sampleRequests_ = null; - sampleRequestsBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest build() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest buildPartial() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest(this); - buildPartialRepeatedFields(result); - if (bitField0_ != 0) { buildPartial0(result); } - onBuilt(); - return result; - } - - private void buildPartialRepeatedFields(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest result) { - if (sampleRequestsBuilder_ == null) { - if (((bitField0_ & 0x00000002) != 0)) { - sampleRequests_ = java.util.Collections.unmodifiableList(sampleRequests_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.sampleRequests_ = sampleRequests_; - } else { - result.sampleRequests_ = sampleRequestsBuilder_.build(); - } - } - - private void buildPartial0(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest result) { - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) != 0)) { - result.maliciousEvent_ = maliciousEventBuilder_ == null - ? maliciousEvent_ - : maliciousEventBuilder_.build(); - to_bitField0_ |= 0x00000001; - } - result.bitField0_ |= to_bitField0_; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) { - return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest other) { - if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest.getDefaultInstance()) return this; - if (other.hasMaliciousEvent()) { - mergeMaliciousEvent(other.getMaliciousEvent()); - } - if (sampleRequestsBuilder_ == null) { - if (!other.sampleRequests_.isEmpty()) { - if (sampleRequests_.isEmpty()) { - sampleRequests_ = other.sampleRequests_; - bitField0_ = (bitField0_ & ~0x00000002); - } else { - ensureSampleRequestsIsMutable(); - sampleRequests_.addAll(other.sampleRequests_); - } - onChanged(); - } - } else { - if (!other.sampleRequests_.isEmpty()) { - if (sampleRequestsBuilder_.isEmpty()) { - sampleRequestsBuilder_.dispose(); - sampleRequestsBuilder_ = null; - sampleRequests_ = other.sampleRequests_; - bitField0_ = (bitField0_ & ~0x00000002); - sampleRequestsBuilder_ = - com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getSampleRequestsFieldBuilder() : null; - } else { - sampleRequestsBuilder_.addAllMessages(other.sampleRequests_); - } - } - } - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - input.readMessage( - getMaliciousEventFieldBuilder().getBuilder(), - extensionRegistry); - bitField0_ |= 0x00000001; - break; - } // case 10 - case 18: { - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest m = - input.readMessage( - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.parser(), - extensionRegistry); - if (sampleRequestsBuilder_ == null) { - ensureSampleRequestsIsMutable(); - sampleRequests_.add(m); - } else { - sampleRequestsBuilder_.addMessage(m); - } - break; - } // case 18 - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - private int bitField0_; - - private com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent maliciousEvent_; - private com.google.protobuf.SingleFieldBuilder< - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder> maliciousEventBuilder_; - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - * @return Whether the maliciousEvent field is set. - */ - public boolean hasMaliciousEvent() { - return ((bitField0_ & 0x00000001) != 0); - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - * @return The maliciousEvent. - */ - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getMaliciousEvent() { - if (maliciousEventBuilder_ == null) { - return maliciousEvent_ == null ? com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance() : maliciousEvent_; - } else { - return maliciousEventBuilder_.getMessage(); - } - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - public Builder setMaliciousEvent(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent value) { - if (maliciousEventBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - maliciousEvent_ = value; - } else { - maliciousEventBuilder_.setMessage(value); - } - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - public Builder setMaliciousEvent( - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder builderForValue) { - if (maliciousEventBuilder_ == null) { - maliciousEvent_ = builderForValue.build(); - } else { - maliciousEventBuilder_.setMessage(builderForValue.build()); - } - bitField0_ |= 0x00000001; - onChanged(); - return this; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - public Builder mergeMaliciousEvent(com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent value) { - if (maliciousEventBuilder_ == null) { - if (((bitField0_ & 0x00000001) != 0) && - maliciousEvent_ != null && - maliciousEvent_ != com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance()) { - getMaliciousEventBuilder().mergeFrom(value); - } else { - maliciousEvent_ = value; - } - } else { - maliciousEventBuilder_.mergeFrom(value); - } - if (maliciousEvent_ != null) { - bitField0_ |= 0x00000001; - onChanged(); - } - return this; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - public Builder clearMaliciousEvent() { - bitField0_ = (bitField0_ & ~0x00000001); - maliciousEvent_ = null; - if (maliciousEventBuilder_ != null) { - maliciousEventBuilder_.dispose(); - maliciousEventBuilder_ = null; - } - onChanged(); - return this; - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder getMaliciousEventBuilder() { - bitField0_ |= 0x00000001; - onChanged(); - return getMaliciousEventFieldBuilder().getBuilder(); - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - public com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder getMaliciousEventOrBuilder() { - if (maliciousEventBuilder_ != null) { - return maliciousEventBuilder_.getMessageOrBuilder(); - } else { - return maliciousEvent_ == null ? - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.getDefaultInstance() : maliciousEvent_; - } - } - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - private com.google.protobuf.SingleFieldBuilder< - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder> - getMaliciousEventFieldBuilder() { - if (maliciousEventBuilder_ == null) { - maliciousEventBuilder_ = new com.google.protobuf.SingleFieldBuilder< - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent.Builder, com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder>( - getMaliciousEvent(), - getParentForChildren(), - isClean()); - maliciousEvent_ = null; - } - return maliciousEventBuilder_; - } - - private java.util.List sampleRequests_ = - java.util.Collections.emptyList(); - private void ensureSampleRequestsIsMutable() { - if (!((bitField0_ & 0x00000002) != 0)) { - sampleRequests_ = new java.util.ArrayList(sampleRequests_); - bitField0_ |= 0x00000002; - } - } - - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder> sampleRequestsBuilder_; - - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public java.util.List getSampleRequestsList() { - if (sampleRequestsBuilder_ == null) { - return java.util.Collections.unmodifiableList(sampleRequests_); - } else { - return sampleRequestsBuilder_.getMessageList(); - } - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public int getSampleRequestsCount() { - if (sampleRequestsBuilder_ == null) { - return sampleRequests_.size(); - } else { - return sampleRequestsBuilder_.getCount(); - } - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getSampleRequests(int index) { - if (sampleRequestsBuilder_ == null) { - return sampleRequests_.get(index); - } else { - return sampleRequestsBuilder_.getMessage(index); - } - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder setSampleRequests( - int index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest value) { - if (sampleRequestsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleRequestsIsMutable(); - sampleRequests_.set(index, value); - onChanged(); - } else { - sampleRequestsBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder setSampleRequests( - int index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder builderForValue) { - if (sampleRequestsBuilder_ == null) { - ensureSampleRequestsIsMutable(); - sampleRequests_.set(index, builderForValue.build()); - onChanged(); - } else { - sampleRequestsBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder addSampleRequests(com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest value) { - if (sampleRequestsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleRequestsIsMutable(); - sampleRequests_.add(value); - onChanged(); - } else { - sampleRequestsBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder addSampleRequests( - int index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest value) { - if (sampleRequestsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureSampleRequestsIsMutable(); - sampleRequests_.add(index, value); - onChanged(); - } else { - sampleRequestsBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder addSampleRequests( - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder builderForValue) { - if (sampleRequestsBuilder_ == null) { - ensureSampleRequestsIsMutable(); - sampleRequests_.add(builderForValue.build()); - onChanged(); - } else { - sampleRequestsBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder addSampleRequests( - int index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder builderForValue) { - if (sampleRequestsBuilder_ == null) { - ensureSampleRequestsIsMutable(); - sampleRequests_.add(index, builderForValue.build()); - onChanged(); - } else { - sampleRequestsBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder addAllSampleRequests( - java.lang.Iterable values) { - if (sampleRequestsBuilder_ == null) { - ensureSampleRequestsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, sampleRequests_); - onChanged(); - } else { - sampleRequestsBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder clearSampleRequests() { - if (sampleRequestsBuilder_ == null) { - sampleRequests_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - } else { - sampleRequestsBuilder_.clear(); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public Builder removeSampleRequests(int index) { - if (sampleRequestsBuilder_ == null) { - ensureSampleRequestsIsMutable(); - sampleRequests_.remove(index); - onChanged(); - } else { - sampleRequestsBuilder_.remove(index); - } - return this; - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder getSampleRequestsBuilder( - int index) { - return getSampleRequestsFieldBuilder().getBuilder(index); - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder getSampleRequestsOrBuilder( - int index) { - if (sampleRequestsBuilder_ == null) { - return sampleRequests_.get(index); } else { - return sampleRequestsBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public java.util.List - getSampleRequestsOrBuilderList() { - if (sampleRequestsBuilder_ != null) { - return sampleRequestsBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(sampleRequests_); - } - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder addSampleRequestsBuilder() { - return getSampleRequestsFieldBuilder().addBuilder( - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.getDefaultInstance()); - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder addSampleRequestsBuilder( - int index) { - return getSampleRequestsFieldBuilder().addBuilder( - index, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.getDefaultInstance()); - } - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - public java.util.List - getSampleRequestsBuilderList() { - return getSampleRequestsFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder> - getSampleRequestsFieldBuilder() { - if (sampleRequestsBuilder_ == null) { - sampleRequestsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest.Builder, com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder>( - sampleRequests_, - ((bitField0_ & 0x00000002) != 0), - getParentForChildren(), - isClean()); - sampleRequests_ = null; - } - return sampleRequestsBuilder_; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) - private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest(); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RecordMaliciousEventRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequestOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequestOrBuilder.java deleted file mode 100644 index b36835dcd6..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventRequestOrBuilder.java +++ /dev/null @@ -1,50 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -public interface RecordMaliciousEventRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - * @return Whether the maliciousEvent field is set. - */ - boolean hasMaliciousEvent(); - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - * @return The maliciousEvent. - */ - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent getMaliciousEvent(); - /** - * .threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1 [json_name = "maliciousEvent"]; - */ - com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventOrBuilder getMaliciousEventOrBuilder(); - - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - java.util.List - getSampleRequestsList(); - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest getSampleRequests(int index); - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - int getSampleRequestsCount(); - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - java.util.List - getSampleRequestsOrBuilderList(); - /** - * repeated .threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2 [json_name = "sampleRequests"]; - */ - com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequestOrBuilder getSampleRequestsOrBuilder( - int index); -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponse.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponse.java deleted file mode 100644 index 5e58989b1a..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponse.java +++ /dev/null @@ -1,358 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -/** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse} - */ -public final class RecordMaliciousEventResponse extends - com.google.protobuf.GeneratedMessage implements - // @@protoc_insertion_point(message_implements:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) - RecordMaliciousEventResponseOrBuilder { -private static final long serialVersionUID = 0L; - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - RecordMaliciousEventResponse.class.getName()); - } - // Use RecordMaliciousEventResponse.newBuilder() to construct. - private RecordMaliciousEventResponse(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - } - private RecordMaliciousEventResponse() { - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.Builder.class); - } - - private byte memoizedIsInitialized = -1; - @java.lang.Override - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized == 1) return true; - if (isInitialized == 0) return false; - - memoizedIsInitialized = 1; - return true; - } - - @java.lang.Override - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getUnknownFields().writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - size += getUnknownFields().getSerializedSize(); - memoizedSize = size; - return size; - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse)) { - return super.equals(obj); - } - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse other = (com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) obj; - - if (!getUnknownFields().equals(other.getUnknownFields())) return false; - return true; - } - - @java.lang.Override - public int hashCode() { - if (memoizedHashCode != 0) { - return memoizedHashCode; - } - int hash = 41; - hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseDelimitedWithIOException(PARSER, input, extensionRegistry); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input); - } - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessage - .parseWithIOException(PARSER, input, extensionRegistry); - } - - @java.lang.Override - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder() { - return DEFAULT_INSTANCE.toBuilder(); - } - public static Builder newBuilder(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse prototype) { - return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); - } - @java.lang.Override - public Builder toBuilder() { - return this == DEFAULT_INSTANCE - ? new Builder() : new Builder().mergeFrom(this); - } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder implements - // @@protoc_insertion_point(builder_implements:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.class, com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.Builder.class); - } - - // Construct using com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.newBuilder() - private Builder() { - - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - - } - @java.lang.Override - public Builder clear() { - super.clear(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.ServiceProto.internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse getDefaultInstanceForType() { - return com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.getDefaultInstance(); - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse build() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse buildPartial() { - com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse result = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse(this); - onBuilt(); - return result; - } - - @java.lang.Override - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) { - return mergeFrom((com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse other) { - if (other == com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse.getDefaultInstance()) return this; - this.mergeUnknownFields(other.getUnknownFields()); - onChanged(); - return this; - } - - @java.lang.Override - public final boolean isInitialized() { - return true; - } - - @java.lang.Override - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!super.parseUnknownField(input, extensionRegistry, tag)) { - done = true; // was an endgroup tag - } - break; - } // default: - } // switch (tag) - } // while (!done) - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.unwrapIOException(); - } finally { - onChanged(); - } // finally - return this; - } - - // @@protoc_insertion_point(builder_scope:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) - } - - // @@protoc_insertion_point(class_scope:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) - private static final com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse(); - } - - public static com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RecordMaliciousEventResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - Builder builder = newBuilder(); - try { - builder.mergeFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(builder.buildPartial()); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException(e) - .setUnfinishedMessage(builder.buildPartial()); - } - return builder.buildPartial(); - } - }; - - public static com.google.protobuf.Parser parser() { - return PARSER; - } - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - @java.lang.Override - public com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - -} - diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponseOrBuilder.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponseOrBuilder.java deleted file mode 100644 index b9dc97a1dc..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/RecordMaliciousEventResponseOrBuilder.java +++ /dev/null @@ -1,11 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -public interface RecordMaliciousEventResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse) - com.google.protobuf.MessageOrBuilder { -} diff --git a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ServiceProto.java b/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ServiceProto.java deleted file mode 100644 index a97ab01ca9..0000000000 --- a/libs/protobuf/src/main/java/com/akto/proto/threat_protection/service/malicious_alert_service/v1/ServiceProto.java +++ /dev/null @@ -1,99 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// NO CHECKED-IN PROTOBUF GENCODE -// source: threat_protection/service/malicious_alert_service/v1/service.proto -// Protobuf Java Version: 4.28.3 - -package com.akto.proto.threat_protection.service.malicious_alert_service.v1; - -public final class ServiceProto { - private ServiceProto() {} - static { - com.google.protobuf.RuntimeVersion.validateProtobufGencodeVersion( - com.google.protobuf.RuntimeVersion.RuntimeDomain.PUBLIC, - /* major= */ 4, - /* minor= */ 28, - /* patch= */ 3, - /* suffix= */ "", - ServiceProto.class.getName()); - } - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { - } - - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); - } - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_fieldAccessorTable; - static final com.google.protobuf.Descriptors.Descriptor - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor; - static final - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\nBthreat_protection/service/malicious_al" + - "ert_service/v1/service.proto\0224threat_pro" + - "tection.service.malicious_alert_service." + - "v1\032:threat_protection/message/malicious_" + - "event/v1/message.proto\0329threat_protectio" + - "n/message/sample_request/v1/message.prot" + - "o\"\036\n\034RecordMaliciousEventResponse\"\362\001\n\033Re" + - "cordMaliciousEventRequest\022e\n\017malicious_e" + - "vent\030\001 \001(\0132<.threat_protection.message.m" + - "alicious_event.v1.MaliciousEventR\016malici" + - "ousEvent\022l\n\017sample_requests\030\002 \003(\0132C.thre" + - "at_protection.message.sample_request.v1." + - "SampleMaliciousRequestR\016sampleRequests2\331" + - "\001\n\025MaliciousEventService\022\277\001\n\024RecordMalic" + - "iousEvent\022Q.threat_protection.service.ma" + - "licious_alert_service.v1.RecordMalicious" + - "EventRequest\032R.threat_protection.service" + - ".malicious_alert_service.v1.RecordMalici" + - "ousEventResponse\"\000B\272\002\nCcom.akto.proto.th" + - "reat_protection.service.malicious_alert_" + - "service.v1B\014ServiceProtoP\001\242\002\003TSM\252\0021Threa" + - "tProtection.Service.MaliciousAlertServic" + - "e.V1\312\0021ThreatProtection\\Service\\Maliciou" + - "sAlertService\\V1\342\002=ThreatProtection\\Serv" + - "ice\\MaliciousAlertService\\V1\\GPBMetadata" + - "\352\0024ThreatProtection::Service::MaliciousA" + - "lertService::V1b\006proto3" - }; - descriptor = com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.getDescriptor(), - com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.getDescriptor(), - }); - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventResponse_descriptor, - new java.lang.String[] { }); - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor = - getDescriptor().getMessageTypes().get(1); - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_threat_protection_service_malicious_alert_service_v1_RecordMaliciousEventRequest_descriptor, - new java.lang.String[] { "MaliciousEvent", "SampleRequests", }); - descriptor.resolveAllFeaturesImmutable(); - com.akto.proto.threat_protection.message.malicious_event.v1.MessageProto.getDescriptor(); - com.akto.proto.threat_protection.message.sample_request.v1.MessageProto.getDescriptor(); - } - - // @@protoc_insertion_point(outer_class_scope) -} From a1e8b76ea144133f66149e50c7cf220a2856f3cf Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 11 Dec 2024 12:24:11 +0530 Subject: [PATCH 46/73] moved kafka config from threat detection to utils --- .../src/main/java/com/akto/threat/detection/Main.java | 6 +++--- .../threat/detection/tasks/AbstractKafkaConsumerTask.java | 2 +- .../akto/threat/detection/tasks/FlushSampleDataTask.java | 3 +-- .../detection/tasks/MaliciousTrafficDetectorTask.java | 2 +- .../detection/tasks/SendMaliciousRequestsToBackend.java | 2 +- .../utils/src/main/java/com/akto}/kafka/KafkaConfig.java | 2 +- .../src/main/java/com/akto}/kafka/KafkaConsumerConfig.java | 2 +- .../src/main/java/com/akto}/kafka/KafkaProducerConfig.java | 2 +- 8 files changed, 10 insertions(+), 11 deletions(-) rename {apps/threat-detection/src/main/java/com/akto/threat/detection/config => libs/utils/src/main/java/com/akto}/kafka/KafkaConfig.java (97%) rename {apps/threat-detection/src/main/java/com/akto/threat/detection/config => libs/utils/src/main/java/com/akto}/kafka/KafkaConsumerConfig.java (95%) rename {apps/threat-detection/src/main/java/com/akto/threat/detection/config => libs/utils/src/main/java/com/akto}/kafka/KafkaProducerConfig.java (94%) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index bb676ca9b1..7a294d96a1 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -1,9 +1,9 @@ package com.akto.threat.detection; import com.akto.DaoInit; -import com.akto.threat.detection.config.kafka.KafkaConfig; -import com.akto.threat.detection.config.kafka.KafkaConsumerConfig; -import com.akto.threat.detection.config.kafka.KafkaProducerConfig; +import com.akto.kafka.KafkaConfig; +import com.akto.kafka.KafkaConsumerConfig; +import com.akto.kafka.KafkaProducerConfig; import com.akto.threat.detection.constants.KafkaTopic; import com.akto.threat.detection.session_factory.SessionFactoryUtils; import com.akto.threat.detection.tasks.CleanupTask; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java index 047e93e3ef..9cfa12a1f2 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java @@ -1,7 +1,7 @@ package com.akto.threat.detection.tasks; +import com.akto.kafka.KafkaConfig; import com.akto.runtime.utils.Utils; -import com.akto.threat.detection.config.kafka.KafkaConfig; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index dd5fd28bce..074a15d98e 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -1,9 +1,8 @@ package com.akto.threat.detection.tasks; import com.akto.dto.type.URLMethods; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.kafka.KafkaConfig; import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; -import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 1cb8468c5f..f2f8db9b23 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -1,11 +1,11 @@ package com.akto.threat.detection.tasks; import com.akto.dao.context.Context; +import com.akto.kafka.KafkaConfig; import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.threat.detection.actor.SourceIPActorGenerator; import com.akto.threat.detection.cache.RedisBackedCounterCache; -import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.constants.KafkaTopic; import com.akto.dao.monitoring.FilterYamlTemplateDao; import com.akto.data_actor.DataActor; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java index 72bf1008f2..e3e3465678 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java @@ -1,11 +1,11 @@ package com.akto.threat.detection.tasks; +import com.akto.kafka.KafkaConfig; import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; -import com.akto.threat.detection.config.kafka.KafkaConfig; import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; import com.akto.threat.detection.grpc.AuthToken; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java b/libs/utils/src/main/java/com/akto/kafka/KafkaConfig.java similarity index 97% rename from apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java rename to libs/utils/src/main/java/com/akto/kafka/KafkaConfig.java index c50d78314e..c7209871f3 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConfig.java +++ b/libs/utils/src/main/java/com/akto/kafka/KafkaConfig.java @@ -1,4 +1,4 @@ -package com.akto.threat.detection.config.kafka; +package com.akto.kafka; public class KafkaConfig { private final String bootstrapServers; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConsumerConfig.java b/libs/utils/src/main/java/com/akto/kafka/KafkaConsumerConfig.java similarity index 95% rename from apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConsumerConfig.java rename to libs/utils/src/main/java/com/akto/kafka/KafkaConsumerConfig.java index 969eb96b50..6feddce794 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaConsumerConfig.java +++ b/libs/utils/src/main/java/com/akto/kafka/KafkaConsumerConfig.java @@ -1,4 +1,4 @@ -package com.akto.threat.detection.config.kafka; +package com.akto.kafka; public class KafkaConsumerConfig { private final int maxPollRecords; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaProducerConfig.java b/libs/utils/src/main/java/com/akto/kafka/KafkaProducerConfig.java similarity index 94% rename from apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaProducerConfig.java rename to libs/utils/src/main/java/com/akto/kafka/KafkaProducerConfig.java index a2d8fd6c70..ca999ee0a6 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/config/kafka/KafkaProducerConfig.java +++ b/libs/utils/src/main/java/com/akto/kafka/KafkaProducerConfig.java @@ -1,4 +1,4 @@ -package com.akto.threat.detection.config.kafka; +package com.akto.kafka; public class KafkaProducerConfig { private final int lingerMs; From d1595cac98c91e1880e475ac4d272e9c791d447a Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 11 Dec 2024 13:50:54 +0530 Subject: [PATCH 47/73] moved consumer and producer on same machine for threat protection backend --- .../tasks/AbstractKafkaConsumerTask.java | 1 + .../akto/threat/protection/BackendServer.java | 6 +- .../java/com/akto/threat/protection/Main.java | 30 ++- .../service/MaliciousEventService.java | 16 +- .../protection/tasks/FlushMessagesToDB.java | 123 ++++++++++++ .../threat/protection/utils/KafkaUtils.java | 185 +----------------- .../src/main/java/com/akto/kafka/Kafka.java | 107 +++++----- 7 files changed, 222 insertions(+), 246 deletions(-) create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java index 9cfa12a1f2..f0ea6fa7e1 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java @@ -16,6 +16,7 @@ public abstract class AbstractKafkaConsumerTask implements Task { protected Consumer kafkaConsumer; protected KafkaConfig kafkaConfig; protected String kafkaTopic; + private ExecutorService executorService; public AbstractKafkaConsumerTask(KafkaConfig kafkaConfig, String kafkaTopic) { this.kafkaTopic = kafkaTopic; diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java index ae2138d715..84b397f0f6 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java @@ -1,5 +1,7 @@ package com.akto.threat.protection; +import com.akto.kafka.Kafka; +import com.akto.kafka.KafkaConfig; import com.akto.threat.protection.interceptors.AuthenticationInterceptor; import com.akto.threat.protection.service.DashboardService; import com.akto.threat.protection.service.MaliciousEventService; @@ -16,11 +18,11 @@ public class BackendServer { private final int port; private final Server server; - public BackendServer(int port, MongoClient mongoClient) { + public BackendServer(int port, MongoClient mongoClient, KafkaConfig kafkaConfig) { this.port = port; this.server = ServerBuilder.forPort(port) - .addService(new MaliciousEventService()) + .addService(new MaliciousEventService(kafkaConfig)) .addService(new DashboardService(mongoClient)) .intercept(new AuthenticationInterceptor()) .build(); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index 4265fd918a..901a31bbe7 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -1,6 +1,10 @@ package com.akto.threat.protection; import com.akto.DaoInit; +import com.akto.kafka.KafkaConfig; +import com.akto.kafka.KafkaConsumerConfig; +import com.akto.kafka.KafkaProducerConfig; +import com.akto.threat.protection.tasks.FlushMessagesToDB; import com.akto.threat.protection.utils.KafkaUtils; import com.mongodb.ConnectionString; import com.mongodb.ReadPreference; @@ -18,18 +22,28 @@ public static void main(String[] args) throws Exception { new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")), ReadPreference.secondary(), WriteConcern.ACKNOWLEDGED); - String initProducer = System.getenv().getOrDefault("INIT_KAFKA_PRODUCER", "true"); - if (initProducer != null && initProducer.equalsIgnoreCase("true")) { - KafkaUtils.initKafkaProducer(); - } else { - KafkaUtils.initMongoClient(threatProtectionMongo); - KafkaUtils.initKafkaConsumer(); - } + + KafkaConfig internalKafkaConfig = + KafkaConfig.newBuilder() + .setBootstrapServers(System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL")) + .setGroupId("akto.threat_protection.flush_db") + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(1000).build()) + .build(); + + new FlushMessagesToDB( + internalKafkaConfig, "akto.threat_protection.flush_events_db", threatProtectionMongo) + .run(); int port = Integer.parseInt( System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); - BackendServer server = new BackendServer(port, threatProtectionMongo); + BackendServer server = new BackendServer(port, threatProtectionMongo, internalKafkaConfig); server.start(); server.blockUntilShutdown(); } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java index df0fe9f690..9c9f27a232 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java @@ -4,21 +4,30 @@ import java.util.List; import com.akto.dto.type.URLMethods; +import com.akto.kafka.Kafka; +import com.akto.kafka.KafkaConfig; import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; +import com.akto.threat.protection.BackendServer; import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.interceptors.Constants; import com.akto.threat.protection.utils.KafkaUtils; +import com.mongodb.BasicDBObject; import com.mongodb.client.model.WriteModel; import io.grpc.stub.StreamObserver; public class MaliciousEventService extends MaliciousEventServiceGrpc.MaliciousEventServiceImplBase { - public MaliciousEventService() {} + private final Kafka kafka; + private static final String kafkaTopic = "akto.threat_protection.flush_events_db"; + + public MaliciousEventService(KafkaConfig kafkaConfig) { + this.kafka = new Kafka(kafkaConfig); + } @Override public void recordMaliciousEvent( @@ -66,10 +75,11 @@ public void recordMaliciousEvent( .build()); } - KafkaUtils.insertData(events, "maliciousEvents", accountId); + this.kafka.send(KafkaUtils.generateMsg(events, "maliciousEvents", accountId), kafkaTopic); } - KafkaUtils.insertData(maliciousEventModel, "smartEvent", accountId); + this.kafka.send( + KafkaUtils.generateMsg(maliciousEventModel, "smartEvent", accountId), kafkaTopic); responseObserver.onNext(RecordMaliciousEventResponse.newBuilder().build()); responseObserver.onCompleted(); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java new file mode 100644 index 0000000000..06f3d051f8 --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java @@ -0,0 +1,123 @@ +package com.akto.threat.protection.tasks; + +import com.akto.kafka.KafkaConfig; +import com.akto.runtime.utils.Utils; +import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; +import com.akto.threat.protection.db.MaliciousEventModel; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import com.mongodb.client.MongoClient; +import com.mongodb.client.model.BulkWriteOptions; +import com.mongodb.client.model.InsertOneModel; +import com.mongodb.client.model.WriteModel; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; + +import java.time.Duration; +import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class FlushMessagesToDB { + + private final KafkaConsumer kafkaConsumer; + private final String kafkaTopic; + private final KafkaConfig kafkaConfig; + private final MongoClient mClient; + + private static final ObjectMapper mapper = new ObjectMapper(); + private static final Gson gson = new Gson(); + + public FlushMessagesToDB(KafkaConfig kafkaConfig, String kafkaTopic, MongoClient mongoClient) { + String kafkaBrokerUrl = kafkaConfig.getBootstrapServers(); + String groupId = kafkaConfig.getGroupId(); + + Properties properties = + Utils.configProperties( + kafkaBrokerUrl, groupId, kafkaConfig.getConsumerConfig().getMaxPollRecords()); + this.kafkaConsumer = new KafkaConsumer<>(properties); + this.kafkaConfig = kafkaConfig; + + this.kafkaTopic = kafkaTopic; + + this.mClient = mongoClient; + } + + private static ExecutorService getPollingExecutor() { + return Executors.newSingleThreadExecutor(); + } + + public void run() { + this.kafkaConsumer.subscribe(Collections.singletonList(this.kafkaTopic)); + + getPollingExecutor() + .execute( + () -> { + // Poll data from Kafka topic + while (true) { + ConsumerRecords records = + kafkaConsumer.poll( + Duration.ofMillis( + this.kafkaConfig.getConsumerConfig().getPollDurationMilli())); + if (records.isEmpty()) { + continue; + } + + processRecords(records); + } + }); + } + + private void processRecords(ConsumerRecords records) { + records.forEach( + r -> { + String message = r.value(); + try { + writeMessage(message); + } catch (JsonProcessingException e) { + System.out.println("Error while parsing message" + e); + } catch (Exception ex) { + ex.printStackTrace(); + } + }); + } + + private void writeMessage(String message) throws JsonProcessingException { + Map json = gson.fromJson(message, Map.class); + String eventType = (String) json.get("eventType"); + String payload = (String) json.get("payload"); + Double accIdDouble = (Double) json.get("accountId"); + int accountId = accIdDouble.intValue(); + + switch (eventType) { + case "maliciousEvents": + List> bulkUpdates = new ArrayList<>(); + List events = + mapper.readValue( + payload, new TypeReference>() {}); + events.forEach( + event -> { + bulkUpdates.add(new InsertOneModel<>(event)); + }); + + this.mClient + .getDatabase(accountId + "") + .getCollection("malicious_events", AggregateSampleMaliciousEventModel.class) + .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); + break; + + case "smartEvent": + MaliciousEventModel event = + mapper.readValue(payload, new TypeReference() {}); + this.mClient + .getDatabase(accountId + "") + .getCollection("smart_events", MaliciousEventModel.class) + .insertOne(event); + break; + default: + throw new IllegalArgumentException("Invalid event type"); + } + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java index ac8b1b380d..50e2bb306c 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java @@ -1,200 +1,19 @@ package com.akto.threat.protection.utils; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.akto.dao.context.Context; -import com.akto.kafka.Kafka; -import com.akto.log.LoggerMaker; -import com.akto.log.LoggerMaker.LogDb; -import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; -import com.akto.threat.protection.db.MaliciousEventModel; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import com.mongodb.BasicDBObject; -import com.mongodb.client.MongoClient; -import com.mongodb.client.model.BulkWriteOptions; -import com.mongodb.client.model.InsertOneModel; -import com.mongodb.client.model.WriteModel; public class KafkaUtils { - private static Kafka kafkaProducer; - private static Consumer consumer; - private static final Logger logger = LoggerFactory.getLogger(KafkaUtils.class); - private static final LoggerMaker loggerMaker = new LoggerMaker(KafkaUtils.class); private static final Gson gson = new Gson(); - private static MongoClient mClient; - private static final ObjectMapper mapper = new ObjectMapper(); - private static long lastSyncOffset = 0; - public static void initKafkaProducer() { - String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); - int batchSize = - Integer.parseInt(System.getenv().getOrDefault("AKTO_KAFKA_PRODUCER_BATCH_SIZE", "100")); - int kafkaLingerMS = - Integer.parseInt(System.getenv().getOrDefault("AKTO_KAFKA_PRODUCER_LINGER_MS", "1000")); - kafkaProducer = new Kafka(kafkaBrokerUrl, kafkaLingerMS, batchSize); - logger.info("Kafka Producer Init " + Context.now()); - } - - public static void insertData(Object writes, String eventType, int accountId) { - String topicName = - System.getenv() - .getOrDefault("THREAT_EVENTS_KAFKA_TOPIC", "akto.threat_protection.internal_events"); + public static String generateMsg(Object writes, String eventType, int accountId) { BasicDBObject obj = new BasicDBObject(); obj.put("eventType", eventType); String payloadStr = gson.toJson(writes); obj.put("payload", payloadStr); obj.put("accountId", accountId); - kafkaProducer.send(obj.toString(), topicName); - } - - public static void initMongoClient(MongoClient mongoClient) { - mClient = mongoClient; - } - - public static void initKafkaConsumer() { - System.out.println("Kafka Init consumer called"); - String topicName = - System.getenv() - .getOrDefault("THREAT_EVENTS_KAFKA_TOPIC", "akto.threat_protection.internal_events"); - String kafkaBrokerUrl = System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL"); // kafka1:19092 - String isKubernetes = System.getenv().getOrDefault("IS_KUBERNETES", "false"); - if (isKubernetes != null && isKubernetes.equalsIgnoreCase("true")) { - kafkaBrokerUrl = "127.0.0.1:29092"; - } - String groupIdConfig = System.getenv("THREAT_EVENTS_KAFKA_GROUP_ID_CONFIG"); - int maxPollRecordsConfig = - Integer.parseInt( - System.getenv().getOrDefault("THREAT_EVENTS_KAFKA_MAX_POLL_RECORDS_CONFIG", "100")); - - Properties properties = configProperties(kafkaBrokerUrl, groupIdConfig, maxPollRecordsConfig); - consumer = new KafkaConsumer<>(properties); - final Thread mainThread = Thread.currentThread(); - final AtomicBoolean exceptionOnCommitSync = new AtomicBoolean(false); - - Runtime.getRuntime() - .addShutdownHook( - new Thread() { - public void run() { - consumer.wakeup(); - try { - if (!exceptionOnCommitSync.get()) { - mainThread.join(); - } - } catch (InterruptedException e) { - e.printStackTrace(); - } catch (Error e) { - loggerMaker.errorAndAddToDb( - "Error in add shut down hook: " + e.getMessage(), LogDb.DASHBOARD); - } - } - }); - - try { - consumer.subscribe(Arrays.asList(topicName)); - loggerMaker.infoAndAddToDb("Kafka Consumer subscribed", LogDb.DASHBOARD); - while (true) { - ConsumerRecords records = consumer.poll(Duration.ofMillis(10000)); - try { - consumer.commitSync(); - } catch (Exception e) { - throw e; - } - - for (ConsumerRecord r : records) { - try { - lastSyncOffset++; - if (lastSyncOffset % 100 == 0) { - logger.info("Committing offset at position: " + lastSyncOffset); - } - - parseAndTriggerWrites(r.value()); - } catch (Exception e) { - loggerMaker.errorAndAddToDb(e, "Error in parseAndTriggerWrites " + e, LogDb.DASHBOARD); - continue; - } - } - } - } catch (WakeupException ignored) { - // nothing to catch. This exception is called from the shutdown hook. - } catch (Exception e) { - exceptionOnCommitSync.set(true); - loggerMaker.errorAndAddToDb( - "Exception in init kafka consumer " + e.getMessage(), LogDb.DASHBOARD); - e.printStackTrace(); - System.exit(0); - } finally { - consumer.close(); - } - } - - private static void parseAndTriggerWrites(String message) throws Exception { - Map json = gson.fromJson(message, Map.class); - String eventType = (String) json.get("eventType"); - String payload = (String) json.get("payload"); - Double accIdDouble = (Double) json.get("accountId"); - int accountId = accIdDouble.intValue(); - Context.accountId.set(accountId); - - switch (eventType) { - case "maliciousEvents": - List> bulkUpdates = new ArrayList<>(); - List events = - mapper.readValue( - payload, new TypeReference>() {}); - events.forEach( - event -> { - bulkUpdates.add(new InsertOneModel<>(event)); - }); - - mClient - .getDatabase(accountId + "") - .getCollection("malicious_events", AggregateSampleMaliciousEventModel.class) - .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); - break; - - case "smartEvent": - MaliciousEventModel event = - mapper.readValue(payload, new TypeReference() {}); - mClient - .getDatabase(accountId + "") - .getCollection("smart_events", MaliciousEventModel.class) - .insertOne(event); - break; - default: - break; - } - } - - public static Properties configProperties( - String kafkaBrokerUrl, String groupIdConfig, int maxPollRecordsConfig) { - Properties properties = new Properties(); - properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaBrokerUrl); - properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecordsConfig); - properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupIdConfig); - properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); - return properties; + return obj.toString(); } } diff --git a/libs/utils/src/main/java/com/akto/kafka/Kafka.java b/libs/utils/src/main/java/com/akto/kafka/Kafka.java index 4dcd0b673c..834782c547 100644 --- a/libs/utils/src/main/java/com/akto/kafka/Kafka.java +++ b/libs/utils/src/main/java/com/akto/kafka/Kafka.java @@ -9,65 +9,72 @@ import java.util.Properties; public class Kafka { - private static final Logger logger = LoggerFactory.getLogger(Kafka.class); - private KafkaProducer producer; - public boolean producerReady; + private static final Logger logger = LoggerFactory.getLogger(Kafka.class); + private KafkaProducer producer; + public boolean producerReady; - public Kafka(String brokerIP, int lingerMS, int batchSize) { - producerReady = false; - try { - setProducer(brokerIP, lingerMS, batchSize); - } catch (Exception e) { - e.printStackTrace(); - } + public Kafka(KafkaConfig kafkaConfig) { + this( + kafkaConfig.getBootstrapServers(), + kafkaConfig.getProducerConfig().getLingerMs(), + kafkaConfig.getProducerConfig().getBatchSize()); + } + + public Kafka(String brokerIP, int lingerMS, int batchSize) { + producerReady = false; + try { + setProducer(brokerIP, lingerMS, batchSize); + } catch (Exception e) { + e.printStackTrace(); } + } - public void send(String message, String topic) { - if (!this.producerReady) return; + public void send(String message, String topic) { + if (!this.producerReady) return; - ProducerRecord record = new ProducerRecord<>(topic, message); - producer.send(record, new DemoProducerCallback()); - } + ProducerRecord record = new ProducerRecord<>(topic, message); + producer.send(record, new DemoProducerCallback()); + } - public void close() { - this.producerReady = false; - producer.close(Duration.ofMillis(0)); // close immediately - } + public void close() { + this.producerReady = false; + producer.close(Duration.ofMillis(0)); // close immediately + } - private void setProducer(String brokerIP, int lingerMS, int batchSize) { - if (producer != null) close(); // close existing producer connection + private void setProducer(String brokerIP, int lingerMS, int batchSize) { + if (producer != null) close(); // close existing producer connection - int requestTimeoutMs = 5000; - Properties kafkaProps = new Properties(); - kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerIP); - kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - kafkaProps.put(ProducerConfig.BATCH_SIZE_CONFIG, batchSize); - kafkaProps.put(ProducerConfig.LINGER_MS_CONFIG, lingerMS); - kafkaProps.put(ProducerConfig.RETRIES_CONFIG, 0); - kafkaProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs); - kafkaProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, lingerMS + requestTimeoutMs); - producer = new KafkaProducer(kafkaProps); + int requestTimeoutMs = 5000; + Properties kafkaProps = new Properties(); + kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerIP); + kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + kafkaProps.put(ProducerConfig.BATCH_SIZE_CONFIG, batchSize); + kafkaProps.put(ProducerConfig.LINGER_MS_CONFIG, lingerMS); + kafkaProps.put(ProducerConfig.RETRIES_CONFIG, 0); + kafkaProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs); + kafkaProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, lingerMS + requestTimeoutMs); + producer = new KafkaProducer(kafkaProps); - // test if connection successful by sending a test message in a blocking way - // calling .get() blocks the thread till we receive a message - // if any error then close the connection - ProducerRecord record = new ProducerRecord<>("akto.misc", "ping"); - try { - producer.send(record).get(); - producerReady = true; - } catch (Exception ignored) { - close(); - } + // test if connection successful by sending a test message in a blocking way + // calling .get() blocks the thread till we receive a message + // if any error then close the connection + ProducerRecord record = new ProducerRecord<>("akto.misc", "ping"); + try { + producer.send(record).get(); + producerReady = true; + } catch (Exception ignored) { + close(); } + } - private class DemoProducerCallback implements Callback { - @Override - public void onCompletion(RecordMetadata recordMetadata, Exception e) { - if (e != null) { - Kafka.this.close(); - logger.error("onCompletion error: " + e.getMessage()); - } - } + private class DemoProducerCallback implements Callback { + @Override + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + if (e != null) { + Kafka.this.close(); + logger.error("onCompletion error: " + e.getMessage()); + } } + } } From 7bd52a686dee6a496f6f942b8bb142bf12223c10 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 11 Dec 2024 14:30:03 +0530 Subject: [PATCH 48/73] refactored dashboard proto messages --- ...uest.java => DashboardMaliciousEvent.java} | 17 +++++++-- .../SuspectSampleDataAction.java | 15 ++++---- .../tasks/MaliciousTrafficDetectorTask.java | 17 ++++----- .../tasks/SendMaliciousRequestsToBackend.java | 9 ++--- .../java/com/akto/threat/protection/Main.java | 4 +-- .../protection/constants/KafkaTopic.java | 7 ++++ .../constants/MongoDBCollection.java | 9 +++++ .../protection/db/MaliciousEventModel.java | 8 ++--- .../protection/service/DashboardService.java | 35 +++++++++++-------- .../service/MaliciousEventService.java | 25 ++++++++----- .../protection/tasks/FlushMessagesToDB.java | 18 +++++----- .../dashboard/v1/message.proto | 22 ++++++++++++ .../event_type/v1/event_type.proto | 12 +++++++ .../message/malicious_event/v1/message.proto | 14 +++----- .../message/sample_request/v1/message.proto | 2 +- .../dashboard_service/v1/service.proto | 16 ++------- .../malicious_alert_service/v1/service.proto | 2 +- 17 files changed, 145 insertions(+), 87 deletions(-) rename apps/dashboard/src/main/java/com/akto/action/threat_detection/{DashboardMaliciousRequest.java => DashboardMaliciousEvent.java} (81%) create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/KafkaTopic.java create mode 100644 apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/MongoDBCollection.java create mode 100644 protobuf/threat_protection/message/malicious_event/dashboard/v1/message.proto create mode 100644 protobuf/threat_protection/message/malicious_event/event_type/v1/event_type.proto diff --git a/apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousRequest.java b/apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousEvent.java similarity index 81% rename from apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousRequest.java rename to apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousEvent.java index 6b691e8190..d63268b448 100644 --- a/apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousRequest.java +++ b/apps/dashboard/src/main/java/com/akto/action/threat_detection/DashboardMaliciousEvent.java @@ -3,24 +3,26 @@ import com.akto.dto.type.URLMethods; import com.akto.dto.type.URLMethods.Method; -public class DashboardMaliciousRequest { +public class DashboardMaliciousEvent { private String id; private String actor; private String filter_id; private String url; private URLMethods.Method method; + private int apiCollectionId; private String ip; private String country; private long timestamp; - public DashboardMaliciousRequest() {} + public DashboardMaliciousEvent() {} - public DashboardMaliciousRequest( + public DashboardMaliciousEvent( String id, String actor, String filter, String url, Method method, + int apiCollectionId, String ip, String country, long timestamp) { @@ -29,6 +31,7 @@ public DashboardMaliciousRequest( this.filter_id = filter; this.url = url; this.method = method; + this.apiCollectionId = apiCollectionId; this.ip = ip; this.country = country; this.timestamp = timestamp; @@ -97,4 +100,12 @@ public long getTimestamp() { public void setTimestamp(long timestamp) { this.timestamp = timestamp; } + + public int getApiCollectionId() { + return apiCollectionId; + } + + public void setApiCollectionId(int apiCollectionId) { + this.apiCollectionId = apiCollectionId; + } } diff --git a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java index ab221922b7..d2c0e710e0 100644 --- a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java +++ b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java @@ -4,12 +4,12 @@ import com.akto.dto.traffic.SuspectSampleData; import com.akto.dto.type.URLMethods; import com.akto.grpc.AuthToken; +import com.akto.proto.threat_protection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc; import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceBlockingStub; import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest; import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse; import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; -import com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest; import io.grpc.Grpc; import io.grpc.InsecureChannelCredentials; import io.grpc.ManagedChannel; @@ -20,7 +20,7 @@ public class SuspectSampleDataAction extends UserAction { List sampleData; - List maliciousRequests; + List maliciousRequests; int skip; static final int LIMIT = 50; List ips; @@ -45,25 +45,26 @@ public SuspectSampleDataAction() { } public String fetchSampleDataV2() { - List maliciousRequests = + List maliciousRequests = this.dsServiceStub .listMaliciousRequests( ListMaliciousRequestsRequest.newBuilder().setPage(0).setLimit(500).build()) - .getMaliciousRequestsList(); + .getMaliciousEventsList(); this.maliciousRequests = maliciousRequests.stream() .map( mr -> - new DashboardMaliciousRequest( + new DashboardMaliciousEvent( mr.getId(), mr.getActor(), mr.getFilterId(), - mr.getUrl(), + mr.getEndpoint(), URLMethods.Method.fromString(mr.getMethod()), + mr.getApiCollectionId(), mr.getIp(), mr.getCountry(), - mr.getTimestamp())) + mr.getDetectedAt())) .collect(Collectors.toList()); return SUCCESS.toUpperCase(); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index f2f8db9b23..ac1c7d76e6 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -2,7 +2,8 @@ import com.akto.dao.context.Context; import com.akto.kafka.KafkaConfig; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventMessage; import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.threat.detection.actor.SourceIPActorGenerator; import com.akto.threat.detection.cache.RedisBackedCounterCache; @@ -202,11 +203,7 @@ private void processRecord(ConsumerRecord record) { if (!isAggFilter) { generateAndPushMaliciousEventRequest( - apiFilter, - actor, - responseParam, - maliciousReq, - MaliciousEvent.EventType.EVENT_TYPE_SINGLE); + apiFilter, actor, responseParam, maliciousReq, EventType.EVENT_TYPE_SINGLE); return; } @@ -221,7 +218,7 @@ private void processRecord(ConsumerRecord record) { actor, responseParam, maliciousReq, - MaliciousEvent.EventType.EVENT_TYPE_AGGREGATED); + EventType.EVENT_TYPE_AGGREGATED); } } }); @@ -250,9 +247,9 @@ private void generateAndPushMaliciousEventRequest( String actor, HttpResponseParams responseParam, SampleMaliciousRequest maliciousReq, - MaliciousEvent.EventType eventType) { - MaliciousEvent maliciousEvent = - MaliciousEvent.newBuilder() + EventType eventType) { + MaliciousEventMessage maliciousEvent = + MaliciousEventMessage.newBuilder() .setFilterId(apiFilter.getId()) .setActor(actor) .setDetectedAt(responseParam.getTime()) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java index e3e3465678..08ee736ef1 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java @@ -1,7 +1,8 @@ package com.akto.threat.detection.tasks; import com.akto.kafka.KafkaConfig; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventMessage; import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; @@ -81,7 +82,7 @@ protected void processRecords(ConsumerRecords records) { records.forEach( r -> { String message = r.value(); - MaliciousEvent.Builder builder = MaliciousEvent.newBuilder(); + MaliciousEventMessage.Builder builder = MaliciousEventMessage.newBuilder(); MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); if (m == null) { return; @@ -94,13 +95,13 @@ protected void processRecords(ConsumerRecords records) { return; } - MaliciousEvent evt = builder.build(); + MaliciousEventMessage evt = builder.build(); // Get sample data from postgres for this alert try { RecordMaliciousEventRequest.Builder reqBuilder = RecordMaliciousEventRequest.newBuilder().setMaliciousEvent(evt); - if (MaliciousEvent.EventType.EVENT_TYPE_AGGREGATED.equals(evt.getEventType())) { + if (EventType.EVENT_TYPE_AGGREGATED.equals(evt.getEventType())) { List sampleData = this.getSampleMaliciousRequests(evt.getActor(), evt.getFilterId()); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index 901a31bbe7..410ce31ac9 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -36,9 +36,7 @@ public static void main(String[] args) throws Exception { KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(1000).build()) .build(); - new FlushMessagesToDB( - internalKafkaConfig, "akto.threat_protection.flush_events_db", threatProtectionMongo) - .run(); + new FlushMessagesToDB(internalKafkaConfig, threatProtectionMongo).run(); int port = Integer.parseInt( diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/KafkaTopic.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/KafkaTopic.java new file mode 100644 index 0000000000..6c528ba08b --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/KafkaTopic.java @@ -0,0 +1,7 @@ +package com.akto.threat.protection.constants; + +public class KafkaTopic { + public static class ThreatDetection { + public static final String INTERNAL_DB_MESSAGES = "akto.threat_detection.internal_db_messages"; + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/MongoDBCollection.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/MongoDBCollection.java new file mode 100644 index 0000000000..77bbcb93ee --- /dev/null +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/MongoDBCollection.java @@ -0,0 +1,9 @@ +package com.akto.threat.protection.constants; + +public class MongoDBCollection { + public static class ThreatDetection { + public static final String MALICIOUS_EVENTS = "malicious_events"; + public static final String AGGREGATE_SAMPLE_MALICIOUS_REQUESTS = + "aggregate_sample_malicious_requests"; + } +} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java index 6d71db936d..cff2b17d73 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java @@ -9,7 +9,7 @@ public class MaliciousEventModel { private String id; private String filterId; private String actor; - private String ip; + private String latestIp; private String latestApiEndpoint; private String country; private URLMethods.Method latestApiMethod; @@ -29,7 +29,7 @@ public MaliciousEventModel(Builder builder) { this.id = UUID.randomUUID().toString(); this.filterId = builder.filterId; this.actor = builder.actor; - this.ip = builder.ip; + this.latestIp = builder.ip; this.country = builder.country; this.latestApiEndpoint = builder.latestApiEndpoint; this.latestApiMethod = builder.latestApiMethod; @@ -118,8 +118,8 @@ public String getActor() { return actor; } - public String getIp() { - return ip; + public String getLatestIp() { + return latestIp; } public String getLatestApiEndpoint() { diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java index a236ab3a4e..b95fc8a6be 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java @@ -1,12 +1,14 @@ package com.akto.threat.protection.service; +import com.akto.proto.threat_protection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceImplBase; import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest; import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse; import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse; -import com.akto.proto.threat_protection.service.dashboard_service.v1.MaliciousRequest; +import com.akto.threat.protection.constants.MongoDBCollection; import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; +import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.interceptors.Constants; import com.mongodb.BasicDBObject; import com.mongodb.client.DistinctIterable; @@ -73,33 +75,38 @@ public void listMaliciousRequests( int limit = request.getLimit(); int skip = (page - 1) * limit; - MongoCollection coll = + MongoCollection coll = this.mongoClient .getDatabase(accountId + "") - .getCollection("malicious_events", AggregateSampleMaliciousEventModel.class); + .getCollection( + MongoDBCollection.ThreatDetection.MALICIOUS_EVENTS, MaliciousEventModel.class); BasicDBObject query = new BasicDBObject(); - try (MongoCursor cursor = + try (MongoCursor cursor = coll.find(query).skip(skip).limit(limit).cursor()) { - List alerts = new ArrayList<>(); + List maliciousEvents = new ArrayList<>(); while (cursor.hasNext()) { - AggregateSampleMaliciousEventModel evt = cursor.next(); - alerts.add( - MaliciousRequest.newBuilder() + MaliciousEventModel evt = cursor.next(); + maliciousEvents.add( + DashboardMaliciousEventMessage.newBuilder() .setActor(evt.getActor()) .setFilterId(evt.getFilterId()) .setFilterId(evt.getFilterId()) .setId(evt.getId()) - .setIp(evt.getIp()) + .setIp(evt.getLatestIp()) .setCountry(evt.getCountry()) - .setOrig(evt.getOrig()) - .setUrl(evt.getUrl()) - .setMethod(evt.getMethod().name()) - .setTimestamp(evt.getRequestTime()) + .setPayload(evt.getLatestApiOrig()) + .setEndpoint(evt.getLatestApiEndpoint()) + .setMethod(evt.getLatestApiMethod().name()) + .setDetectedAt(evt.getDetectedAt()) .build()); } ListMaliciousRequestsResponse response = - ListMaliciousRequestsResponse.newBuilder().setPage(page).setTotal(alerts.size()).build(); + ListMaliciousRequestsResponse.newBuilder() + .setPage(page) + .setTotal(maliciousEvents.size()) + .addAllMaliciousEvents(maliciousEvents) + .build(); responseObserver.onNext(response); responseObserver.onCompleted(); } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java index 9c9f27a232..bae309e6ce 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java @@ -6,24 +6,24 @@ import com.akto.dto.type.URLMethods; import com.akto.kafka.Kafka; import com.akto.kafka.KafkaConfig; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEvent; +import com.akto.proto.threat_protection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventMessage; import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; -import com.akto.threat.protection.BackendServer; +import com.akto.threat.protection.constants.KafkaTopic; +import com.akto.threat.protection.constants.MongoDBCollection; import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.interceptors.Constants; import com.akto.threat.protection.utils.KafkaUtils; -import com.mongodb.BasicDBObject; import com.mongodb.client.model.WriteModel; import io.grpc.stub.StreamObserver; public class MaliciousEventService extends MaliciousEventServiceGrpc.MaliciousEventServiceImplBase { private final Kafka kafka; - private static final String kafkaTopic = "akto.threat_protection.flush_events_db"; public MaliciousEventService(KafkaConfig kafkaConfig) { this.kafka = new Kafka(kafkaConfig); @@ -34,16 +34,16 @@ public void recordMaliciousEvent( RecordMaliciousEventRequest request, StreamObserver responseObserver) { - MaliciousEvent evt = request.getMaliciousEvent(); + MaliciousEventMessage evt = request.getMaliciousEvent(); String actor = evt.getActor(); String filterId = evt.getFilterId(); List> bulkUpdates = new ArrayList<>(); int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); - MaliciousEvent.EventType eventType = evt.getEventType(); + EventType eventType = evt.getEventType(); MaliciousEventModel.EventType maliciousEventType = - MaliciousEvent.EventType.EVENT_TYPE_AGGREGATED.equals(eventType) + EventType.EVENT_TYPE_AGGREGATED.equals(eventType) ? MaliciousEventModel.EventType.AGGREGATED : MaliciousEventModel.EventType.SINGLE; @@ -75,11 +75,18 @@ public void recordMaliciousEvent( .build()); } - this.kafka.send(KafkaUtils.generateMsg(events, "maliciousEvents", accountId), kafkaTopic); + this.kafka.send( + KafkaUtils.generateMsg( + events, + MongoDBCollection.ThreatDetection.AGGREGATE_SAMPLE_MALICIOUS_REQUESTS, + accountId), + KafkaTopic.ThreatDetection.INTERNAL_DB_MESSAGES); } this.kafka.send( - KafkaUtils.generateMsg(maliciousEventModel, "smartEvent", accountId), kafkaTopic); + KafkaUtils.generateMsg( + maliciousEventModel, MongoDBCollection.ThreatDetection.MALICIOUS_EVENTS, accountId), + KafkaTopic.ThreatDetection.INTERNAL_DB_MESSAGES); responseObserver.onNext(RecordMaliciousEventResponse.newBuilder().build()); responseObserver.onCompleted(); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java index 06f3d051f8..438c2893b0 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java @@ -2,6 +2,8 @@ import com.akto.kafka.KafkaConfig; import com.akto.runtime.utils.Utils; +import com.akto.threat.protection.constants.KafkaTopic; +import com.akto.threat.protection.constants.MongoDBCollection; import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; import com.akto.threat.protection.db.MaliciousEventModel; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,14 +25,13 @@ public class FlushMessagesToDB { private final KafkaConsumer kafkaConsumer; - private final String kafkaTopic; private final KafkaConfig kafkaConfig; private final MongoClient mClient; private static final ObjectMapper mapper = new ObjectMapper(); private static final Gson gson = new Gson(); - public FlushMessagesToDB(KafkaConfig kafkaConfig, String kafkaTopic, MongoClient mongoClient) { + public FlushMessagesToDB(KafkaConfig kafkaConfig, MongoClient mongoClient) { String kafkaBrokerUrl = kafkaConfig.getBootstrapServers(); String groupId = kafkaConfig.getGroupId(); @@ -40,8 +41,6 @@ public FlushMessagesToDB(KafkaConfig kafkaConfig, String kafkaTopic, MongoClient this.kafkaConsumer = new KafkaConsumer<>(properties); this.kafkaConfig = kafkaConfig; - this.kafkaTopic = kafkaTopic; - this.mClient = mongoClient; } @@ -50,7 +49,8 @@ private static ExecutorService getPollingExecutor() { } public void run() { - this.kafkaConsumer.subscribe(Collections.singletonList(this.kafkaTopic)); + this.kafkaConsumer.subscribe( + Collections.singletonList(KafkaTopic.ThreatDetection.INTERNAL_DB_MESSAGES)); getPollingExecutor() .execute( @@ -92,7 +92,7 @@ private void writeMessage(String message) throws JsonProcessingException { int accountId = accIdDouble.intValue(); switch (eventType) { - case "maliciousEvents": + case MongoDBCollection.ThreatDetection.AGGREGATE_SAMPLE_MALICIOUS_REQUESTS: List> bulkUpdates = new ArrayList<>(); List events = mapper.readValue( @@ -104,16 +104,16 @@ private void writeMessage(String message) throws JsonProcessingException { this.mClient .getDatabase(accountId + "") - .getCollection("malicious_events", AggregateSampleMaliciousEventModel.class) + .getCollection(eventType, AggregateSampleMaliciousEventModel.class) .bulkWrite(bulkUpdates, new BulkWriteOptions().ordered(false)); break; - case "smartEvent": + case MongoDBCollection.ThreatDetection.MALICIOUS_EVENTS: MaliciousEventModel event = mapper.readValue(payload, new TypeReference() {}); this.mClient .getDatabase(accountId + "") - .getCollection("smart_events", MaliciousEventModel.class) + .getCollection(eventType, MaliciousEventModel.class) .insertOne(event); break; default: diff --git a/protobuf/threat_protection/message/malicious_event/dashboard/v1/message.proto b/protobuf/threat_protection/message/malicious_event/dashboard/v1/message.proto new file mode 100644 index 0000000000..7864d62a1c --- /dev/null +++ b/protobuf/threat_protection/message/malicious_event/dashboard/v1/message.proto @@ -0,0 +1,22 @@ +syntax = "proto3"; + +package threat_protection.message.malicious_event.dashboard.v1; + +import "threat_protection/message/malicious_event/event_type/v1/event_type.proto"; + +option java_outer_classname = "MaliciousEventDashboardProto"; +option java_package = "threat_protection.message.malicious_event.dashboard.v1"; + +message DashboardMaliciousEventMessage { + string id = 1; + string actor = 2; + string filter_id = 3; + int64 detected_at = 4; + string ip = 5; + string endpoint = 6; + string method = 7; + int32 api_collection_id = 8; + string payload = 9; + string country = 10; + threat_protection.message.malicious_event.event_type.v1.EventType event_type = 11; +} diff --git a/protobuf/threat_protection/message/malicious_event/event_type/v1/event_type.proto b/protobuf/threat_protection/message/malicious_event/event_type/v1/event_type.proto new file mode 100644 index 0000000000..440c7a8355 --- /dev/null +++ b/protobuf/threat_protection/message/malicious_event/event_type/v1/event_type.proto @@ -0,0 +1,12 @@ +syntax = "proto3"; + +package threat_protection.message.malicious_event.event_type.v1; + +option java_outer_classname = "MaliciousEventProto"; +option java_package = "threat_protection.message.malicious_event.event_type.v1"; + +enum EventType { + EVENT_TYPE_UNSPECIFIED = 0; + EVENT_TYPE_SINGLE = 1; + EVENT_TYPE_AGGREGATED = 2; +} diff --git a/protobuf/threat_protection/message/malicious_event/v1/message.proto b/protobuf/threat_protection/message/malicious_event/v1/message.proto index 4b2ed7b0b8..66175ad8ff 100644 --- a/protobuf/threat_protection/message/malicious_event/v1/message.proto +++ b/protobuf/threat_protection/message/malicious_event/v1/message.proto @@ -2,10 +2,12 @@ syntax = "proto3"; package threat_protection.message.malicious_event.v1; +import "threat_protection/message/malicious_event/event_type/v1/event_type.proto"; + option java_outer_classname = "MaliciousEventProto"; -option java_package = "threat_protection.v1.message.malicious_event"; +option java_package = "threat_protection.message.malicious_event.v1"; -message MaliciousEvent { +message MaliciousEventMessage { string actor = 1; string filter_id = 2; int64 detected_at = 3; @@ -14,11 +16,5 @@ message MaliciousEvent { string latest_api_method = 6; int32 latest_api_collection_id = 7; string latest_api_payload = 8; - EventType event_type = 9; - - enum EventType { - EVENT_TYPE_UNSPECIFIED = 0; - EVENT_TYPE_SINGLE = 1; - EVENT_TYPE_AGGREGATED = 2; - } + threat_protection.message.malicious_event.event_type.v1.EventType event_type = 9; } diff --git a/protobuf/threat_protection/message/sample_request/v1/message.proto b/protobuf/threat_protection/message/sample_request/v1/message.proto index 37a19b1411..a8263d76c5 100644 --- a/protobuf/threat_protection/message/sample_request/v1/message.proto +++ b/protobuf/threat_protection/message/sample_request/v1/message.proto @@ -3,7 +3,7 @@ syntax = "proto3"; package threat_protection.message.sample_request.v1; option java_outer_classname = "SampleRequestProto"; -option java_package = "threat_protection.v1.message.sample_request"; +option java_package = "threat_protection.message.sample_request.v1"; message SampleMaliciousRequest { string ip = 1; diff --git a/protobuf/threat_protection/service/dashboard_service/v1/service.proto b/protobuf/threat_protection/service/dashboard_service/v1/service.proto index ecb286836b..0a44c43113 100644 --- a/protobuf/threat_protection/service/dashboard_service/v1/service.proto +++ b/protobuf/threat_protection/service/dashboard_service/v1/service.proto @@ -2,24 +2,14 @@ syntax = "proto3"; package threat_protection.service.dashboard_service.v1; +import "threat_protection/message/malicious_event/dashboard/v1/message.proto"; + // Dashboard service which the dashboard actions will call instead of directly calling DB option java_outer_classname = "DashboardServiceProto"; option java_package = "threat_protection.service.dashboard_service.v1"; -message MaliciousRequest { - string id = 1; - string actor = 2; - string filter_id = 3; - string url = 4; - string method = 5; - string orig = 6; - string ip = 7; - string country = 8; - int64 timestamp = 9; -} - message ListMaliciousRequestsResponse { - repeated MaliciousRequest malicious_requests = 1; + repeated threat_protection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage malicious_events = 1; int32 total = 2; int32 page = 3; } diff --git a/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto b/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto index 55b3322149..c344ac3e01 100644 --- a/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto +++ b/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto @@ -14,7 +14,7 @@ message RecordMaliciousEventResponse { } message RecordMaliciousEventRequest { - threat_protection.message.malicious_event.v1.MaliciousEvent malicious_event = 1; + threat_protection.message.malicious_event.v1.MaliciousEventMessage malicious_event = 1; repeated threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2; } From 9f3cb08ff37cd959a08e7eb05278720b43e26242 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Thu, 12 Dec 2024 15:00:46 +0530 Subject: [PATCH 49/73] added check if buf is installed in proto-gen script --- scripts/proto-gen.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/scripts/proto-gen.sh b/scripts/proto-gen.sh index f05dbd0230..d503d4da52 100644 --- a/scripts/proto-gen.sh +++ b/scripts/proto-gen.sh @@ -2,6 +2,11 @@ # Check if buf is installed or not # Please install buf if not already installed by following the instructions at https://docs.buf.build/installation +if ! command -v buf >/dev/null 2>&1; then + echo "Please install buf if not already installed by following the instructions at https://docs.buf.build/installation" + exit +fi + buf lint protobuf rm -rf ./libs/protobuf/src buf generate protobuf \ No newline at end of file From c5f627bd1d4963c974105667bc63a5b2d42ad1ff Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Mon, 16 Dec 2024 15:03:40 +0530 Subject: [PATCH 50/73] fixed MaliciousEventModel not populating data from mongo --- .../SuspectSampleDataAction.java | 16 +++-- .../components/layouts/leftnav/LeftNav.js | 23 ++++--- .../components/SusDataTable.jsx | 22 +++---- .../java/com/akto/threat/protection/Main.java | 66 ++++++++++++------- .../protection/db/MaliciousEventModel.java | 61 ++++++++++++++--- .../protection/service/DashboardService.java | 23 +++---- .../service/MaliciousEventService.java | 7 +- 7 files changed, 143 insertions(+), 75 deletions(-) diff --git a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java index d2c0e710e0..397612583f 100644 --- a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java +++ b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java @@ -20,7 +20,7 @@ public class SuspectSampleDataAction extends UserAction { List sampleData; - List maliciousRequests; + List maliciousEvents; int skip; static final int LIMIT = 50; List ips; @@ -45,14 +45,14 @@ public SuspectSampleDataAction() { } public String fetchSampleDataV2() { - List maliciousRequests = + List maliciousEvts = this.dsServiceStub .listMaliciousRequests( ListMaliciousRequestsRequest.newBuilder().setPage(0).setLimit(500).build()) .getMaliciousEventsList(); - this.maliciousRequests = - maliciousRequests.stream() + this.maliciousEvents = + maliciousEvts.stream() .map( mr -> new DashboardMaliciousEvent( @@ -154,4 +154,12 @@ public int getEndTimestamp() { public void setEndTimestamp(int endTimestamp) { this.endTimestamp = endTimestamp; } + + public List getMaliciousEvents() { + return maliciousEvents; + } + + public void setMaliciousEvents(List maliciousRequests) { + this.maliciousEvents = maliciousRequests; + } } diff --git a/apps/dashboard/web/polaris_web/web/src/apps/dashboard/components/layouts/leftnav/LeftNav.js b/apps/dashboard/web/polaris_web/web/src/apps/dashboard/components/layouts/leftnav/LeftNav.js index 1e17e6f3ec..22dafb2083 100644 --- a/apps/dashboard/web/polaris_web/web/src/apps/dashboard/components/layouts/leftnav/LeftNav.js +++ b/apps/dashboard/web/polaris_web/web/src/apps/dashboard/components/layouts/leftnav/LeftNav.js @@ -154,18 +154,17 @@ export default function LeftNav(){ selected: leftNavSelected === 'dashboard_issues', key: '6', }, - window?.STIGG_FEATURE_WISE_ALLOWED?.THREAT_DETECTION?.isGranted ? - { - label: API Runtime Threats, - icon: DiamondAlertMinor, - onClick: () => { - handleSelect("dashboard_threat_detection") - navigate("/dashboard/threat-detection") - setActive("normal") - }, - selected: leftNavSelected === 'dashboard_threat_detection', - key: '7', - } : {} + { + label: API Runtime Threats, + icon: DiamondAlertMinor, + onClick: () => { + handleSelect("dashboard_threat_detection") + navigate("/dashboard/threat-detection") + setActive("normal") + }, + selected: leftNavSelected === 'dashboard_threat_detection', + key: '7', + } ]} /> { + let ret = res?.maliciousEvents.map(x => { return { ...x, + actorComp: x?.actor, endpointComp: , apiCollectionName: collectionsMap[x.apiCollectionId] || '-', - discoveredTs: func.prettifyEpoch(x.discovered), - sourceIPComponent: {x?.sourceIPs ? x.sourceIPs.reduce((a, b) => a += ((a.length > 0 ? ", " : "") + b), "") : "-"} + discoveredTs: func.prettifyEpoch(x.timestamp), + sourceIPComponent: x?.ip || '-', } }) setLoading(false); @@ -168,7 +168,7 @@ function SusDataTable({ currDateRange, rowClicked }) { sortOptions={sortOptions} disambiguateLabel={disambiguateLabel} loading={loading} - onRowClick={(data) => rowClicked(data)} + // onRowClick={(data) => rowClicked(data)} [For now removing on row click functionality] fetchData={fetchData} filters={filters} selectable={false} diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java index 410ce31ac9..b147ba51df 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java @@ -1,15 +1,21 @@ package com.akto.threat.protection; +import static org.bson.codecs.configuration.CodecRegistries.fromProviders; +import static org.bson.codecs.configuration.CodecRegistries.fromRegistries; + import com.akto.DaoInit; import com.akto.kafka.KafkaConfig; import com.akto.kafka.KafkaConsumerConfig; import com.akto.kafka.KafkaProducerConfig; import com.akto.threat.protection.tasks.FlushMessagesToDB; -import com.akto.threat.protection.utils.KafkaUtils; import com.mongodb.ConnectionString; +import com.mongodb.MongoClientSettings; import com.mongodb.ReadPreference; import com.mongodb.WriteConcern; import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import org.bson.codecs.configuration.CodecRegistry; +import org.bson.codecs.pojo.PojoCodecProvider; public class Main { public static void main(String[] args) throws Exception { @@ -17,32 +23,42 @@ public static void main(String[] args) throws Exception { DaoInit.init(new ConnectionString(mongoURI)); - MongoClient threatProtectionMongo = - DaoInit.createMongoClient( - new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")), - ReadPreference.secondary(), - WriteConcern.ACKNOWLEDGED); - - KafkaConfig internalKafkaConfig = - KafkaConfig.newBuilder() - .setBootstrapServers(System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL")) - .setGroupId("akto.threat_protection.flush_db") - .setConsumerConfig( - KafkaConsumerConfig.newBuilder() - .setMaxPollRecords(100) - .setPollDurationMilli(100) - .build()) - .setProducerConfig( - KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(1000).build()) + ConnectionString connectionString = + new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")); + CodecRegistry pojoCodecRegistry = + fromProviders(PojoCodecProvider.builder().automatic(true).build()); + CodecRegistry codecRegistry = + fromRegistries(MongoClientSettings.getDefaultCodecRegistry(), pojoCodecRegistry); + MongoClientSettings clientSettings = + MongoClientSettings.builder() + .readPreference(ReadPreference.secondary()) + .writeConcern(WriteConcern.ACKNOWLEDGED) + .applyConnectionString(connectionString) + .codecRegistry(codecRegistry) .build(); - new FlushMessagesToDB(internalKafkaConfig, threatProtectionMongo).run(); + try (MongoClient threatProtectionMongo = MongoClients.create(clientSettings)) { + KafkaConfig internalKafkaConfig = + KafkaConfig.newBuilder() + .setBootstrapServers(System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL")) + .setGroupId("akto.threat_protection.flush_db") + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(1000).build()) + .build(); + + new FlushMessagesToDB(internalKafkaConfig, threatProtectionMongo).run(); - int port = - Integer.parseInt( - System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); - BackendServer server = new BackendServer(port, threatProtectionMongo, internalKafkaConfig); - server.start(); - server.blockUntilShutdown(); + int port = + Integer.parseInt( + System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); + BackendServer server = new BackendServer(port, threatProtectionMongo, internalKafkaConfig); + server.start(); + server.blockUntilShutdown(); + } } } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java index cff2b17d73..5735454442 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java @@ -1,7 +1,6 @@ package com.akto.threat.protection.db; import com.akto.dto.type.URLMethods; - import java.util.UUID; public class MaliciousEventModel { @@ -9,7 +8,7 @@ public class MaliciousEventModel { private String id; private String filterId; private String actor; - private String latestIp; + private String latestApiIp; private String latestApiEndpoint; private String country; private URLMethods.Method latestApiMethod; @@ -25,11 +24,11 @@ public enum EventType { public MaliciousEventModel() {} - public MaliciousEventModel(Builder builder) { + private MaliciousEventModel(Builder builder) { this.id = UUID.randomUUID().toString(); this.filterId = builder.filterId; this.actor = builder.actor; - this.latestIp = builder.ip; + this.latestApiIp = builder.latestApiIp; this.country = builder.country; this.latestApiEndpoint = builder.latestApiEndpoint; this.latestApiMethod = builder.latestApiMethod; @@ -43,7 +42,7 @@ public static class Builder { public EventType eventType; private String filterId; private String actor; - private String ip; + private String latestApiIp; private String country; private String latestApiEndpoint; private URLMethods.Method latestApiMethod; @@ -61,8 +60,8 @@ public Builder setActor(String actor) { return this; } - public Builder setIp(String ip) { - this.ip = ip; + public Builder setLatestApiIp(String ip) { + this.latestApiIp = ip; return this; } @@ -118,8 +117,8 @@ public String getActor() { return actor; } - public String getLatestIp() { - return latestIp; + public String getLatestApiIp() { + return latestApiIp; } public String getLatestApiEndpoint() { @@ -153,4 +152,48 @@ public EventType getEventType() { public static Builder newBuilder() { return new Builder(); } + + public void setId(String id) { + this.id = id; + } + + public void setFilterId(String filterId) { + this.filterId = filterId; + } + + public void setActor(String actor) { + this.actor = actor; + } + + public void setLatestApiIp(String ip) { + this.latestApiIp = ip; + } + + public void setLatestApiEndpoint(String latestApiEndpoint) { + this.latestApiEndpoint = latestApiEndpoint; + } + + public void setCountry(String country) { + this.country = country; + } + + public void setLatestApiMethod(URLMethods.Method latestApiMethod) { + this.latestApiMethod = latestApiMethod; + } + + public void setLatestApiOrig(String latestApiOrig) { + this.latestApiOrig = latestApiOrig; + } + + public void setDetectedAt(long detectedAt) { + this.detectedAt = detectedAt; + } + + public void setLatestApiCollectionId(int latestApiCollectionId) { + this.latestApiCollectionId = latestApiCollectionId; + } + + public void setEventType(EventType eventType) { + this.eventType = eventType; + } } diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java index b95fc8a6be..d0009ee9c8 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java @@ -7,7 +7,6 @@ import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse; import com.akto.threat.protection.constants.MongoDBCollection; -import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; import com.akto.threat.protection.db.MaliciousEventModel; import com.akto.threat.protection.interceptors.Constants; import com.mongodb.BasicDBObject; @@ -31,10 +30,7 @@ public DashboardService(MongoClient mongoClient) { } private static Set findDistinctFields( - MongoCollection coll, - String fieldName, - Class tClass, - Bson filters) { + MongoCollection coll, String fieldName, Class tClass, Bson filters) { DistinctIterable r = coll.distinct(fieldName, filters, tClass); Set result = new HashSet<>(); MongoCursor cursor = r.cursor(); @@ -49,15 +45,16 @@ public void fetchAlertFilters( FetchAlertFiltersRequest request, StreamObserver responseObserver) { int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); - MongoCollection coll = + MongoCollection coll = this.mongoClient .getDatabase(accountId + "") - .getCollection("malicious_events", AggregateSampleMaliciousEventModel.class); + .getCollection("malicious_events", MaliciousEventModel.class); Set actors = DashboardService.findDistinctFields(coll, "actor", String.class, Filters.empty()); Set urls = - DashboardService.findDistinctFields(coll, "url", String.class, Filters.empty()); + DashboardService.findDistinctFields( + coll, "latestApiEndpoint", String.class, Filters.empty()); FetchAlertFiltersResponse response = FetchAlertFiltersResponse.newBuilder().addAllActors(actors).addAllUrls(urls).build(); @@ -71,7 +68,7 @@ public void listMaliciousRequests( StreamObserver responseObserver) { int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); - int page = request.hasPage() ? request.getPage() : 1; + int page = request.hasPage() && request.getPage() > 0 ? request.getPage() : 1; int limit = request.getLimit(); int skip = (page - 1) * limit; @@ -83,7 +80,11 @@ public void listMaliciousRequests( BasicDBObject query = new BasicDBObject(); try (MongoCursor cursor = - coll.find(query).skip(skip).limit(limit).cursor()) { + coll.find(query) + .sort(new BasicDBObject("detectedAt", -1)) + .skip(skip) + .limit(limit) + .cursor()) { List maliciousEvents = new ArrayList<>(); while (cursor.hasNext()) { MaliciousEventModel evt = cursor.next(); @@ -93,7 +94,7 @@ public void listMaliciousRequests( .setFilterId(evt.getFilterId()) .setFilterId(evt.getFilterId()) .setId(evt.getId()) - .setIp(evt.getLatestIp()) + .setIp(evt.getLatestApiIp()) .setCountry(evt.getCountry()) .setPayload(evt.getLatestApiOrig()) .setEndpoint(evt.getLatestApiEndpoint()) diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java index bae309e6ce..d50b8e845b 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java +++ b/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java @@ -1,8 +1,5 @@ package com.akto.threat.protection.service; -import java.util.ArrayList; -import java.util.List; - import com.akto.dto.type.URLMethods; import com.akto.kafka.Kafka; import com.akto.kafka.KafkaConfig; @@ -20,6 +17,8 @@ import com.akto.threat.protection.utils.KafkaUtils; import com.mongodb.client.model.WriteModel; import io.grpc.stub.StreamObserver; +import java.util.ArrayList; +import java.util.List; public class MaliciousEventService extends MaliciousEventServiceGrpc.MaliciousEventServiceImplBase { @@ -57,6 +56,8 @@ public void recordMaliciousEvent( .setLatestApiOrig(evt.getLatestApiPayload()) .setLatestApiCollectionId(evt.getLatestApiCollectionId()) .setEventType(maliciousEventType) + .setLatestApiIp(evt.getLatestApiIp()) + .setCountry("US") .build(); if (MaliciousEventModel.EventType.AGGREGATED.equals(maliciousEventType)) { From 5daab8d4d7a0fbb8456da6b0baa71f360b479998 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 17 Dec 2024 12:40:58 +0530 Subject: [PATCH 51/73] refactored code --- .../java/com/akto/threat/detection/Main.java | 2 +- .../session_factory/SessionFactoryUtils.java | 3 +- .../tasks/AbstractKafkaConsumerTask.java | 42 +++++++++---------- .../threat/detection/tasks/CleanupTask.java | 6 +-- .../detection/tasks/FlushSampleDataTask.java | 12 +----- .../tasks/MaliciousTrafficDetectorTask.java | 31 +++++++------- .../tasks/SendMaliciousRequestsToBackend.java | 4 -- 7 files changed, 41 insertions(+), 59 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index 7a294d96a1..65a5cdb4b5 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -12,7 +12,6 @@ import com.akto.threat.detection.tasks.SendMaliciousRequestsToBackend; import com.mongodb.ConnectionString; import io.lettuce.core.RedisClient; - import org.flywaydb.core.Flyway; import org.hibernate.SessionFactory; @@ -25,6 +24,7 @@ public static void main(String[] args) throws Exception { SessionFactory sessionFactory = SessionFactoryUtils.createFactory(); + // TODO: Remove this before merging. Will be using cyborg for fetching templates DaoInit.init(new ConnectionString(System.getenv("AKTO_MONGO_CONN"))); KafkaConfig trafficKafka = KafkaConfig.newBuilder() diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java index 556486182f..7bf80f0f7b 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/session_factory/SessionFactoryUtils.java @@ -1,11 +1,10 @@ package com.akto.threat.detection.session_factory; +import com.akto.threat.detection.db.entity.MaliciousEventEntity; import org.hibernate.SessionFactory; import org.hibernate.boot.registry.StandardServiceRegistryBuilder; import org.hibernate.cfg.Configuration; -import com.akto.threat.detection.db.entity.MaliciousEventEntity; - public class SessionFactoryUtils { public static SessionFactory createFactory() { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java index f0ea6fa7e1..e918efff11 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java @@ -2,21 +2,20 @@ import com.akto.kafka.KafkaConfig; import com.akto.runtime.utils.Utils; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; - import java.time.Duration; import java.util.Collections; import java.util.Properties; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; public abstract class AbstractKafkaConsumerTask implements Task { protected Consumer kafkaConsumer; protected KafkaConfig kafkaConfig; protected String kafkaTopic; - private ExecutorService executorService; public AbstractKafkaConsumerTask(KafkaConfig kafkaConfig, String kafkaTopic) { this.kafkaTopic = kafkaTopic; @@ -31,27 +30,26 @@ public AbstractKafkaConsumerTask(KafkaConfig kafkaConfig, String kafkaTopic) { this.kafkaConsumer = new KafkaConsumer<>(properties); } - abstract ExecutorService getPollingExecutor(); - @Override public void run() { this.kafkaConsumer.subscribe(Collections.singletonList(this.kafkaTopic)); - this.getPollingExecutor() - .execute( - () -> { - // Poll data from Kafka topic - while (true) { - ConsumerRecords records = - kafkaConsumer.poll( - Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); - if (records.isEmpty()) { - continue; - } - - processRecords(records); - } - }); + ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); + + pollingExecutor.execute( + () -> { + // Poll data from Kafka topic + while (true) { + ConsumerRecords records = + kafkaConsumer.poll( + Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); + if (records.isEmpty()) { + continue; + } + + processRecords(records); + } + }); } abstract void processRecords(ConsumerRecords records); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java index acfc6abbc8..93a6d7c906 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/CleanupTask.java @@ -5,7 +5,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - import org.hibernate.Session; import org.hibernate.SessionFactory; import org.hibernate.Transaction; @@ -14,15 +13,14 @@ public class CleanupTask implements Task { private final SessionFactory sessionFactory; - private final ScheduledExecutorService cronExecutorService = Executors.newScheduledThreadPool(1); - public CleanupTask(SessionFactory sessionFactory) { this.sessionFactory = sessionFactory; } @Override public void run() { - this.cronExecutorService.scheduleAtFixedRate(this::cleanup, 5, 10 * 60, TimeUnit.SECONDS); + ScheduledExecutorService cronExecutorService = Executors.newScheduledThreadPool(1); + cronExecutorService.scheduleAtFixedRate(this::cleanup, 5, 30, TimeUnit.MINUTES); } private void cleanup() { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 074a15d98e..ab3dcf3765 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -7,16 +7,13 @@ import com.akto.threat.detection.dto.MessageEnvelope; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; +import java.util.ArrayList; +import java.util.List; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.hibernate.Session; import org.hibernate.SessionFactory; import org.hibernate.Transaction; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - /* This will read sample malicious data from kafka topic and save it to DB. */ @@ -30,11 +27,6 @@ public FlushSampleDataTask( this.sessionFactory = sessionFactory; } - @Override - ExecutorService getPollingExecutor() { - return Executors.newSingleThreadExecutor(); - } - protected void processRecords(ConsumerRecords records) { List events = new ArrayList<>(); records.forEach( diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index ac1c7d76e6..34031f2d57 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -1,19 +1,11 @@ package com.akto.threat.detection.tasks; import com.akto.dao.context.Context; -import com.akto.kafka.KafkaConfig; -import com.akto.proto.threat_protection.message.malicious_event.event_type.v1.EventType; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventMessage; -import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; -import com.akto.threat.detection.actor.SourceIPActorGenerator; -import com.akto.threat.detection.cache.RedisBackedCounterCache; -import com.akto.threat.detection.constants.KafkaTopic; import com.akto.dao.monitoring.FilterYamlTemplateDao; import com.akto.data_actor.DataActor; import com.akto.data_actor.DataActorFactory; import com.akto.dto.ApiInfo; import com.akto.dto.HttpResponseParams; -import com.akto.threat.detection.dto.MessageEnvelope; import com.akto.dto.RawApi; import com.akto.dto.api_protection_parse_layer.AggregationRules; import com.akto.dto.api_protection_parse_layer.Condition; @@ -23,30 +15,36 @@ import com.akto.dto.type.URLMethods; import com.akto.hybrid_parsers.HttpCallParser; import com.akto.kafka.Kafka; +import com.akto.kafka.KafkaConfig; +import com.akto.proto.threat_protection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventMessage; +import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.rules.TestPlugin; import com.akto.runtime.utils.Utils; -import com.akto.threat.detection.smart_event_detector.window_based.WindowBasedThresholdNotifier; import com.akto.test_editor.execution.VariableResolver; import com.akto.test_editor.filter.data_operands_impl.ValidationResult; +import com.akto.threat.detection.actor.SourceIPActorGenerator; +import com.akto.threat.detection.cache.RedisBackedCounterCache; +import com.akto.threat.detection.constants.KafkaTopic; +import com.akto.threat.detection.dto.MessageEnvelope; +import com.akto.threat.detection.smart_event_detector.window_based.WindowBasedThresholdNotifier; import com.google.protobuf.InvalidProtocolBufferException; import io.lettuce.core.RedisClient; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; - import java.time.Duration; import java.util.*; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; /* Class is responsible for consuming traffic data from the Kafka topic. Pass data through filters and identify malicious traffic. */ -public class MaliciousTrafficDetectorTask { +public class MaliciousTrafficDetectorTask implements Task { - private static final ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); private final Consumer kafkaConsumer; private final KafkaConfig kafkaConfig; private final HttpCallParser httpCallParser; @@ -87,6 +85,7 @@ public MaliciousTrafficDetectorTask( public void run() { this.kafkaConsumer.subscribe(Collections.singletonList("akto.api.logs")); + ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); pollingExecutor.execute( () -> { // Poll data from Kafka topic diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java index 08ee736ef1..b774b1f0ef 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java @@ -51,10 +51,6 @@ public SendMaliciousRequestsToBackend( new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); } - ExecutorService getPollingExecutor() { - return Executors.newSingleThreadExecutor(); - } - private List getSampleMaliciousRequests(String actor, String filterId) { Session session = this.sessionFactory.openSession(); Transaction txn = session.beginTransaction(); From f4ced15cdd3864f1a5329be22daf4e04c691fce9 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 17 Dec 2024 12:41:19 +0530 Subject: [PATCH 52/73] disabled debug logging for hibernate --- apps/threat-detection/src/main/resources/log4j2.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/apps/threat-detection/src/main/resources/log4j2.xml b/apps/threat-detection/src/main/resources/log4j2.xml index addedab226..521b62a5cb 100644 --- a/apps/threat-detection/src/main/resources/log4j2.xml +++ b/apps/threat-detection/src/main/resources/log4j2.xml @@ -22,19 +22,19 @@ - + - + - + - + - + From 8d4529439b43156a7099a7905283849dff74bdc0 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 17 Dec 2024 12:42:20 +0530 Subject: [PATCH 53/73] removed unused imports --- .../tasks/SendMaliciousRequestsToBackend.java | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java index b774b1f0ef..dbbc4ce4c3 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java @@ -16,18 +16,14 @@ import io.grpc.InsecureChannelCredentials; import io.grpc.ManagedChannel; import io.grpc.stub.StreamObserver; - +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.hibernate.Session; import org.hibernate.SessionFactory; import org.hibernate.Transaction; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.stream.Collectors; - /* This will send alerts to threat detection backend */ @@ -57,7 +53,8 @@ private List getSampleMaliciousRequests(String actor, Stri try { return session .createQuery( - "from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId order by m.createdAt desc", + "from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId order" + + " by m.createdAt desc", MaliciousEventEntity.class) .setParameter("actor", actor) .setParameter("filterId", filterId) From 36b999542396863e51d970d4fba2b462503945d7 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 17 Dec 2024 13:19:12 +0530 Subject: [PATCH 54/73] disabled debug logging for hibernate --- apps/threat-detection/src/main/resources/log4j2.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/apps/threat-detection/src/main/resources/log4j2.xml b/apps/threat-detection/src/main/resources/log4j2.xml index 521b62a5cb..133e6bdd7e 100644 --- a/apps/threat-detection/src/main/resources/log4j2.xml +++ b/apps/threat-detection/src/main/resources/log4j2.xml @@ -18,7 +18,7 @@ - + From 9c1c7576127577654fd34a0a33d1260ecadd8500 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 17 Dec 2024 13:47:22 +0530 Subject: [PATCH 55/73] committing kafka offsets manually now --- .../threat/detection/tasks/AbstractKafkaConsumerTask.java | 4 ++++ .../com/akto/threat/detection/tasks/FlushSampleDataTask.java | 2 +- .../threat/detection/tasks/MaliciousTrafficDetectorTask.java | 5 +++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java index e918efff11..c1eb835da1 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java @@ -48,6 +48,10 @@ public void run() { } processRecords(records); + + if (!records.isEmpty()) { + kafkaConsumer.commitSync(); + } } }); } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index ab3dcf3765..2e2e55d920 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -64,7 +64,7 @@ protected void processRecords(ConsumerRecords records) { Transaction txn = session.beginTransaction(); try { // Commit these events in 2 batches - for (int i = 0; i < events.size(); i += 2) { + for (int i = 0; i < events.size(); i++) { session.persist(events.get(i)); if (i % 50 == 0) { session.flush(); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 34031f2d57..92b819ae2a 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -96,6 +96,11 @@ public void run() { for (ConsumerRecord record : records) { processRecord(record); } + + if (!records.isEmpty()) { + // Should we commit even if there are no records ? + kafkaConsumer.commitSync(); + } } }); } From 88065771d5ab842795f5e2be53e8e52a47558bbd Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 17 Dec 2024 14:43:23 +0530 Subject: [PATCH 56/73] renamed threat detection backend service --- apps/pom.xml | 7 +++---- .../pom.xml | 8 ++++---- .../com/akto/threat/backend}/BackendServer.java | 14 +++++--------- .../main/java/com/akto/threat/backend}/Main.java | 4 ++-- .../threat/backend}/constants/KafkaTopic.java | 2 +- .../backend}/constants/MongoDBCollection.java | 2 +- .../db/AggregateSampleMaliciousEventModel.java | 2 +- .../threat/backend}/db/MaliciousEventModel.java | 2 +- .../interceptors/AuthenticationInterceptor.java | 3 +-- .../threat/backend}/interceptors/Constants.java | 6 +++--- .../backend}/service/DashboardService.java | 8 ++++---- .../backend}/service/MaliciousEventService.java | 16 +++++++--------- .../threat/backend}/tasks/FlushMessagesToDB.java | 15 +++++++-------- .../akto/threat/backend}/utils/KafkaUtils.java | 2 +- 14 files changed, 41 insertions(+), 50 deletions(-) rename apps/{threat-protection-backend => threat-detection-backend}/pom.xml (96%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/BackendServer.java (85%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/Main.java (96%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/constants/KafkaTopic.java (78%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/constants/MongoDBCollection.java (85%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/db/AggregateSampleMaliciousEventModel.java (98%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/db/MaliciousEventModel.java (99%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/interceptors/AuthenticationInterceptor.java (97%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/interceptors/Constants.java (89%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/service/DashboardService.java (95%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/service/MaliciousEventService.java (87%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/tasks/FlushMessagesToDB.java (93%) rename apps/{threat-protection-backend/src/main/java/com/akto/threat/protection => threat-detection-backend/src/main/java/com/akto/threat/backend}/utils/KafkaUtils.java (91%) diff --git a/apps/pom.xml b/apps/pom.xml index 0dcd5138f0..20dcc1cb26 100644 --- a/apps/pom.xml +++ b/apps/pom.xml @@ -137,16 +137,15 @@
- threat-protection-backend + threat-detection-backend - threat-protection-backend/pom.xml + threat-detection-backend/pom.xml - threat-protection-backend + threat-detection-backend - diff --git a/apps/threat-protection-backend/pom.xml b/apps/threat-detection-backend/pom.xml similarity index 96% rename from apps/threat-protection-backend/pom.xml rename to apps/threat-detection-backend/pom.xml index d63032154d..1933ab7387 100644 --- a/apps/threat-protection-backend/pom.xml +++ b/apps/threat-detection-backend/pom.xml @@ -10,8 +10,8 @@ ${revision} - com.akto.apps.threat-protection-backend - threat-protection-backend + com.akto.apps.threat-detection-backend + threat-detection-backend jar @@ -125,7 +125,7 @@ - com.akto.threat.detection.Main + com.akto.threat.backend.Main @@ -152,7 +152,7 @@ true - com.akto.threat.detection.Main + com.akto.threat.backend.Main dependency-jars/ diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java similarity index 85% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java index 84b397f0f6..294cf5971b 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/BackendServer.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java @@ -1,19 +1,15 @@ -package com.akto.threat.protection; +package com.akto.threat.backend; -import com.akto.kafka.Kafka; import com.akto.kafka.KafkaConfig; -import com.akto.threat.protection.interceptors.AuthenticationInterceptor; -import com.akto.threat.protection.service.DashboardService; -import com.akto.threat.protection.service.MaliciousEventService; - +import com.akto.threat.backend.interceptors.AuthenticationInterceptor; +import com.akto.threat.backend.service.DashboardService; +import com.akto.threat.backend.service.MaliciousEventService; +import com.mongodb.client.MongoClient; import io.grpc.Server; import io.grpc.ServerBuilder; - import java.io.IOException; import java.util.concurrent.TimeUnit; -import com.mongodb.client.MongoClient; - public class BackendServer { private final int port; private final Server server; diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java similarity index 96% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java index b147ba51df..642c781ffc 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/Main.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection; +package com.akto.threat.backend; import static org.bson.codecs.configuration.CodecRegistries.fromProviders; import static org.bson.codecs.configuration.CodecRegistries.fromRegistries; @@ -7,7 +7,7 @@ import com.akto.kafka.KafkaConfig; import com.akto.kafka.KafkaConsumerConfig; import com.akto.kafka.KafkaProducerConfig; -import com.akto.threat.protection.tasks.FlushMessagesToDB; +import com.akto.threat.backend.tasks.FlushMessagesToDB; import com.mongodb.ConnectionString; import com.mongodb.MongoClientSettings; import com.mongodb.ReadPreference; diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/KafkaTopic.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/constants/KafkaTopic.java similarity index 78% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/KafkaTopic.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/constants/KafkaTopic.java index 6c528ba08b..3f056fc39a 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/KafkaTopic.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/constants/KafkaTopic.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection.constants; +package com.akto.threat.backend.constants; public class KafkaTopic { public static class ThreatDetection { diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/MongoDBCollection.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/constants/MongoDBCollection.java similarity index 85% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/MongoDBCollection.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/constants/MongoDBCollection.java index 77bbcb93ee..879af94557 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/constants/MongoDBCollection.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/constants/MongoDBCollection.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection.constants; +package com.akto.threat.backend.constants; public class MongoDBCollection { public static class ThreatDetection { diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/AggregateSampleMaliciousEventModel.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/db/AggregateSampleMaliciousEventModel.java similarity index 98% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/AggregateSampleMaliciousEventModel.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/db/AggregateSampleMaliciousEventModel.java index bbc8338bc2..cc5f876d8b 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/AggregateSampleMaliciousEventModel.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/db/AggregateSampleMaliciousEventModel.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection.db; +package com.akto.threat.backend.db; import com.akto.dto.type.URLMethods.Method; import java.util.UUID; diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/db/MaliciousEventModel.java similarity index 99% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/db/MaliciousEventModel.java index 5735454442..56156eab9c 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/db/MaliciousEventModel.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/db/MaliciousEventModel.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection.db; +package com.akto.threat.backend.db; import com.akto.dto.type.URLMethods; import java.util.UUID; diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/AuthenticationInterceptor.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java similarity index 97% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/AuthenticationInterceptor.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java index 18c428e654..813a237e55 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/AuthenticationInterceptor.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection.interceptors; +package com.akto.threat.backend.interceptors; import com.akto.dao.ConfigsDao; import com.akto.dto.Config; @@ -6,7 +6,6 @@ import io.jsonwebtoken.Claims; import io.jsonwebtoken.Jws; import io.jsonwebtoken.Jwts; - import java.io.IOException; import java.security.KeyFactory; import java.security.NoSuchAlgorithmException; diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/Constants.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/Constants.java similarity index 89% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/Constants.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/Constants.java index 33979da98c..7c5eb033ea 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/interceptors/Constants.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/Constants.java @@ -1,10 +1,10 @@ -package com.akto.threat.protection.interceptors; +package com.akto.threat.backend.interceptors; + +import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; import io.grpc.Context; import io.grpc.Metadata; -import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; - public class Constants { public static final Metadata.Key AUTHORIZATION_METADATA_KEY = Metadata.Key.of("Authorization", ASCII_STRING_MARSHALLER); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java similarity index 95% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java index d0009ee9c8..eae48feda0 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/DashboardService.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection.service; +package com.akto.threat.backend.service; import com.akto.proto.threat_protection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceImplBase; @@ -6,9 +6,9 @@ import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse; import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse; -import com.akto.threat.protection.constants.MongoDBCollection; -import com.akto.threat.protection.db.MaliciousEventModel; -import com.akto.threat.protection.interceptors.Constants; +import com.akto.threat.backend.constants.MongoDBCollection; +import com.akto.threat.backend.db.MaliciousEventModel; +import com.akto.threat.backend.interceptors.Constants; import com.mongodb.BasicDBObject; import com.mongodb.client.DistinctIterable; import com.mongodb.client.MongoClient; diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java similarity index 87% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java index d50b8e845b..faf9fd1857 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/service/MaliciousEventService.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection.service; +package com.akto.threat.backend.service; import com.akto.dto.type.URLMethods; import com.akto.kafka.Kafka; @@ -9,13 +9,12 @@ import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; -import com.akto.threat.protection.constants.KafkaTopic; -import com.akto.threat.protection.constants.MongoDBCollection; -import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; -import com.akto.threat.protection.db.MaliciousEventModel; -import com.akto.threat.protection.interceptors.Constants; -import com.akto.threat.protection.utils.KafkaUtils; -import com.mongodb.client.model.WriteModel; +import com.akto.threat.backend.constants.KafkaTopic; +import com.akto.threat.backend.constants.MongoDBCollection; +import com.akto.threat.backend.db.AggregateSampleMaliciousEventModel; +import com.akto.threat.backend.db.MaliciousEventModel; +import com.akto.threat.backend.interceptors.Constants; +import com.akto.threat.backend.utils.KafkaUtils; import io.grpc.stub.StreamObserver; import java.util.ArrayList; import java.util.List; @@ -36,7 +35,6 @@ public void recordMaliciousEvent( MaliciousEventMessage evt = request.getMaliciousEvent(); String actor = evt.getActor(); String filterId = evt.getFilterId(); - List> bulkUpdates = new ArrayList<>(); int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); EventType eventType = evt.getEventType(); diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java similarity index 93% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java index 438c2893b0..f018e7be5c 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/tasks/FlushMessagesToDB.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java @@ -1,11 +1,11 @@ -package com.akto.threat.protection.tasks; +package com.akto.threat.backend.tasks; import com.akto.kafka.KafkaConfig; import com.akto.runtime.utils.Utils; -import com.akto.threat.protection.constants.KafkaTopic; -import com.akto.threat.protection.constants.MongoDBCollection; -import com.akto.threat.protection.db.AggregateSampleMaliciousEventModel; -import com.akto.threat.protection.db.MaliciousEventModel; +import com.akto.threat.backend.constants.KafkaTopic; +import com.akto.threat.backend.constants.MongoDBCollection; +import com.akto.threat.backend.db.AggregateSampleMaliciousEventModel; +import com.akto.threat.backend.db.MaliciousEventModel; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -14,13 +14,12 @@ import com.mongodb.client.model.BulkWriteOptions; import com.mongodb.client.model.InsertOneModel; import com.mongodb.client.model.WriteModel; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; - import java.time.Duration; import java.util.*; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; public class FlushMessagesToDB { diff --git a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/utils/KafkaUtils.java similarity index 91% rename from apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java rename to apps/threat-detection-backend/src/main/java/com/akto/threat/backend/utils/KafkaUtils.java index 50e2bb306c..e6c6007b70 100644 --- a/apps/threat-protection-backend/src/main/java/com/akto/threat/protection/utils/KafkaUtils.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/utils/KafkaUtils.java @@ -1,4 +1,4 @@ -package com.akto.threat.protection.utils; +package com.akto.threat.backend.utils; import com.google.gson.Gson; import com.mongodb.BasicDBObject; From 7d3b40765ef1693153ec3585e5ea5d82b520eb25 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 17 Dec 2024 18:20:16 +0530 Subject: [PATCH 57/73] added health check server --- apps/threat-detection-backend/Dockerfile | 4 ++ apps/threat-detection-backend/pom.xml | 19 +++++- .../threat/backend/HealthCheckServer.java | 37 +++++++++++ .../java/com/akto/threat/backend/Main.java | 64 ++++++++++++------- .../backend/tasks/FlushMessagesToDB.java | 39 ++++++----- libs/protobuf/pom.xml | 20 ++++-- 6 files changed, 133 insertions(+), 50 deletions(-) create mode 100644 apps/threat-detection-backend/Dockerfile create mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java diff --git a/apps/threat-detection-backend/Dockerfile b/apps/threat-detection-backend/Dockerfile new file mode 100644 index 0000000000..e2ec8f2cbf --- /dev/null +++ b/apps/threat-detection-backend/Dockerfile @@ -0,0 +1,4 @@ +FROM openjdk +WORKDIR /app +COPY ./target/threat-detection-backend-1.0-SNAPSHOT-jar-with-dependencies.jar /app/threat-detection-backend-1.0-SNAPSHOT-jar-with-dependencies.jar +CMD "java" "-XX:+ExitOnOutOfMemoryError" "-jar" "/app/threat-detection-backend-1.0-SNAPSHOT-jar-with-dependencies.jar" \ No newline at end of file diff --git a/apps/threat-detection-backend/pom.xml b/apps/threat-detection-backend/pom.xml index 1933ab7387..a27231a6a0 100644 --- a/apps/threat-detection-backend/pom.xml +++ b/apps/threat-detection-backend/pom.xml @@ -1,9 +1,13 @@ + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + + 9.4.27.v20200227 + + com.akto.apps apps @@ -62,6 +66,17 @@ compile + + org.eclipse.jetty + jetty-server + ${jetty.version} + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + + diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java new file mode 100644 index 0000000000..9bfe585b65 --- /dev/null +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java @@ -0,0 +1,37 @@ +package com.akto.threat.backend; + +import java.io.IOException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.servlet.ServletHandler; +import org.eclipse.jetty.servlet.ServletHolder; + +public class HealthCheckServer { + public static void startHttpServer() throws Exception { + + Server server = new Server(9090); // HTTP server port + + // Create and configure the servlet handler + ServletHandler handler = new ServletHandler(); + server.setHandler(handler); + + // Define a simple servlet to handle HTTP requests + handler.addServletWithMapping( + new ServletHolder( + new HttpServlet() { + @Override + protected void doGet(HttpServletRequest req, HttpServletResponse resp) + throws IOException { + resp.getWriter().println("OK"); + } + }), + "/"); + + // Start the HTTP server + server.start(); + System.out.println("HTTP Server started on port 8080"); + server.join(); + } +} diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java index 642c781ffc..3520be924b 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java @@ -14,14 +14,16 @@ import com.mongodb.WriteConcern; import com.mongodb.client.MongoClient; import com.mongodb.client.MongoClients; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.bson.codecs.configuration.CodecRegistry; import org.bson.codecs.pojo.PojoCodecProvider; public class Main { public static void main(String[] args) throws Exception { - String mongoURI = System.getenv("AKTO_MONGO_CONN"); + ExecutorService executor = Executors.newFixedThreadPool(2); - DaoInit.init(new ConnectionString(mongoURI)); + DaoInit.init(new ConnectionString(System.getenv("AKTO_MONGO_CONN"))); ConnectionString connectionString = new ConnectionString(System.getenv("AKTO_THREAT_PROTECTION_MONGO_CONN")); @@ -37,28 +39,44 @@ public static void main(String[] args) throws Exception { .codecRegistry(codecRegistry) .build(); - try (MongoClient threatProtectionMongo = MongoClients.create(clientSettings)) { - KafkaConfig internalKafkaConfig = - KafkaConfig.newBuilder() - .setBootstrapServers(System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL")) - .setGroupId("akto.threat_protection.flush_db") - .setConsumerConfig( - KafkaConsumerConfig.newBuilder() - .setMaxPollRecords(100) - .setPollDurationMilli(100) - .build()) - .setProducerConfig( - KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(1000).build()) - .build(); + MongoClient threatProtectionMongo = MongoClients.create(clientSettings); + KafkaConfig internalKafkaConfig = + KafkaConfig.newBuilder() + .setBootstrapServers(System.getenv("THREAT_EVENTS_KAFKA_BROKER_URL")) + .setGroupId("akto.threat_protection.flush_db") + .setConsumerConfig( + KafkaConsumerConfig.newBuilder() + .setMaxPollRecords(100) + .setPollDurationMilli(100) + .build()) + .setProducerConfig( + KafkaProducerConfig.newBuilder().setBatchSize(100).setLingerMs(1000).build()) + .build(); + + new FlushMessagesToDB(internalKafkaConfig, threatProtectionMongo).run(); - new FlushMessagesToDB(internalKafkaConfig, threatProtectionMongo).run(); + executor.submit( + () -> { + int port = + Integer.parseInt( + System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); + BackendServer server = + new BackendServer(port, threatProtectionMongo, internalKafkaConfig); + try { + server.start(); + server.blockUntilShutdown(); + } catch (Exception e) { + e.printStackTrace(); + } + }); - int port = - Integer.parseInt( - System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); - BackendServer server = new BackendServer(port, threatProtectionMongo, internalKafkaConfig); - server.start(); - server.blockUntilShutdown(); - } + executor.submit( + () -> { + try { + HealthCheckServer.startHttpServer(); + } catch (Exception e) { + e.printStackTrace(); + } + }); } } diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java index f018e7be5c..78ea5095ab 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java @@ -43,30 +43,29 @@ public FlushMessagesToDB(KafkaConfig kafkaConfig, MongoClient mongoClient) { this.mClient = mongoClient; } - private static ExecutorService getPollingExecutor() { - return Executors.newSingleThreadExecutor(); - } - public void run() { + ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); this.kafkaConsumer.subscribe( Collections.singletonList(KafkaTopic.ThreatDetection.INTERNAL_DB_MESSAGES)); - getPollingExecutor() - .execute( - () -> { - // Poll data from Kafka topic - while (true) { - ConsumerRecords records = - kafkaConsumer.poll( - Duration.ofMillis( - this.kafkaConfig.getConsumerConfig().getPollDurationMilli())); - if (records.isEmpty()) { - continue; - } - - processRecords(records); - } - }); + pollingExecutor.execute( + () -> { + // Poll data from Kafka topic + while (true) { + ConsumerRecords records = + kafkaConsumer.poll( + Duration.ofMillis(this.kafkaConfig.getConsumerConfig().getPollDurationMilli())); + if (records.isEmpty()) { + continue; + } + + processRecords(records); + + if (!records.isEmpty()) { + kafkaConsumer.commitSync(); + } + } + }); } private void processRecords(ConsumerRecords records) { diff --git a/libs/protobuf/pom.xml b/libs/protobuf/pom.xml index 2542736113..b60e78371d 100644 --- a/libs/protobuf/pom.xml +++ b/libs/protobuf/pom.xml @@ -1,7 +1,7 @@ + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 com.akto.libs @@ -9,6 +9,11 @@ ${revision} + + 1.69.0 + + + com.akto.libs.protobuf protobuf jar @@ -23,17 +28,22 @@ io.grpc grpc-core - 1.68.1 + ${grpc.version} io.grpc grpc-protobuf - 1.68.1 + ${grpc.version} io.grpc grpc-stub - 1.68.1 + ${grpc.version} + + + io.grpc + grpc-services + ${grpc.version} com.google.protobuf From b506fe4e0cee90a74241311218104dc9f42d0232 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Tue, 17 Dec 2024 18:38:06 +0530 Subject: [PATCH 58/73] building threat detection backend image in staging workflow --- .github/workflows/staging.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/staging.yml b/.github/workflows/staging.yml index afb68670e2..ff95a5d2bb 100644 --- a/.github/workflows/staging.yml +++ b/.github/workflows/staging.yml @@ -86,6 +86,8 @@ jobs: docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-internal:$IMAGE_TAG . --push cd ../threat-detection docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-threat-detection:$IMAGE_TAG . --push + cd ../threat-detection-backend + docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-threat-detection-backend:$IMAGE_TAG . --push cd ../source-code-analyser docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/source-code-analyser:$IMAGE_TAG . --push From cfe9b1baf35590c455c5fabe5a982bae15a01d9e Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 18 Dec 2024 11:38:42 +0530 Subject: [PATCH 59/73] refactored generated proto files --- .gitignore | 1 - .../SuspectSampleDataAction.java | 16 ++++----- .../threat/backend/HealthCheckServer.java | 6 ++-- .../AuthenticationInterceptor.java | 7 ++-- .../backend/service/DashboardService.java | 12 +++---- .../service/MaliciousEventService.java | 12 +++---- .../akto/threat/detection/grpc/AuthToken.java | 34 ------------------- .../WindowBasedThresholdNotifier.java | 4 +-- .../detection/tasks/FlushSampleDataTask.java | 2 +- .../tasks/MaliciousTrafficDetectorTask.java | 6 ++-- .../tasks/SendMaliciousRequestsToBackend.java | 16 ++++----- buf.gen.yaml | 6 ++-- libs/protobuf/.gitignore | 1 + .../java/com/akto/grpc/auth}/AuthToken.java | 21 ++++++++++-- .../dashboard/v1/message.proto | 8 ++--- .../event_type/v1/event_type.proto | 4 +-- .../message/malicious_event/v1/message.proto | 8 ++--- .../message/sample_request/v1/message.proto | 4 +-- .../dashboard_service/v1/service.proto | 8 ++--- .../malicious_alert_service/v1/service.proto | 23 +++++++++++++ .../malicious_alert_service/v1/service.proto | 23 ------------- scripts/proto-gen.sh | 2 +- 22 files changed, 104 insertions(+), 120 deletions(-) delete mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/grpc/AuthToken.java create mode 100644 libs/protobuf/.gitignore rename {apps/dashboard/src/main/java/com/akto/grpc => libs/protobuf/src/main/java/com/akto/grpc/auth}/AuthToken.java (59%) rename protobuf/{threat_protection => threat_detection}/message/malicious_event/dashboard/v1/message.proto (52%) rename protobuf/{threat_protection => threat_detection}/message/malicious_event/event_type/v1/event_type.proto (54%) rename protobuf/{threat_protection => threat_detection}/message/malicious_event/v1/message.proto (54%) rename protobuf/{threat_protection => threat_detection}/message/sample_request/v1/message.proto (68%) rename protobuf/{threat_protection => threat_detection}/service/dashboard_service/v1/service.proto (69%) create mode 100644 protobuf/threat_detection/service/malicious_alert_service/v1/service.proto delete mode 100644 protobuf/threat_protection/service/malicious_alert_service/v1/service.proto diff --git a/.gitignore b/.gitignore index b02305d731..685da2accc 100644 --- a/.gitignore +++ b/.gitignore @@ -13,7 +13,6 @@ **/.settings **/dist **/gen -libs/protobuf/src libawesome.dylib temp_* *.templates-config.json diff --git a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java index 397612583f..b9e1061c0f 100644 --- a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java +++ b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java @@ -3,13 +3,13 @@ import com.akto.action.UserAction; import com.akto.dto.traffic.SuspectSampleData; import com.akto.dto.type.URLMethods; -import com.akto.grpc.AuthToken; -import com.akto.proto.threat_protection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; -import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc; -import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceBlockingStub; -import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest; -import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse; -import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; +import com.akto.grpc.auth.AuthToken; +import com.akto.proto.generated.threat_detection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.DashboardServiceGrpc; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceBlockingStub; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersRequest; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersResponse; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.ListMaliciousRequestsRequest; import io.grpc.Grpc; import io.grpc.InsecureChannelCredentials; import io.grpc.ManagedChannel; @@ -41,7 +41,7 @@ public SuspectSampleDataAction() { this.dsServiceStub = DashboardServiceGrpc.newBlockingStub(channel) .withCallCredentials( - new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); + new AuthToken(System.getenv("AKTO_threat_detection_BACKEND_TOKEN"))); } public String fetchSampleDataV2() { diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java index 9bfe585b65..2a54962869 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java @@ -9,9 +9,11 @@ import org.eclipse.jetty.servlet.ServletHolder; public class HealthCheckServer { + public static final int PORT = 9090; + public static void startHttpServer() throws Exception { - Server server = new Server(9090); // HTTP server port + Server server = new Server(PORT); // HTTP server port // Create and configure the servlet handler ServletHandler handler = new ServletHandler(); @@ -31,7 +33,7 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) // Start the HTTP server server.start(); - System.out.println("HTTP Server started on port 8080"); + System.out.println("HTTP Server started on port " + PORT); server.join(); } } diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java index 813a237e55..95683b1c63 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java @@ -2,6 +2,7 @@ import com.akto.dao.ConfigsDao; import com.akto.dto.Config; +import com.akto.grpc.auth.AuthToken; import io.grpc.*; import io.jsonwebtoken.Claims; import io.jsonwebtoken.Jws; @@ -18,9 +19,9 @@ public class AuthenticationInterceptor implements ServerInterceptor { @Override public ServerCall.Listener interceptCall( ServerCall call, Metadata metadata, ServerCallHandler next) { - String value = metadata.get(Constants.AUTHORIZATION_METADATA_KEY); + String token = AuthToken.getBearerTokenFromMeta(metadata).orElse(null); - if (value == null) { + if (token == null) { call.close( Status.UNAUTHENTICATED.withDescription("Authorization token is required"), metadata); return null; @@ -29,7 +30,7 @@ public ServerCall.Listener interceptCall( try { PublicKey publicKey = getPublicKey(); Jws claims = - Jwts.parserBuilder().setSigningKey(publicKey).build().parseClaimsJws(value); + Jwts.parserBuilder().setSigningKey(publicKey).build().parseClaimsJws(token); int accountId = (int) claims.getBody().get("accountId"); Context ctx = Context.current().withValue(Constants.ACCOUNT_ID_CONTEXT_KEY, accountId); return Contexts.interceptCall(ctx, call, metadata, next); diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java index eae48feda0..cd83111b62 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java @@ -1,11 +1,11 @@ package com.akto.threat.backend.service; -import com.akto.proto.threat_protection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; -import com.akto.proto.threat_protection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceImplBase; -import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersRequest; -import com.akto.proto.threat_protection.service.dashboard_service.v1.FetchAlertFiltersResponse; -import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsRequest; -import com.akto.proto.threat_protection.service.dashboard_service.v1.ListMaliciousRequestsResponse; +import com.akto.proto.generated.threat_detection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceImplBase; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersRequest; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersResponse; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.ListMaliciousRequestsRequest; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.ListMaliciousRequestsResponse; import com.akto.threat.backend.constants.MongoDBCollection; import com.akto.threat.backend.db.MaliciousEventModel; import com.akto.threat.backend.interceptors.Constants; diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java index faf9fd1857..fe3f0db472 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java @@ -3,12 +3,12 @@ import com.akto.dto.type.URLMethods; import com.akto.kafka.Kafka; import com.akto.kafka.KafkaConfig; -import com.akto.proto.threat_protection.message.malicious_event.event_type.v1.EventType; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventMessage; -import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; +import com.akto.proto.generated.threat_detection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventMessage; +import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; +import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; +import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; import com.akto.threat.backend.constants.KafkaTopic; import com.akto.threat.backend.constants.MongoDBCollection; import com.akto.threat.backend.db.AggregateSampleMaliciousEventModel; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/grpc/AuthToken.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/grpc/AuthToken.java deleted file mode 100644 index 17bc9cbc38..0000000000 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/grpc/AuthToken.java +++ /dev/null @@ -1,34 +0,0 @@ -package com.akto.threat.detection.grpc; - -import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; - -import io.grpc.CallCredentials; -import io.grpc.Metadata; -import io.grpc.Status; -import java.util.concurrent.Executor; - -public class AuthToken extends CallCredentials { - - private final String token; - public static final Metadata.Key AUTHORIZATION_METADATA_KEY = - Metadata.Key.of("Authorization", ASCII_STRING_MARSHALLER); - - public AuthToken(String token) { - this.token = token; - } - - @Override - public void applyRequestMetadata( - RequestInfo requestInfo, Executor appExecutor, MetadataApplier applier) { - appExecutor.execute( - () -> { - try { - Metadata headers = new Metadata(); - headers.put(AUTHORIZATION_METADATA_KEY, token); - applier.apply(headers); - } catch (Throwable e) { - applier.fail(Status.UNAUTHENTICATED.withCause(e)); - } - }); - } -} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java index 24c87f9b4c..f9e9b9fbfb 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java @@ -1,8 +1,8 @@ package com.akto.threat.detection.smart_event_detector.window_based; -import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; -import com.akto.threat.detection.cache.CounterCache; import com.akto.dto.api_protection_parse_layer.Rule; +import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.threat.detection.cache.CounterCache; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ConcurrentHashMap; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 2e2e55d920..8e82b688c1 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -2,7 +2,7 @@ import com.akto.dto.type.URLMethods; import com.akto.kafka.KafkaConfig; -import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 92b819ae2a..6a185cda84 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -16,9 +16,9 @@ import com.akto.hybrid_parsers.HttpCallParser; import com.akto.kafka.Kafka; import com.akto.kafka.KafkaConfig; -import com.akto.proto.threat_protection.message.malicious_event.event_type.v1.EventType; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventMessage; -import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.proto.generated.threat_detection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventMessage; +import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.rules.TestPlugin; import com.akto.runtime.utils.Utils; import com.akto.test_editor.execution.VariableResolver; diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java index dbbc4ce4c3..cdfa6a378a 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java @@ -1,15 +1,15 @@ package com.akto.threat.detection.tasks; +import com.akto.grpc.auth.AuthToken; import com.akto.kafka.KafkaConfig; -import com.akto.proto.threat_protection.message.malicious_event.event_type.v1.EventType; -import com.akto.proto.threat_protection.message.malicious_event.v1.MaliciousEventMessage; -import com.akto.proto.threat_protection.message.sample_request.v1.SampleMaliciousRequest; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; -import com.akto.proto.threat_protection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; +import com.akto.proto.generated.threat_detection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventMessage; +import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; +import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; +import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; -import com.akto.threat.detection.grpc.AuthToken; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; import io.grpc.Grpc; @@ -44,7 +44,7 @@ public SendMaliciousRequestsToBackend( this.consumerServiceStub = MaliciousEventServiceGrpc.newStub(channel) .withCallCredentials( - new AuthToken(System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"))); + new AuthToken(System.getenv("AKTO_threat_detection_BACKEND_TOKEN"))); } private List getSampleMaliciousRequests(String actor, String filterId) { diff --git a/buf.gen.yaml b/buf.gen.yaml index ac36b4761c..f282e25075 100644 --- a/buf.gen.yaml +++ b/buf.gen.yaml @@ -1,7 +1,5 @@ version: v2 -clean: true - managed: enabled: true @@ -10,10 +8,10 @@ managed: value: true - file_option: java_package_prefix - value: com.akto.proto + value: com.akto.proto.generated plugins: - - remote: buf.build/grpc/java:v1.68.1 + - remote: buf.build/grpc/java:v1.69.0 out: libs/protobuf/src/main/java - remote: buf.build/protocolbuffers/java:v28.3 diff --git a/libs/protobuf/.gitignore b/libs/protobuf/.gitignore new file mode 100644 index 0000000000..90a7881dc5 --- /dev/null +++ b/libs/protobuf/.gitignore @@ -0,0 +1 @@ +src/main/java/com/akto/proto/generated \ No newline at end of file diff --git a/apps/dashboard/src/main/java/com/akto/grpc/AuthToken.java b/libs/protobuf/src/main/java/com/akto/grpc/auth/AuthToken.java similarity index 59% rename from apps/dashboard/src/main/java/com/akto/grpc/AuthToken.java rename to libs/protobuf/src/main/java/com/akto/grpc/auth/AuthToken.java index c10bf4f224..670e945419 100644 --- a/apps/dashboard/src/main/java/com/akto/grpc/AuthToken.java +++ b/libs/protobuf/src/main/java/com/akto/grpc/auth/AuthToken.java @@ -1,10 +1,11 @@ -package com.akto.grpc; +package com.akto.grpc.auth; import static io.grpc.Metadata.ASCII_STRING_MARSHALLER; import io.grpc.CallCredentials; import io.grpc.Metadata; import io.grpc.Status; +import java.util.Optional; import java.util.concurrent.Executor; public class AuthToken extends CallCredentials { @@ -14,7 +15,10 @@ public class AuthToken extends CallCredentials { Metadata.Key.of("Authorization", ASCII_STRING_MARSHALLER); public AuthToken(String token) { - this.token = token; + if (token == null || token.trim().isEmpty()) { + throw new IllegalArgumentException("Token cannot be null or empty"); + } + this.token = String.format("Bearer %s", token.trim()); } @Override @@ -31,4 +35,17 @@ public void applyRequestMetadata( } }); } + + public static Optional getBearerTokenFromMeta(Metadata metadata) { + String val = metadata.get(AUTHORIZATION_METADATA_KEY); + if (val == null || val.trim().isEmpty()) { + return Optional.empty(); + } + + if (val.startsWith("Bearer ")) { + return Optional.of(val.substring("Bearer ".length()).trim()); + } + + return Optional.empty(); + } } diff --git a/protobuf/threat_protection/message/malicious_event/dashboard/v1/message.proto b/protobuf/threat_detection/message/malicious_event/dashboard/v1/message.proto similarity index 52% rename from protobuf/threat_protection/message/malicious_event/dashboard/v1/message.proto rename to protobuf/threat_detection/message/malicious_event/dashboard/v1/message.proto index 7864d62a1c..fd9e0c806b 100644 --- a/protobuf/threat_protection/message/malicious_event/dashboard/v1/message.proto +++ b/protobuf/threat_detection/message/malicious_event/dashboard/v1/message.proto @@ -1,11 +1,11 @@ syntax = "proto3"; -package threat_protection.message.malicious_event.dashboard.v1; +package threat_detection.message.malicious_event.dashboard.v1; -import "threat_protection/message/malicious_event/event_type/v1/event_type.proto"; +import "threat_detection/message/malicious_event/event_type/v1/event_type.proto"; option java_outer_classname = "MaliciousEventDashboardProto"; -option java_package = "threat_protection.message.malicious_event.dashboard.v1"; +option java_package = "threat_detection.message.malicious_event.dashboard.v1"; message DashboardMaliciousEventMessage { string id = 1; @@ -18,5 +18,5 @@ message DashboardMaliciousEventMessage { int32 api_collection_id = 8; string payload = 9; string country = 10; - threat_protection.message.malicious_event.event_type.v1.EventType event_type = 11; + threat_detection.message.malicious_event.event_type.v1.EventType event_type = 11; } diff --git a/protobuf/threat_protection/message/malicious_event/event_type/v1/event_type.proto b/protobuf/threat_detection/message/malicious_event/event_type/v1/event_type.proto similarity index 54% rename from protobuf/threat_protection/message/malicious_event/event_type/v1/event_type.proto rename to protobuf/threat_detection/message/malicious_event/event_type/v1/event_type.proto index 440c7a8355..5887f5c6e4 100644 --- a/protobuf/threat_protection/message/malicious_event/event_type/v1/event_type.proto +++ b/protobuf/threat_detection/message/malicious_event/event_type/v1/event_type.proto @@ -1,9 +1,9 @@ syntax = "proto3"; -package threat_protection.message.malicious_event.event_type.v1; +package threat_detection.message.malicious_event.event_type.v1; option java_outer_classname = "MaliciousEventProto"; -option java_package = "threat_protection.message.malicious_event.event_type.v1"; +option java_package = "threat_detection.message.malicious_event.event_type.v1"; enum EventType { EVENT_TYPE_UNSPECIFIED = 0; diff --git a/protobuf/threat_protection/message/malicious_event/v1/message.proto b/protobuf/threat_detection/message/malicious_event/v1/message.proto similarity index 54% rename from protobuf/threat_protection/message/malicious_event/v1/message.proto rename to protobuf/threat_detection/message/malicious_event/v1/message.proto index 66175ad8ff..8c9fabff98 100644 --- a/protobuf/threat_protection/message/malicious_event/v1/message.proto +++ b/protobuf/threat_detection/message/malicious_event/v1/message.proto @@ -1,11 +1,11 @@ syntax = "proto3"; -package threat_protection.message.malicious_event.v1; +package threat_detection.message.malicious_event.v1; -import "threat_protection/message/malicious_event/event_type/v1/event_type.proto"; +import "threat_detection/message/malicious_event/event_type/v1/event_type.proto"; option java_outer_classname = "MaliciousEventProto"; -option java_package = "threat_protection.message.malicious_event.v1"; +option java_package = "threat_detection.message.malicious_event.v1"; message MaliciousEventMessage { string actor = 1; @@ -16,5 +16,5 @@ message MaliciousEventMessage { string latest_api_method = 6; int32 latest_api_collection_id = 7; string latest_api_payload = 8; - threat_protection.message.malicious_event.event_type.v1.EventType event_type = 9; + threat_detection.message.malicious_event.event_type.v1.EventType event_type = 9; } diff --git a/protobuf/threat_protection/message/sample_request/v1/message.proto b/protobuf/threat_detection/message/sample_request/v1/message.proto similarity index 68% rename from protobuf/threat_protection/message/sample_request/v1/message.proto rename to protobuf/threat_detection/message/sample_request/v1/message.proto index a8263d76c5..1020e56636 100644 --- a/protobuf/threat_protection/message/sample_request/v1/message.proto +++ b/protobuf/threat_detection/message/sample_request/v1/message.proto @@ -1,9 +1,9 @@ syntax = "proto3"; -package threat_protection.message.sample_request.v1; +package threat_detection.message.sample_request.v1; option java_outer_classname = "SampleRequestProto"; -option java_package = "threat_protection.message.sample_request.v1"; +option java_package = "threat_detection.message.sample_request.v1"; message SampleMaliciousRequest { string ip = 1; diff --git a/protobuf/threat_protection/service/dashboard_service/v1/service.proto b/protobuf/threat_detection/service/dashboard_service/v1/service.proto similarity index 69% rename from protobuf/threat_protection/service/dashboard_service/v1/service.proto rename to protobuf/threat_detection/service/dashboard_service/v1/service.proto index 0a44c43113..56320d0408 100644 --- a/protobuf/threat_protection/service/dashboard_service/v1/service.proto +++ b/protobuf/threat_detection/service/dashboard_service/v1/service.proto @@ -1,15 +1,15 @@ syntax = "proto3"; -package threat_protection.service.dashboard_service.v1; +package threat_detection.service.dashboard_service.v1; -import "threat_protection/message/malicious_event/dashboard/v1/message.proto"; +import "threat_detection/message/malicious_event/dashboard/v1/message.proto"; // Dashboard service which the dashboard actions will call instead of directly calling DB option java_outer_classname = "DashboardServiceProto"; -option java_package = "threat_protection.service.dashboard_service.v1"; +option java_package = "threat_detection.service.dashboard_service.v1"; message ListMaliciousRequestsResponse { - repeated threat_protection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage malicious_events = 1; + repeated threat_detection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage malicious_events = 1; int32 total = 2; int32 page = 3; } diff --git a/protobuf/threat_detection/service/malicious_alert_service/v1/service.proto b/protobuf/threat_detection/service/malicious_alert_service/v1/service.proto new file mode 100644 index 0000000000..16238d2d7a --- /dev/null +++ b/protobuf/threat_detection/service/malicious_alert_service/v1/service.proto @@ -0,0 +1,23 @@ +syntax = "proto3"; + +package threat_detection.service.malicious_alert_service.v1; + +import "threat_detection/message/malicious_event/v1/message.proto"; +import "threat_detection/message/sample_request/v1/message.proto"; + +// This is a consumer service for recording malicious alerts +// For dashboard purposes we will have a separate service to retrieve these events. +option java_outer_classname = "MaliciousAlertServiceProto"; +option java_package = "threat_detection.service.malicious_alert_service.v1"; + +message RecordMaliciousEventResponse { +} + +message RecordMaliciousEventRequest { + threat_detection.message.malicious_event.v1.MaliciousEventMessage malicious_event = 1; + repeated threat_detection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2; +} + +service MaliciousEventService { + rpc RecordMaliciousEvent(RecordMaliciousEventRequest) returns (RecordMaliciousEventResponse) {} +} \ No newline at end of file diff --git a/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto b/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto deleted file mode 100644 index c344ac3e01..0000000000 --- a/protobuf/threat_protection/service/malicious_alert_service/v1/service.proto +++ /dev/null @@ -1,23 +0,0 @@ -syntax = "proto3"; - -package threat_protection.service.malicious_alert_service.v1; - -import "threat_protection/message/malicious_event/v1/message.proto"; -import "threat_protection/message/sample_request/v1/message.proto"; - -// This is a consumer service for recording malicious alerts -// For dashboard purposes we will have a separate service to retrieve these events. -option java_outer_classname = "MaliciousAlertServiceProto"; -option java_package = "threat_protection.service.malicious_alert_service.v1"; - -message RecordMaliciousEventResponse { -} - -message RecordMaliciousEventRequest { - threat_protection.message.malicious_event.v1.MaliciousEventMessage malicious_event = 1; - repeated threat_protection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2; -} - -service MaliciousEventService { - rpc RecordMaliciousEvent(RecordMaliciousEventRequest) returns (RecordMaliciousEventResponse) {} -} \ No newline at end of file diff --git a/scripts/proto-gen.sh b/scripts/proto-gen.sh index d503d4da52..ff32a1f164 100644 --- a/scripts/proto-gen.sh +++ b/scripts/proto-gen.sh @@ -8,5 +8,5 @@ if ! command -v buf >/dev/null 2>&1; then fi buf lint protobuf -rm -rf ./libs/protobuf/src +rm -rf libs/protobuf/src/main/java/com/akto/proto/generated/ buf generate protobuf \ No newline at end of file From 3ae54bea465f57f0774c04bd17e1c8a91d97281c Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 18 Dec 2024 13:01:55 +0530 Subject: [PATCH 60/73] added health check for rpc and enabled reflection --- .../main/java/com/akto/threat/backend/BackendServer.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java index 294cf5971b..e2bb44b421 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java @@ -7,6 +7,9 @@ import com.mongodb.client.MongoClient; import io.grpc.Server; import io.grpc.ServerBuilder; +import io.grpc.health.v1.HealthCheckResponse.ServingStatus; +import io.grpc.protobuf.services.HealthStatusManager; +import io.grpc.protobuf.services.ProtoReflectionServiceV1; import java.io.IOException; import java.util.concurrent.TimeUnit; @@ -15,13 +18,19 @@ public class BackendServer { private final Server server; public BackendServer(int port, MongoClient mongoClient, KafkaConfig kafkaConfig) { + HealthStatusManager health = new HealthStatusManager(); + this.port = port; this.server = ServerBuilder.forPort(port) + .addService(ProtoReflectionServiceV1.newInstance()) .addService(new MaliciousEventService(kafkaConfig)) .addService(new DashboardService(mongoClient)) + .addService(health.getHealthService()) .intercept(new AuthenticationInterceptor()) .build(); + + health.setStatus("", ServingStatus.SERVING); } public void stop() throws InterruptedException { From e30f66c4919857360ad71b826247cf4013477df6 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 18 Dec 2024 13:39:56 +0530 Subject: [PATCH 61/73] added health service for grpc --- .../akto/threat/backend/BackendServer.java | 22 ++++++++--------- .../threat/backend/service/HealthService.java | 17 +++++++++++++ protobuf/health/service/v1/service.proto | 24 +++++++++++++++++++ 3 files changed, 52 insertions(+), 11 deletions(-) create mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/HealthService.java create mode 100644 protobuf/health/service/v1/service.proto diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java index e2bb44b421..8407bf975d 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java @@ -3,13 +3,12 @@ import com.akto.kafka.KafkaConfig; import com.akto.threat.backend.interceptors.AuthenticationInterceptor; import com.akto.threat.backend.service.DashboardService; +import com.akto.threat.backend.service.HealthService; import com.akto.threat.backend.service.MaliciousEventService; import com.mongodb.client.MongoClient; import io.grpc.Server; import io.grpc.ServerBuilder; -import io.grpc.health.v1.HealthCheckResponse.ServingStatus; -import io.grpc.protobuf.services.HealthStatusManager; -import io.grpc.protobuf.services.ProtoReflectionServiceV1; +import io.grpc.ServerInterceptors; import java.io.IOException; import java.util.concurrent.TimeUnit; @@ -18,19 +17,20 @@ public class BackendServer { private final Server server; public BackendServer(int port, MongoClient mongoClient, KafkaConfig kafkaConfig) { - HealthStatusManager health = new HealthStatusManager(); + AuthenticationInterceptor authenticationInterceptor = new AuthenticationInterceptor(); this.port = port; + this.server = ServerBuilder.forPort(port) - .addService(ProtoReflectionServiceV1.newInstance()) - .addService(new MaliciousEventService(kafkaConfig)) - .addService(new DashboardService(mongoClient)) - .addService(health.getHealthService()) - .intercept(new AuthenticationInterceptor()) + .addService( + ServerInterceptors.intercept( + new MaliciousEventService(kafkaConfig), authenticationInterceptor)) + .addService( + ServerInterceptors.intercept( + new DashboardService(mongoClient), authenticationInterceptor)) + .addService(new HealthService()) .build(); - - health.setStatus("", ServingStatus.SERVING); } public void stop() throws InterruptedException { diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/HealthService.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/HealthService.java new file mode 100644 index 0000000000..5469ed7d16 --- /dev/null +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/HealthService.java @@ -0,0 +1,17 @@ +package com.akto.threat.backend.service; + +import com.akto.proto.generated.health.service.v1.CheckRequest; +import com.akto.proto.generated.health.service.v1.CheckResponse; +import com.akto.proto.generated.health.service.v1.CheckResponse.ServingStatus; +import com.akto.proto.generated.health.service.v1.HealthServiceGrpc; +import io.grpc.stub.StreamObserver; + +public class HealthService extends HealthServiceGrpc.HealthServiceImplBase { + + @Override + public void check(CheckRequest request, StreamObserver responseObserver) { + responseObserver.onNext( + CheckResponse.newBuilder().setStatus(ServingStatus.SERVING_STATUS_SERVING).build()); + responseObserver.onCompleted(); + } +} diff --git a/protobuf/health/service/v1/service.proto b/protobuf/health/service/v1/service.proto new file mode 100644 index 0000000000..755ed7d4c1 --- /dev/null +++ b/protobuf/health/service/v1/service.proto @@ -0,0 +1,24 @@ +syntax = "proto3"; + +package health.service.v1; + +option java_outer_classname = "HealthServiceProto"; +option java_package = "health.service.v1"; + +message CheckRequest { + string service = 1; +} + +message CheckResponse { + enum ServingStatus { + SERVING_STATUS_UNSPECIFIED = 0; + SERVING_STATUS_SERVING = 1; + SERVING_STATUS_NOT_SERVING = 2; + SERVING_STATUS_SERVICE_UNKNOWN = 3; // Used only by the Watch method. + } + ServingStatus status = 1; +} + +service HealthService { + rpc Check(CheckRequest) returns (CheckResponse); +} \ No newline at end of file From 43c39127659a78f4a5b9d14f9e6f69a01fbe1e41 Mon Sep 17 00:00:00 2001 From: Ajinkya <109141486+ag060@users.noreply.github.com> Date: Thu, 19 Dec 2024 14:32:14 +0530 Subject: [PATCH 62/73] using vertx http server as threat detection backend (#1842) --- .../SuspectSampleDataAction.java | 106 +++++++++++------- apps/threat-detection-backend/pom.xml | 18 +-- .../akto/threat/backend/BackendServer.java | 64 ----------- .../akto/threat/backend/BackendVerticle.java | 75 +++++++++++++ .../threat/backend/HealthCheckServer.java | 39 ------- .../java/com/akto/threat/backend/Main.java | 27 +---- .../AuthenticationInterceptor.java | 53 +++++---- .../akto/threat/backend/router/ARouter.java | 9 ++ .../backend/router/DashboardRouter.java | 54 +++++++++ .../backend/router/ThreatDetectionRouter.java | 43 +++++++ .../backend/service/DashboardService.java | 42 +++---- .../service/MaliciousEventService.java | 17 +-- .../backend/tasks/FlushMessagesToDB.java | 3 +- .../akto/threat/backend/utils/KafkaUtils.java | 2 +- .../java/com/akto/threat/detection/Main.java | 4 +- .../WindowBasedThresholdNotifier.java | 2 +- ...java => SendMaliciousEventsToBackend.java} | 68 +++++------ libs/protobuf/pom.xml | 6 + .../akto/proto/utils/ProtoMessageUtils.java | 27 +++++ .../dashboard_service/v1/service.proto | 5 - .../malicious_alert_service/v1/service.proto | 4 - 21 files changed, 365 insertions(+), 303 deletions(-) delete mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java create mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendVerticle.java delete mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java create mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/ARouter.java create mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/DashboardRouter.java create mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/ThreatDetectionRouter.java rename apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/{SendMaliciousRequestsToBackend.java => SendMaliciousEventsToBackend.java} (70%) create mode 100644 libs/protobuf/src/main/java/com/akto/proto/utils/ProtoMessageUtils.java diff --git a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java index b9e1061c0f..26831788b7 100644 --- a/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java +++ b/apps/dashboard/src/main/java/com/akto/action/threat_detection/SuspectSampleDataAction.java @@ -3,19 +3,17 @@ import com.akto.action.UserAction; import com.akto.dto.traffic.SuspectSampleData; import com.akto.dto.type.URLMethods; -import com.akto.grpc.auth.AuthToken; -import com.akto.proto.generated.threat_detection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; -import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.DashboardServiceGrpc; -import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceBlockingStub; -import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersRequest; import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersResponse; -import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.ListMaliciousRequestsRequest; -import io.grpc.Grpc; -import io.grpc.InsecureChannelCredentials; -import io.grpc.ManagedChannel; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.ListMaliciousRequestsResponse; +import com.akto.proto.utils.ProtoMessageUtils; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; public class SuspectSampleDataAction extends UserAction { @@ -30,51 +28,73 @@ public class SuspectSampleDataAction extends UserAction { Map sort; int startTimestamp, endTimestamp; - private final DashboardServiceBlockingStub dsServiceStub; + private final CloseableHttpClient httpClient; + private final String backendUrl; + private final String backendToken; public SuspectSampleDataAction() { super(); - - String target = "localhost:8980"; - ManagedChannel channel = - Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - this.dsServiceStub = - DashboardServiceGrpc.newBlockingStub(channel) - .withCallCredentials( - new AuthToken(System.getenv("AKTO_threat_detection_BACKEND_TOKEN"))); + this.httpClient = HttpClients.createDefault(); + this.backendUrl = System.getenv("THREAT_DETECTION_BACKEND_URL"); + this.backendToken = System.getenv("THREAT_DETECTION_BACKEND_TOKEN"); } public String fetchSampleDataV2() { - List maliciousEvts = - this.dsServiceStub - .listMaliciousRequests( - ListMaliciousRequestsRequest.newBuilder().setPage(0).setLimit(500).build()) - .getMaliciousEventsList(); - - this.maliciousEvents = - maliciousEvts.stream() - .map( - mr -> - new DashboardMaliciousEvent( - mr.getId(), - mr.getActor(), - mr.getFilterId(), - mr.getEndpoint(), - URLMethods.Method.fromString(mr.getMethod()), - mr.getApiCollectionId(), - mr.getIp(), - mr.getCountry(), - mr.getDetectedAt())) - .collect(Collectors.toList()); + HttpGet get = + new HttpGet(String.format("%s/api/dashboard/list_malicious_requests", backendUrl)); + get.addHeader("Authorization", "Bearer " + backendToken); + get.addHeader("Content-Type", "application/json"); + + try (CloseableHttpResponse resp = this.httpClient.execute(get)) { + String responseBody = EntityUtils.toString(resp.getEntity()); + + ProtoMessageUtils.toProtoMessage( + ListMaliciousRequestsResponse.class, responseBody) + .ifPresent( + msg -> { + this.maliciousEvents = + msg.getMaliciousEventsList().stream() + .map( + smr -> + new DashboardMaliciousEvent( + smr.getId(), + smr.getActor(), + smr.getFilterId(), + smr.getEndpoint(), + URLMethods.Method.fromString(smr.getMethod()), + smr.getApiCollectionId(), + smr.getIp(), + smr.getCountry(), + smr.getDetectedAt())) + .collect(Collectors.toList()); + }); + } catch (Exception e) { + e.printStackTrace(); + return ERROR.toUpperCase(); + } return SUCCESS.toUpperCase(); } public String fetchFiltersV2() { - FetchAlertFiltersResponse filters = - this.dsServiceStub.fetchAlertFilters(FetchAlertFiltersRequest.newBuilder().build()); - ips = filters.getActorsList(); - urls = filters.getUrlsList(); + HttpGet get = new HttpGet(String.format("%s/api/dashboard/fetch_filters", backendUrl)); + get.addHeader("Authorization", "Bearer " + backendToken); + get.addHeader("Content-Type", "application/json"); + + try (CloseableHttpResponse resp = this.httpClient.execute(get)) { + String responseBody = EntityUtils.toString(resp.getEntity()); + + ProtoMessageUtils.toProtoMessage( + FetchAlertFiltersResponse.class, responseBody) + .ifPresent( + msg -> { + this.ips = msg.getActorsList(); + this.urls = msg.getUrlsList(); + }); + } catch (Exception e) { + e.printStackTrace(); + return ERROR.toUpperCase(); + } return SUCCESS.toUpperCase(); } diff --git a/apps/threat-detection-backend/pom.xml b/apps/threat-detection-backend/pom.xml index a27231a6a0..28766c73bb 100644 --- a/apps/threat-detection-backend/pom.xml +++ b/apps/threat-detection-backend/pom.xml @@ -1,11 +1,12 @@ + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 9.4.27.v20200227 + 4.5.11 @@ -67,14 +68,15 @@ - org.eclipse.jetty - jetty-server - ${jetty.version} + io.vertx + vertx-web + ${vertex.version} + - org.eclipse.jetty - jetty-servlet - ${jetty.version} + io.vertx + vertx-core + ${vertex.version} diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java deleted file mode 100644 index 8407bf975d..0000000000 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendServer.java +++ /dev/null @@ -1,64 +0,0 @@ -package com.akto.threat.backend; - -import com.akto.kafka.KafkaConfig; -import com.akto.threat.backend.interceptors.AuthenticationInterceptor; -import com.akto.threat.backend.service.DashboardService; -import com.akto.threat.backend.service.HealthService; -import com.akto.threat.backend.service.MaliciousEventService; -import com.mongodb.client.MongoClient; -import io.grpc.Server; -import io.grpc.ServerBuilder; -import io.grpc.ServerInterceptors; -import java.io.IOException; -import java.util.concurrent.TimeUnit; - -public class BackendServer { - private final int port; - private final Server server; - - public BackendServer(int port, MongoClient mongoClient, KafkaConfig kafkaConfig) { - AuthenticationInterceptor authenticationInterceptor = new AuthenticationInterceptor(); - - this.port = port; - - this.server = - ServerBuilder.forPort(port) - .addService( - ServerInterceptors.intercept( - new MaliciousEventService(kafkaConfig), authenticationInterceptor)) - .addService( - ServerInterceptors.intercept( - new DashboardService(mongoClient), authenticationInterceptor)) - .addService(new HealthService()) - .build(); - } - - public void stop() throws InterruptedException { - if (server != null) { - server.shutdown().awaitTermination(30, TimeUnit.SECONDS); - } - } - - public void start() throws IOException { - server.start(); - System.out.println("Server started, listening on " + port); - Runtime.getRuntime() - .addShutdownHook( - new Thread( - () -> { - System.err.println("*** shutting down gRPC server since JVM is shutting down"); - try { - BackendServer.this.stop(); - } catch (InterruptedException e) { - e.printStackTrace(System.err); - } - System.err.println("*** server shut down"); - })); - } - - public void blockUntilShutdown() throws InterruptedException { - if (server != null) { - server.awaitTermination(); - } - } -} diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendVerticle.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendVerticle.java new file mode 100644 index 0000000000..1774fc0a36 --- /dev/null +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/BackendVerticle.java @@ -0,0 +1,75 @@ +package com.akto.threat.backend; + +import com.akto.kafka.KafkaConfig; +import com.akto.threat.backend.interceptors.AuthenticationInterceptor; +import com.akto.threat.backend.router.DashboardRouter; +import com.akto.threat.backend.router.ThreatDetectionRouter; +import com.akto.threat.backend.service.DashboardService; +import com.akto.threat.backend.service.MaliciousEventService; +import com.mongodb.client.MongoClient; +import io.vertx.core.AbstractVerticle; +import io.vertx.core.Vertx; +import io.vertx.ext.web.Router; +import io.vertx.ext.web.handler.BodyHandler; + +public class BackendVerticle extends AbstractVerticle { + + private final MongoClient mongoClient; + private final KafkaConfig kafkaConfig; + + public BackendVerticle(MongoClient mongoClient, KafkaConfig kafkaConfig) { + this.mongoClient = mongoClient; + this.kafkaConfig = kafkaConfig; + } + + @Override + public void start() { + Vertx vertx = Vertx.vertx(); + + // Create the router + Router router = Router.router(vertx); + + Router api = Router.router(vertx); + + api.route().handler(BodyHandler.create()); + api.route().handler(new AuthenticationInterceptor()); + + Router dashboardRouter = new DashboardRouter(new DashboardService(mongoClient)).setup(vertx); + Router threatDetectionRouter = + new ThreatDetectionRouter(new MaliciousEventService(kafkaConfig)).setup(vertx); + + api.route("/dashboard/*").subRouter(dashboardRouter); + api.route("/threat_detection/*").subRouter(threatDetectionRouter); + + router.route("/api/*").subRouter(api); + + // Start the HTTP server + + router.route("/health").handler(ctx -> ctx.response().setStatusCode(200).end("OK")); + + // 404 handler + router + .route() + .handler( + rc -> { + rc.response().setStatusCode(404).end("404 - Not Found: " + rc.request().uri()); + }); + + int port = + Integer.parseInt( + System.getenv().getOrDefault("THREAT_DETECTION_BACKEND_SERVER_PORT", "9090")); + + vertx + .createHttpServer() + .requestHandler(router) + .listen(port) + .onSuccess( + server -> { + System.out.println("HTTP server started on port " + port); + }) + .onFailure( + err -> { + System.err.println("Failed to start HTTP server: " + err.getMessage()); + }); + } +} diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java deleted file mode 100644 index 2a54962869..0000000000 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/HealthCheckServer.java +++ /dev/null @@ -1,39 +0,0 @@ -package com.akto.threat.backend; - -import java.io.IOException; -import javax.servlet.http.HttpServlet; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.servlet.ServletHandler; -import org.eclipse.jetty.servlet.ServletHolder; - -public class HealthCheckServer { - public static final int PORT = 9090; - - public static void startHttpServer() throws Exception { - - Server server = new Server(PORT); // HTTP server port - - // Create and configure the servlet handler - ServletHandler handler = new ServletHandler(); - server.setHandler(handler); - - // Define a simple servlet to handle HTTP requests - handler.addServletWithMapping( - new ServletHolder( - new HttpServlet() { - @Override - protected void doGet(HttpServletRequest req, HttpServletResponse resp) - throws IOException { - resp.getWriter().println("OK"); - } - }), - "/"); - - // Start the HTTP server - server.start(); - System.out.println("HTTP Server started on port " + PORT); - server.join(); - } -} diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java index 3520be924b..3738a4bd82 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/Main.java @@ -14,14 +14,11 @@ import com.mongodb.WriteConcern; import com.mongodb.client.MongoClient; import com.mongodb.client.MongoClients; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import org.bson.codecs.configuration.CodecRegistry; import org.bson.codecs.pojo.PojoCodecProvider; public class Main { public static void main(String[] args) throws Exception { - ExecutorService executor = Executors.newFixedThreadPool(2); DaoInit.init(new ConnectionString(System.getenv("AKTO_MONGO_CONN"))); @@ -55,28 +52,6 @@ public static void main(String[] args) throws Exception { new FlushMessagesToDB(internalKafkaConfig, threatProtectionMongo).run(); - executor.submit( - () -> { - int port = - Integer.parseInt( - System.getenv().getOrDefault("AKTO_THREAT_PROTECTION_BACKEND_PORT", "8980")); - BackendServer server = - new BackendServer(port, threatProtectionMongo, internalKafkaConfig); - try { - server.start(); - server.blockUntilShutdown(); - } catch (Exception e) { - e.printStackTrace(); - } - }); - - executor.submit( - () -> { - try { - HealthCheckServer.startHttpServer(); - } catch (Exception e) { - e.printStackTrace(); - } - }); + new BackendVerticle(threatProtectionMongo, internalKafkaConfig).start(); } } diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java index 95683b1c63..f17f7bdd97 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/interceptors/AuthenticationInterceptor.java @@ -2,11 +2,11 @@ import com.akto.dao.ConfigsDao; import com.akto.dto.Config; -import com.akto.grpc.auth.AuthToken; -import io.grpc.*; import io.jsonwebtoken.Claims; import io.jsonwebtoken.Jws; import io.jsonwebtoken.Jwts; +import io.vertx.core.Handler; +import io.vertx.ext.web.RoutingContext; import java.io.IOException; import java.security.KeyFactory; import java.security.NoSuchAlgorithmException; @@ -15,32 +15,7 @@ import java.security.spec.X509EncodedKeySpec; import java.util.Base64; -public class AuthenticationInterceptor implements ServerInterceptor { - @Override - public ServerCall.Listener interceptCall( - ServerCall call, Metadata metadata, ServerCallHandler next) { - String token = AuthToken.getBearerTokenFromMeta(metadata).orElse(null); - - if (token == null) { - call.close( - Status.UNAUTHENTICATED.withDescription("Authorization token is required"), metadata); - return null; - } - - try { - PublicKey publicKey = getPublicKey(); - Jws claims = - Jwts.parserBuilder().setSigningKey(publicKey).build().parseClaimsJws(token); - int accountId = (int) claims.getBody().get("accountId"); - Context ctx = Context.current().withValue(Constants.ACCOUNT_ID_CONTEXT_KEY, accountId); - return Contexts.interceptCall(ctx, call, metadata, next); - } catch (Exception e) { - e.printStackTrace(); - call.close(Status.UNAUTHENTICATED.withDescription("Invalid token"), metadata); - } - - return null; - } +public class AuthenticationInterceptor implements Handler { private static PublicKey getPublicKey() throws NoSuchAlgorithmException, InvalidKeySpecException, IOException { @@ -69,4 +44,26 @@ private static PublicKey getPublicKey() throw e; } } + + @Override + public void handle(RoutingContext context) { + String token = context.request().getHeader("Authorization"); + if (token == null || !token.startsWith("Bearer ")) { + context.response().setStatusCode(401).end("Missing or Invalid Authorization header"); + return; + } + + token = token.substring(7); + + try { + PublicKey publicKey = getPublicKey(); + Jws claims = + Jwts.parserBuilder().setSigningKey(publicKey).build().parseClaimsJws(token); + int accountId = (int) claims.getBody().get("accountId"); + context.put("accountId", accountId + ""); + context.next(); + } catch (Exception e) { + context.response().setStatusCode(401).end("Missing or Invalid Authorization header"); + } + } } diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/ARouter.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/ARouter.java new file mode 100644 index 0000000000..b4e54e8109 --- /dev/null +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/ARouter.java @@ -0,0 +1,9 @@ +package com.akto.threat.backend.router; + +import io.vertx.core.Vertx; +import io.vertx.ext.web.Router; + +public interface ARouter { + + Router setup(Vertx vertx); +} diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/DashboardRouter.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/DashboardRouter.java new file mode 100644 index 0000000000..3e91777cbf --- /dev/null +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/DashboardRouter.java @@ -0,0 +1,54 @@ +package com.akto.threat.backend.router; + +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersRequest; +import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.ListMaliciousRequestsRequest; +import com.akto.proto.utils.ProtoMessageUtils; +import com.akto.threat.backend.service.DashboardService; +import io.vertx.core.Vertx; +import io.vertx.ext.web.RequestBody; +import io.vertx.ext.web.Router; + +public class DashboardRouter implements ARouter { + + private final DashboardService dsService; + + public DashboardRouter(DashboardService dsService) { + this.dsService = dsService; + } + + @Override + public Router setup(Vertx vertx) { + Router router = Router.router(vertx); + + router + .get("/fetch_filters") + .blockingHandler( + ctx -> { + ProtoMessageUtils.toString( + dsService.fetchAlertFilters( + ctx.get("accountId"), FetchAlertFiltersRequest.newBuilder().build())) + .ifPresent(s -> ctx.response().setStatusCode(200).end(s)); + }); + + router + .post("/list_malicious_requests") + .blockingHandler( + ctx -> { + RequestBody reqBody = ctx.body(); + ListMaliciousRequestsRequest req = + ProtoMessageUtils.toProtoMessage( + ListMaliciousRequestsRequest.class, reqBody.asString()) + .orElse(null); + + if (req == null) { + ctx.response().setStatusCode(400).end("Invalid request"); + return; + } + + ProtoMessageUtils.toString(dsService.listMaliciousRequests(ctx.get("accountId"), req)) + .ifPresent(s -> ctx.response().setStatusCode(200).end(s)); + }); + + return router; + } +} diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/ThreatDetectionRouter.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/ThreatDetectionRouter.java new file mode 100644 index 0000000000..67deec84dc --- /dev/null +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/router/ThreatDetectionRouter.java @@ -0,0 +1,43 @@ +package com.akto.threat.backend.router; + +import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; +import com.akto.proto.utils.ProtoMessageUtils; +import com.akto.threat.backend.service.MaliciousEventService; +import io.vertx.core.Vertx; +import io.vertx.ext.web.RequestBody; +import io.vertx.ext.web.Router; + +public class ThreatDetectionRouter implements ARouter { + + private final MaliciousEventService maliciousEventService; + + public ThreatDetectionRouter(MaliciousEventService maliciousEventService) { + this.maliciousEventService = maliciousEventService; + } + + @Override + public Router setup(Vertx vertx) { + Router router = Router.router(vertx); + + router + .post("/record_malicious_event") + .blockingHandler( + ctx -> { + RequestBody reqBody = ctx.body(); + RecordMaliciousEventRequest req = + ProtoMessageUtils.toProtoMessage( + RecordMaliciousEventRequest.class, reqBody.asString()) + .orElse(null); + + if (req == null) { + ctx.response().setStatusCode(400).end("Invalid request"); + return; + } + + maliciousEventService.recordMaliciousEvent(ctx.get("accountId"), req); + ctx.response().setStatusCode(202).end(); + }); + + return router; + } +} diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java index cd83111b62..835b290238 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/DashboardService.java @@ -1,28 +1,25 @@ package com.akto.threat.backend.service; import com.akto.proto.generated.threat_detection.message.malicious_event.dashboard.v1.DashboardMaliciousEventMessage; -import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.DashboardServiceGrpc.DashboardServiceImplBase; import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersRequest; import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.FetchAlertFiltersResponse; import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.ListMaliciousRequestsRequest; import com.akto.proto.generated.threat_detection.service.dashboard_service.v1.ListMaliciousRequestsResponse; import com.akto.threat.backend.constants.MongoDBCollection; import com.akto.threat.backend.db.MaliciousEventModel; -import com.akto.threat.backend.interceptors.Constants; import com.mongodb.BasicDBObject; import com.mongodb.client.DistinctIterable; import com.mongodb.client.MongoClient; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; import com.mongodb.client.model.Filters; -import io.grpc.stub.StreamObserver; import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Set; import org.bson.conversions.Bson; -public class DashboardService extends DashboardServiceImplBase { +public class DashboardService { private final MongoClient mongoClient; public DashboardService(MongoClient mongoClient) { @@ -40,14 +37,11 @@ private static Set findDistinctFields( return result; } - @Override - public void fetchAlertFilters( - FetchAlertFiltersRequest request, - StreamObserver responseObserver) { - int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); + public FetchAlertFiltersResponse fetchAlertFilters( + String accountId, FetchAlertFiltersRequest request) { MongoCollection coll = this.mongoClient - .getDatabase(accountId + "") + .getDatabase(accountId) .getCollection("malicious_events", MaliciousEventModel.class); Set actors = @@ -56,25 +50,18 @@ public void fetchAlertFilters( DashboardService.findDistinctFields( coll, "latestApiEndpoint", String.class, Filters.empty()); - FetchAlertFiltersResponse response = - FetchAlertFiltersResponse.newBuilder().addAllActors(actors).addAllUrls(urls).build(); - responseObserver.onNext(response); - responseObserver.onCompleted(); + return FetchAlertFiltersResponse.newBuilder().addAllActors(actors).addAllUrls(urls).build(); } - @Override - public void listMaliciousRequests( - ListMaliciousRequestsRequest request, - StreamObserver responseObserver) { - int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); - + public ListMaliciousRequestsResponse listMaliciousRequests( + String accountId, ListMaliciousRequestsRequest request) { int page = request.hasPage() && request.getPage() > 0 ? request.getPage() : 1; int limit = request.getLimit(); int skip = (page - 1) * limit; MongoCollection coll = this.mongoClient - .getDatabase(accountId + "") + .getDatabase(accountId) .getCollection( MongoDBCollection.ThreatDetection.MALICIOUS_EVENTS, MaliciousEventModel.class); @@ -102,14 +89,11 @@ public void listMaliciousRequests( .setDetectedAt(evt.getDetectedAt()) .build()); } - ListMaliciousRequestsResponse response = - ListMaliciousRequestsResponse.newBuilder() - .setPage(page) - .setTotal(maliciousEvents.size()) - .addAllMaliciousEvents(maliciousEvents) - .build(); - responseObserver.onNext(response); - responseObserver.onCompleted(); + return ListMaliciousRequestsResponse.newBuilder() + .setPage(page) + .setTotal(maliciousEvents.size()) + .addAllMaliciousEvents(maliciousEvents) + .build(); } } } diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java index fe3f0db472..e81732aa79 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/MaliciousEventService.java @@ -6,20 +6,16 @@ import com.akto.proto.generated.threat_detection.message.malicious_event.event_type.v1.EventType; import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventMessage; import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; -import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; -import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; import com.akto.threat.backend.constants.KafkaTopic; import com.akto.threat.backend.constants.MongoDBCollection; import com.akto.threat.backend.db.AggregateSampleMaliciousEventModel; import com.akto.threat.backend.db.MaliciousEventModel; -import com.akto.threat.backend.interceptors.Constants; import com.akto.threat.backend.utils.KafkaUtils; -import io.grpc.stub.StreamObserver; import java.util.ArrayList; import java.util.List; -public class MaliciousEventService extends MaliciousEventServiceGrpc.MaliciousEventServiceImplBase { +public class MaliciousEventService { private final Kafka kafka; @@ -27,15 +23,13 @@ public MaliciousEventService(KafkaConfig kafkaConfig) { this.kafka = new Kafka(kafkaConfig); } - @Override - public void recordMaliciousEvent( - RecordMaliciousEventRequest request, - StreamObserver responseObserver) { + public void recordMaliciousEvent(String accountId, RecordMaliciousEventRequest request) { + + System.out.println("Received malicious event: " + request); MaliciousEventMessage evt = request.getMaliciousEvent(); String actor = evt.getActor(); String filterId = evt.getFilterId(); - int accountId = Constants.ACCOUNT_ID_CONTEXT_KEY.get(); EventType eventType = evt.getEventType(); @@ -86,8 +80,5 @@ public void recordMaliciousEvent( KafkaUtils.generateMsg( maliciousEventModel, MongoDBCollection.ThreatDetection.MALICIOUS_EVENTS, accountId), KafkaTopic.ThreatDetection.INTERNAL_DB_MESSAGES); - - responseObserver.onNext(RecordMaliciousEventResponse.newBuilder().build()); - responseObserver.onCompleted(); } } diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java index 78ea5095ab..543793b3ac 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/tasks/FlushMessagesToDB.java @@ -86,8 +86,7 @@ private void writeMessage(String message) throws JsonProcessingException { Map json = gson.fromJson(message, Map.class); String eventType = (String) json.get("eventType"); String payload = (String) json.get("payload"); - Double accIdDouble = (Double) json.get("accountId"); - int accountId = accIdDouble.intValue(); + String accountId = (String) json.get("accountId"); switch (eventType) { case MongoDBCollection.ThreatDetection.AGGREGATE_SAMPLE_MALICIOUS_REQUESTS: diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/utils/KafkaUtils.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/utils/KafkaUtils.java index e6c6007b70..afaefdd802 100644 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/utils/KafkaUtils.java +++ b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/utils/KafkaUtils.java @@ -7,7 +7,7 @@ public class KafkaUtils { private static final Gson gson = new Gson(); - public static String generateMsg(Object writes, String eventType, int accountId) { + public static String generateMsg(Object writes, String eventType, String accountId) { BasicDBObject obj = new BasicDBObject(); obj.put("eventType", eventType); String payloadStr = gson.toJson(writes); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java index 65a5cdb4b5..1d1b061d83 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/Main.java @@ -9,7 +9,7 @@ import com.akto.threat.detection.tasks.CleanupTask; import com.akto.threat.detection.tasks.FlushSampleDataTask; import com.akto.threat.detection.tasks.MaliciousTrafficDetectorTask; -import com.akto.threat.detection.tasks.SendMaliciousRequestsToBackend; +import com.akto.threat.detection.tasks.SendMaliciousEventsToBackend; import com.mongodb.ConnectionString; import io.lettuce.core.RedisClient; import org.flywaydb.core.Flyway; @@ -56,7 +56,7 @@ public static void main(String[] args) throws Exception { new FlushSampleDataTask( sessionFactory, internalKafka, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS) .run(); - new SendMaliciousRequestsToBackend( + new SendMaliciousEventsToBackend( sessionFactory, internalKafka, KafkaTopic.ThreatDetection.ALERTS) .run(); new CleanupTask(sessionFactory).run(); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java index f9e9b9fbfb..20b749326b 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java @@ -85,7 +85,7 @@ public Result shouldNotify(String aggKey, SampleMaliciousRequest maliciousEvent, boolean cooldownBreached = (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); - if (thresholdBreached && cooldownBreached) { + if (thresholdBreached) { this.notifiedMap.put(aggKey, now); return new Result(true); } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java similarity index 70% rename from apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java rename to apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java index cdfa6a378a..8150b71903 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousRequestsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java @@ -1,24 +1,24 @@ package com.akto.threat.detection.tasks; -import com.akto.grpc.auth.AuthToken; import com.akto.kafka.KafkaConfig; import com.akto.proto.generated.threat_detection.message.malicious_event.event_type.v1.EventType; import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventMessage; import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; -import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.MaliciousEventServiceGrpc; import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; -import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventResponse; +import com.akto.proto.utils.ProtoMessageUtils; import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; -import io.grpc.Grpc; -import io.grpc.InsecureChannelCredentials; -import io.grpc.ManagedChannel; -import io.grpc.stub.StreamObserver; +import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.stream.Collectors; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.hibernate.Session; import org.hibernate.SessionFactory; @@ -27,24 +27,16 @@ /* This will send alerts to threat detection backend */ -public class SendMaliciousRequestsToBackend extends AbstractKafkaConsumerTask { +public class SendMaliciousEventsToBackend extends AbstractKafkaConsumerTask { private final SessionFactory sessionFactory; + private final CloseableHttpClient httpClient; - private final MaliciousEventServiceGrpc.MaliciousEventServiceStub consumerServiceStub; - - public SendMaliciousRequestsToBackend( + public SendMaliciousEventsToBackend( SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { super(trafficConfig, topic); this.sessionFactory = sessionFactory; - - String target = "localhost:8980"; - ManagedChannel channel = - Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()).build(); - this.consumerServiceStub = - MaliciousEventServiceGrpc.newStub(channel) - .withCallCredentials( - new AuthToken(System.getenv("AKTO_threat_detection_BACKEND_TOKEN"))); + this.httpClient = HttpClients.createDefault(); } private List getSampleMaliciousRequests(String actor, String filterId) { @@ -113,25 +105,25 @@ protected void processRecords(ConsumerRecords records) { .collect(Collectors.toList())); } - this.consumerServiceStub.recordMaliciousEvent( - reqBuilder.build(), - new StreamObserver() { - @Override - public void onNext(RecordMaliciousEventResponse value) { - // Do nothing - } - - @Override - public void onError(Throwable t) { - t.printStackTrace(); - } - - @Override - public void onCompleted() { - // Do nothing - System.out.println("Completed"); - } - }); + String url = "http://localhost:9090"; + String token = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"); + ProtoMessageUtils.toString(reqBuilder.build()) + .ifPresent( + msg -> { + StringEntity requestEntity = + new StringEntity(msg, ContentType.APPLICATION_JSON); + HttpPost req = + new HttpPost( + String.format("%s/api/threat_detection/record_malicious_event", url)); + req.addHeader("Authorization", "Bearer " + token); + req.setEntity(requestEntity); + try { + System.out.println("Sending request to backend: " + msg); + this.httpClient.execute(req); + } catch (IOException e) { + e.printStackTrace(); + } + }); } catch (Exception e) { e.printStackTrace(); } diff --git a/libs/protobuf/pom.xml b/libs/protobuf/pom.xml index b60e78371d..6943beca2a 100644 --- a/libs/protobuf/pom.xml +++ b/libs/protobuf/pom.xml @@ -50,6 +50,12 @@ protobuf-java 4.28.3 + + com.google.protobuf + protobuf-java-util + 3.25.5 + compile + diff --git a/libs/protobuf/src/main/java/com/akto/proto/utils/ProtoMessageUtils.java b/libs/protobuf/src/main/java/com/akto/proto/utils/ProtoMessageUtils.java new file mode 100644 index 0000000000..b6d4973e86 --- /dev/null +++ b/libs/protobuf/src/main/java/com/akto/proto/utils/ProtoMessageUtils.java @@ -0,0 +1,27 @@ +package com.akto.proto.utils; + +import com.google.protobuf.Message; +import com.google.protobuf.util.JsonFormat; +import java.util.Optional; + +public class ProtoMessageUtils { + public static Optional toString(Message msg) { + try { + return Optional.of(JsonFormat.printer().print(msg)); + } catch (Exception e) { + // Ignore + } + return Optional.empty(); + } + + public static Optional toProtoMessage(Class clz, String msg) { + try { + T.Builder builder = (T.Builder) clz.getMethod("newBuilder").invoke(null); + JsonFormat.parser().merge(msg, builder); + return Optional.of((T) builder.build()); + } catch (Exception e) { + // Ignore + } + return Optional.empty(); + } +} diff --git a/protobuf/threat_detection/service/dashboard_service/v1/service.proto b/protobuf/threat_detection/service/dashboard_service/v1/service.proto index 56320d0408..9d5c35bfc3 100644 --- a/protobuf/threat_detection/service/dashboard_service/v1/service.proto +++ b/protobuf/threat_detection/service/dashboard_service/v1/service.proto @@ -26,8 +26,3 @@ message FetchAlertFiltersResponse { repeated string actors = 1; repeated string urls = 2; } - -service DashboardService { - rpc ListMaliciousRequests(ListMaliciousRequestsRequest) returns (ListMaliciousRequestsResponse); - rpc FetchAlertFilters(FetchAlertFiltersRequest) returns (FetchAlertFiltersResponse) {} -} \ No newline at end of file diff --git a/protobuf/threat_detection/service/malicious_alert_service/v1/service.proto b/protobuf/threat_detection/service/malicious_alert_service/v1/service.proto index 16238d2d7a..bde487cfe1 100644 --- a/protobuf/threat_detection/service/malicious_alert_service/v1/service.proto +++ b/protobuf/threat_detection/service/malicious_alert_service/v1/service.proto @@ -17,7 +17,3 @@ message RecordMaliciousEventRequest { threat_detection.message.malicious_event.v1.MaliciousEventMessage malicious_event = 1; repeated threat_detection.message.sample_request.v1.SampleMaliciousRequest sample_requests = 2; } - -service MaliciousEventService { - rpc RecordMaliciousEvent(RecordMaliciousEventRequest) returns (RecordMaliciousEventResponse) {} -} \ No newline at end of file From 926488108db9f8de9bd92f0feab3023c757b003f Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Fri, 20 Dec 2024 14:13:11 +0530 Subject: [PATCH 63/73] reading akto threat detection backend url from env --- .../threat/detection/tasks/SendMaliciousEventsToBackend.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java index 8150b71903..7ba785e154 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java @@ -105,7 +105,7 @@ protected void processRecords(ConsumerRecords records) { .collect(Collectors.toList())); } - String url = "http://localhost:9090"; + String url = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); String token = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"); ProtoMessageUtils.toString(reqBuilder.build()) .ifPresent( From b46e0670a8d8dc83d0f6eda69a796ba194857058 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Sat, 21 Dec 2024 12:59:00 +0530 Subject: [PATCH 64/73] added try catch --- .../tasks/MaliciousTrafficDetectorTask.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 6a185cda84..9a9dbbf328 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -93,13 +93,18 @@ public void run() { ConsumerRecords records = kafkaConsumer.poll( Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); - for (ConsumerRecord record : records) { - processRecord(record); - } - if (!records.isEmpty()) { - // Should we commit even if there are no records ? - kafkaConsumer.commitSync(); + try { + for (ConsumerRecord record : records) { + processRecord(record); + } + + if (!records.isEmpty()) { + // Should we commit even if there are no records ? + kafkaConsumer.commitSync(); + } + } catch (Exception e) { + e.printStackTrace(); } } }); @@ -151,6 +156,7 @@ private boolean validateFilterForRequest( } private void processRecord(ConsumerRecord record) { + System.out.println("Kafka record: " + record.value()); HttpResponseParams responseParam = HttpCallParser.parseKafkaMessage(record.value()); Context.accountId.set(Integer.parseInt(responseParam.getAccountId())); Map filters = this.getFilters(); From b5e96752c37e4b35463e38b662a4c9b0e0fc9036 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Sat, 21 Dec 2024 13:59:18 +0530 Subject: [PATCH 65/73] fixed sample not being to backend --- .../threat/detection/dto/MessageEnvelope.java | 21 +++++++++++++++---- .../detection/tasks/FlushSampleDataTask.java | 2 +- .../tasks/MaliciousTrafficDetectorTask.java | 13 +++++++++--- 3 files changed, 28 insertions(+), 8 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java index 2ac2535d9b..06fe644be4 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/dto/MessageEnvelope.java @@ -4,20 +4,21 @@ import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.Message; import com.google.protobuf.util.JsonFormat; - import java.util.Optional; // Kafka Message Wrapper for suspect data public class MessageEnvelope { private String accountId; private String data; + private String actor; private static final ObjectMapper objectMapper = new ObjectMapper(); public MessageEnvelope() {} - public MessageEnvelope(String accountId, String data) { + public MessageEnvelope(String accountId, String actor, String data) { this.accountId = accountId; + this.actor = actor; this.data = data; } @@ -57,9 +58,21 @@ public static Optional unmarshal(String message) { return Optional.empty(); } - public static MessageEnvelope generateEnvelope(String accountId, Message msg) + public static MessageEnvelope generateEnvelope(String accountId, String actor, Message msg) throws InvalidProtocolBufferException { String data = JsonFormat.printer().print(msg); - return new MessageEnvelope(accountId, data); + return new MessageEnvelope(accountId, actor, data); + } + + public String getActor() { + return actor; + } + + public void setActor(String actor) { + this.actor = actor; + } + + public static ObjectMapper getObjectmapper() { + return objectMapper; } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 8e82b688c1..6829d6ecad 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -49,7 +49,7 @@ protected void processRecords(ConsumerRecords records) { events.add( MaliciousEventEntity.newBuilder() - .setActor(m.getAccountId()) + .setActor(m.getActor()) .setFilterId(evt.getFilterId()) .setUrl(evt.getUrl()) .setMethod(URLMethods.Method.fromString(evt.getMethod())) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 9a9dbbf328..07dd61d82c 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -161,11 +161,14 @@ private void processRecord(ConsumerRecord record) { Context.accountId.set(Integer.parseInt(responseParam.getAccountId())); Map filters = this.getFilters(); if (filters.isEmpty()) { + System.out.println("No filters found"); return; } List maliciousMessages = new ArrayList<>(); + System.out.println("Total number of filters: " + filters.size()); + for (FilterConfig apiFilter : apiFilters.values()) { boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); @@ -177,9 +180,11 @@ private void processRecord(ConsumerRecord record) { // But regardless of whether request falls in aggregation or not, // we still push malicious requests to kafka + System.out.println("Filter passed: " + apiFilter.getId()); + // todo: modify fetch yaml and read aggregate rules from it List rules = new ArrayList<>(); - rules.add(new Rule("Lfi Rule 1", new Condition(100, 10))); + rules.add(new Rule("Lfi Rule 1", new Condition(10, 10))); AggregationRules aggRules = new AggregationRules(); aggRules.setRule(rules); @@ -206,7 +211,7 @@ private void processRecord(ConsumerRecord record) { try { maliciousMessages.add( MessageEnvelope.generateEnvelope( - responseParam.getAccountId(), maliciousReq)); + responseParam.getAccountId(), actor, maliciousReq)); } catch (InvalidProtocolBufferException e) { return; } @@ -223,6 +228,7 @@ private void processRecord(ConsumerRecord record) { this.windowBasedThresholdNotifier.shouldNotify(aggKey, maliciousReq, rule); if (result.shouldNotify()) { + System.out.print("Notifying for aggregation rule: " + rule); generateAndPushMaliciousEventRequest( apiFilter, actor, @@ -271,7 +277,8 @@ private void generateAndPushMaliciousEventRequest( .setDetectedAt(responseParam.getTime()) .build(); try { - MessageEnvelope.generateEnvelope(responseParam.getAccountId(), maliciousEvent) + System.out.println("Pushing malicious event to kafka: " + maliciousEvent); + MessageEnvelope.generateEnvelope(responseParam.getAccountId(), actor, maliciousEvent) .marshal() .ifPresent( data -> { From e273f0b2ce20e5fdc1435783871123ca38634581 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Mon, 23 Dec 2024 11:33:04 +0530 Subject: [PATCH 66/73] clearing cache as soon the alert is raised --- .../threat/detection/cache/CounterCache.java | 2 ++ .../cache/RedisBackedCounterCache.java | 23 +++++++++++-------- .../WindowBasedThresholdNotifier.java | 21 ++--------------- 3 files changed, 18 insertions(+), 28 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/CounterCache.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/CounterCache.java index 49a59419e8..f1eaeffa90 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/CounterCache.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/CounterCache.java @@ -9,4 +9,6 @@ public interface CounterCache { long get(String key); boolean exists(String key); + + void clear(String key); } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java index 55ac19424e..f2c30de012 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java @@ -1,13 +1,12 @@ package com.akto.threat.detection.cache; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; import io.lettuce.core.ExpireArgs; import io.lettuce.core.RedisClient; import io.lettuce.core.api.StatefulRedisConnection; - import java.util.Optional; import java.util.concurrent.*; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; public class RedisBackedCounterCache implements CounterCache { @@ -48,8 +47,8 @@ public RedisBackedCounterCache(RedisClient redisClient, String prefix) { this.pendingOps = new ConcurrentLinkedQueue<>(); } - private String getKey(String key) { - return prefix + "|" + key; + private String addPrefixToKey(String key) { + return new StringBuilder().append(prefix).append("|").append(key).toString(); } @Override @@ -59,7 +58,7 @@ public void increment(String key) { @Override public void incrementBy(String key, long val) { - String _key = getKey(key); + String _key = addPrefixToKey(key); localCache.asMap().merge(_key, val, Long::sum); pendingOps.add(new Op(_key, val)); @@ -68,19 +67,25 @@ public void incrementBy(String key, long val) { @Override public long get(String key) { - return Optional.ofNullable(this.localCache.getIfPresent(getKey(key))).orElse(0L); + return Optional.ofNullable(this.localCache.getIfPresent(addPrefixToKey(key))).orElse(0L); } @Override public boolean exists(String key) { - return localCache.asMap().containsKey(getKey(key)); + return localCache.asMap().containsKey(addPrefixToKey(key)); + } + + @Override + public void clear(String key) { + localCache.invalidate(addPrefixToKey(key)); + redis.async().del(addPrefixToKey(key)); } private void setExpiryIfNotSet(String key, long seconds) { // We only set expiry for redis entry. For local cache we have lower expiry for // all entries. ExpireArgs args = ExpireArgs.Builder.nx(); - redis.async().expire(getKey(key), seconds, args); + redis.async().expire(addPrefixToKey(key), seconds, args); } private void syncToRedis() { diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java index 20b749326b..01d3b531a7 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/smart_event_detector/window_based/WindowBasedThresholdNotifier.java @@ -5,20 +5,11 @@ import com.akto.threat.detection.cache.CounterCache; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; public class WindowBasedThresholdNotifier { private final Config config; - // We can use an in-memory cache for this, since we dont mind being notified - // more than once by multiple instances of the service. - // But on 1 instance, we should not notify more than once in the cooldown - // period. - // TODO: Move this to redis - private final ConcurrentMap notifiedMap; - public static class Config { private final int threshold; private final int windowSizeInMinutes; @@ -63,7 +54,6 @@ public Config getConfig() { public WindowBasedThresholdNotifier(CounterCache cache, Config config) { this.cache = cache; this.config = config; - this.notifiedMap = new ConcurrentHashMap<>(); } public Result shouldNotify(String aggKey, SampleMaliciousRequest maliciousEvent, Rule rule) { @@ -79,18 +69,11 @@ public Result shouldNotify(String aggKey, SampleMaliciousRequest maliciousEvent, boolean thresholdBreached = windowCount >= rule.getCondition().getMatchCount(); - long now = System.currentTimeMillis() / 1000L; - long lastNotified = this.notifiedMap.getOrDefault(aggKey, 0L); - - boolean cooldownBreached = - (now - lastNotified) >= this.config.getNotificationCooldownInSeconds(); - if (thresholdBreached) { - this.notifiedMap.put(aggKey, now); - return new Result(true); + this.cache.clear(cacheKey); } - return new Result(false); + return new Result(thresholdBreached); } public List getBins(String aggKey, int binStart, int binEnd) { From f640fdd4069e98a5ea90e9419d9fca8cb045b171 Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Mon, 23 Dec 2024 13:17:48 +0530 Subject: [PATCH 67/73] sending sample malicious events to backend only once --- .../cache/RedisBackedCounterCache.java | 25 ++++++++---- .../db/entity/MaliciousEventEntity.java | 10 +++-- .../tasks/AbstractKafkaConsumerTask.java | 10 +++-- .../tasks/SendMaliciousEventsToBackend.java | 38 ++++++++++++++++--- .../V3__add_alert_to_backend_column.sql | 4 ++ 5 files changed, 69 insertions(+), 18 deletions(-) create mode 100644 apps/threat-detection/src/main/resources/db/migration/V3__add_alert_to_backend_column.sql diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java index f2c30de012..42de770b62 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/cache/RedisBackedCounterCache.java @@ -32,7 +32,8 @@ public long getValue() { private final Cache localCache; - private final ConcurrentLinkedQueue pendingOps; + private final ConcurrentLinkedQueue pendingIncOps; + private final ConcurrentMap deletedKeys; private final String prefix; public RedisBackedCounterCache(RedisClient redisClient, String prefix) { @@ -44,7 +45,8 @@ public RedisBackedCounterCache(RedisClient redisClient, String prefix) { ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); executor.scheduleAtFixedRate(this::syncToRedis, 60, 5, TimeUnit.SECONDS); - this.pendingOps = new ConcurrentLinkedQueue<>(); + this.pendingIncOps = new ConcurrentLinkedQueue<>(); + this.deletedKeys = new ConcurrentHashMap<>(); } private String addPrefixToKey(String key) { @@ -60,7 +62,7 @@ public void increment(String key) { public void incrementBy(String key, long val) { String _key = addPrefixToKey(key); localCache.asMap().merge(_key, val, Long::sum); - pendingOps.add(new Op(_key, val)); + pendingIncOps.add(new Op(_key, val)); this.setExpiryIfNotSet(_key, 3 * 60 * 60); // added 3 hours expiry for now } @@ -77,8 +79,10 @@ public boolean exists(String key) { @Override public void clear(String key) { - localCache.invalidate(addPrefixToKey(key)); - redis.async().del(addPrefixToKey(key)); + String _key = addPrefixToKey(key); + localCache.invalidate(_key); + this.deletedKeys.put(_key, true); + redis.async().del(_key); } private void setExpiryIfNotSet(String key, long seconds) { @@ -89,10 +93,15 @@ private void setExpiryIfNotSet(String key, long seconds) { } private void syncToRedis() { - while (!pendingOps.isEmpty()) { - Op op = pendingOps.poll(); + while (!pendingIncOps.isEmpty()) { + Op op = pendingIncOps.poll(); String key = op.getKey(); long val = op.getValue(); + + if (this.deletedKeys.containsKey(key)) { + continue; + } + redis .async() .incrby(key, val) @@ -107,5 +116,7 @@ private void syncToRedis() { } }); } + + this.deletedKeys.clear(); } } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java index 6ec18ea559..41bf2422ee 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/db/entity/MaliciousEventEntity.java @@ -1,11 +1,9 @@ package com.akto.threat.detection.db.entity; import com.akto.dto.type.URLMethods; - import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.UUID; - import javax.persistence.Column; import javax.persistence.Entity; import javax.persistence.EnumType; @@ -14,7 +12,6 @@ import javax.persistence.Id; import javax.persistence.PrePersist; import javax.persistence.Table; - import org.hibernate.annotations.GenericGenerator; @Entity @@ -55,6 +52,9 @@ public class MaliciousEventEntity { @Column(name = "created_at", updatable = false) private LocalDateTime createdAt; + @Column(name = "_alerted_to_backend") + private boolean alertedToBackend; + public MaliciousEventEntity() {} @PrePersist @@ -172,6 +172,10 @@ public LocalDateTime getCreatedAt() { return createdAt; } + public boolean isAlertedToBackend() { + return alertedToBackend; + } + @Override public String toString() { return "MaliciousEventEntity{" diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java index c1eb835da1..cfd9ccecd6 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java @@ -47,10 +47,14 @@ public void run() { continue; } - processRecords(records); + try { + processRecords(records); - if (!records.isEmpty()) { - kafkaConsumer.commitSync(); + if (!records.isEmpty()) { + kafkaConsumer.commitSync(); + } + } catch (Exception ex) { + ex.printStackTrace(); } } }); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java index 7ba785e154..c2a9e7cf54 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java @@ -13,6 +13,7 @@ import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.UUID; import java.util.stream.Collectors; import org.apache.http.client.methods.HttpPost; import org.apache.http.entity.ContentType; @@ -39,14 +40,32 @@ public SendMaliciousEventsToBackend( this.httpClient = HttpClients.createDefault(); } + private void markSampleDataAsSent(List ids) { + Session session = this.sessionFactory.openSession(); + Transaction txn = session.beginTransaction(); + try { + session + .createQuery( + "update MaliciousEventEntity m set m.alertedToBackend = true where m.id in :ids") + .setParameterList("ids", ids) + .executeUpdate(); + } catch (Exception ex) { + ex.printStackTrace(); + txn.rollback(); + } finally { + txn.commit(); + session.close(); + } + } + private List getSampleMaliciousRequests(String actor, String filterId) { Session session = this.sessionFactory.openSession(); Transaction txn = session.beginTransaction(); try { return session .createQuery( - "from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId order" - + " by m.createdAt desc", + "from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId and" + + " m.alertedToBackend = false order by m.createdAt desc", MaliciousEventEntity.class) .setParameter("actor", actor) .setParameter("filterId", filterId) @@ -83,12 +102,13 @@ protected void processRecords(ConsumerRecords records) { MaliciousEventMessage evt = builder.build(); // Get sample data from postgres for this alert + List sampleData = + this.getSampleMaliciousRequests(evt.getActor(), evt.getFilterId()); try { RecordMaliciousEventRequest.Builder reqBuilder = RecordMaliciousEventRequest.newBuilder().setMaliciousEvent(evt); if (EventType.EVENT_TYPE_AGGREGATED.equals(evt.getEventType())) { - List sampleData = - this.getSampleMaliciousRequests(evt.getActor(), evt.getFilterId()); + sampleData = this.getSampleMaliciousRequests(evt.getActor(), evt.getFilterId()); reqBuilder.addAllSampleRequests( sampleData.stream() @@ -105,9 +125,13 @@ protected void processRecords(ConsumerRecords records) { .collect(Collectors.toList())); } + List sampleIds = + sampleData.stream().map(MaliciousEventEntity::getId).collect(Collectors.toList()); + + RecordMaliciousEventRequest maliciousEventRequest = reqBuilder.build(); String url = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); String token = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"); - ProtoMessageUtils.toString(reqBuilder.build()) + ProtoMessageUtils.toString(maliciousEventRequest) .ifPresent( msg -> { StringEntity requestEntity = @@ -123,6 +147,10 @@ protected void processRecords(ConsumerRecords records) { } catch (IOException e) { e.printStackTrace(); } + + if (!sampleIds.isEmpty()) { + markSampleDataAsSent(sampleIds); + } }); } catch (Exception e) { e.printStackTrace(); diff --git a/apps/threat-detection/src/main/resources/db/migration/V3__add_alert_to_backend_column.sql b/apps/threat-detection/src/main/resources/db/migration/V3__add_alert_to_backend_column.sql new file mode 100644 index 0000000000..756815e5b0 --- /dev/null +++ b/apps/threat-detection/src/main/resources/db/migration/V3__add_alert_to_backend_column.sql @@ -0,0 +1,4 @@ +alter table threat_detection.malicious_event add column _alerted_to_backend boolean default false; + +-- set all existing rows to false +update threat_detection.malicious_event set _alerted_to_backend = false; From d1b2e76eea770d5c264cff60ea416ab913e2b227 Mon Sep 17 00:00:00 2001 From: ayushaga14 Date: Mon, 30 Dec 2024 00:14:38 +0530 Subject: [PATCH 68/73] use fastjson lib and increase template fetch interval --- .../tasks/MaliciousTrafficDetectorTask.java | 5 +++-- .../java/com/akto/dto/OriginalHttpRequest.java | 18 ++++++++++++++++++ .../com/akto/dto/OriginalHttpResponse.java | 9 +++++++++ .../dao/src/main/java/com/akto/dto/RawApi.java | 13 +++++++++++++ 4 files changed, 43 insertions(+), 2 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 07dd61d82c..d33401ec1e 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -52,6 +52,7 @@ public class MaliciousTrafficDetectorTask implements Task { private Map apiFilters; private int filterLastUpdatedAt = 0; + private int filterUpdateIntervalSec = 300; private final Kafka internalKafka; @@ -112,7 +113,7 @@ public void run() { private Map getFilters() { int now = (int) (System.currentTimeMillis() / 1000); - if (now - filterLastUpdatedAt < 60) { + if (now - filterLastUpdatedAt < filterUpdateIntervalSec) { return apiFilters; } @@ -126,7 +127,7 @@ private boolean validateFilterForRequest( HttpResponseParams responseParam, FilterConfig apiFilter) { try { String message = responseParam.getOrig(); - RawApi rawApi = RawApi.buildFromMessage(message); + RawApi rawApi = RawApi.buildFromMessageNew(message); int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); responseParam.requestParams.setApiCollectionId(apiCollectionId); String url = responseParam.getRequestParams().getURL(); diff --git a/libs/dao/src/main/java/com/akto/dto/OriginalHttpRequest.java b/libs/dao/src/main/java/com/akto/dto/OriginalHttpRequest.java index f6c85ebc52..abdb32e1af 100644 --- a/libs/dao/src/main/java/com/akto/dto/OriginalHttpRequest.java +++ b/libs/dao/src/main/java/com/akto/dto/OriginalHttpRequest.java @@ -83,6 +83,24 @@ public void buildFromSampleMessage(String message) { this.headers = buildHeadersMap(json, "requestHeaders"); } + public void buildFromSampleMessageNew(JSONObject json) { + String rawUrl = (String) json.get("path"); + String[] rawUrlArr = rawUrl.split("\\?"); + this.url = rawUrlArr[0]; + if (rawUrlArr.length > 1) { + this.queryParams = rawUrlArr[1]; + } + + this.type = (String) json.get("type"); + + this.method = (String) json.get("method"); + + String requestPayload = (String) json.get("requestPayload"); + this.body = requestPayload.trim(); + + this.headers = buildHeadersMap(json, "requestHeaders"); + } + public String getJsonRequestBody() { return HttpRequestResponseUtils.rawToJsonString(this.body, this.headers); } diff --git a/libs/dao/src/main/java/com/akto/dto/OriginalHttpResponse.java b/libs/dao/src/main/java/com/akto/dto/OriginalHttpResponse.java index 3964e35eb0..6e464605fd 100644 --- a/libs/dao/src/main/java/com/akto/dto/OriginalHttpResponse.java +++ b/libs/dao/src/main/java/com/akto/dto/OriginalHttpResponse.java @@ -1,6 +1,8 @@ package com.akto.dto; import com.akto.util.HttpRequestResponseUtils; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; import com.google.gson.Gson; import org.apache.commons.lang3.math.NumberUtils; @@ -39,6 +41,13 @@ public void buildFromSampleMessage(String message) { this.statusCode = Integer.parseInt(json.get("statusCode").toString()); } + public void buildFromSampleMessageNew(JSONObject json) { + String responsePayload = (String) json.get("responsePayload"); + this.body = responsePayload != null ? responsePayload.trim() : null; + this.headers = OriginalHttpRequest.buildHeadersMap(json, "responseHeaders"); + this.statusCode = Integer.parseInt(json.get("statusCode").toString()); + } + public void addHeaderFromLine(String line) { if (this.headers == null || this.headers.isEmpty()) { this.headers = new HashMap<>(); diff --git a/libs/dao/src/main/java/com/akto/dto/RawApi.java b/libs/dao/src/main/java/com/akto/dto/RawApi.java index bbc8b9488c..748bbd2a9e 100644 --- a/libs/dao/src/main/java/com/akto/dto/RawApi.java +++ b/libs/dao/src/main/java/com/akto/dto/RawApi.java @@ -7,6 +7,8 @@ import java.util.Map; import com.akto.dto.type.RequestTemplate; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; import com.mongodb.BasicDBList; import com.fasterxml.jackson.databind.ObjectMapper; import com.mongodb.BasicDBObject; @@ -55,6 +57,17 @@ public static RawApi buildFromMessage(String message) { return new RawApi(request, response, message); } + public static RawApi buildFromMessageNew(String message) { + JSONObject json = JSON.parseObject(message); + OriginalHttpRequest request = new OriginalHttpRequest(); + request.buildFromSampleMessageNew(json); + + OriginalHttpResponse response = new OriginalHttpResponse(); + response.buildFromSampleMessageNew(json); + + return new RawApi(request, response, message); + } + public BasicDBObject fetchReqPayload() { OriginalHttpRequest req = this.getRequest(); String reqBody = req.getBody(); From afe3a65648e68970bea00f0b7b9bd61ba6d71b17 Mon Sep 17 00:00:00 2001 From: ayushaga14 Date: Mon, 30 Dec 2024 00:28:32 +0530 Subject: [PATCH 69/73] add log --- .../threat/detection/tasks/MaliciousTrafficDetectorTask.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index d33401ec1e..bc652e6714 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -127,6 +127,8 @@ private boolean validateFilterForRequest( HttpResponseParams responseParam, FilterConfig apiFilter) { try { String message = responseParam.getOrig(); + // todo: remove + System.out.println("using buildFromMessageNew func"); RawApi rawApi = RawApi.buildFromMessageNew(message); int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); responseParam.requestParams.setApiCollectionId(apiCollectionId); From f44605aa7daa7bccce7b608c20054deb96cb974c Mon Sep 17 00:00:00 2001 From: ayushaga14 Date: Mon, 30 Dec 2024 21:03:02 +0530 Subject: [PATCH 70/73] avoid reevaluating params for each filter check --- .../tasks/MaliciousTrafficDetectorTask.java | 43 ++++++++++--------- 1 file changed, 22 insertions(+), 21 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index bc652e6714..d92a40ba02 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -123,28 +123,20 @@ private Map getFilters() { return apiFilters; } - private boolean validateFilterForRequest( - HttpResponseParams responseParam, FilterConfig apiFilter) { + private boolean validateFilterForRequest(FilterConfig apiFilter, RawApi rawApi, ApiInfo.ApiInfoKey apiInfoKey, String message) { try { - String message = responseParam.getOrig(); - // todo: remove System.out.println("using buildFromMessageNew func"); - RawApi rawApi = RawApi.buildFromMessageNew(message); - int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); - responseParam.requestParams.setApiCollectionId(apiCollectionId); - String url = responseParam.getRequestParams().getURL(); - URLMethods.Method method = - URLMethods.Method.fromString(responseParam.getRequestParams().getMethod()); - ApiInfo.ApiInfoKey apiInfoKey = new ApiInfo.ApiInfoKey(apiCollectionId, url, method); + Map varMap = apiFilter.resolveVarMap(); VariableResolver.resolveWordList( - varMap, - new HashMap>() { - { - put(apiInfoKey, Collections.singletonList(message)); - } - }, - apiInfoKey); + varMap, + new HashMap>() { + { + put(apiInfoKey, Collections.singletonList(message)); + } + }, + apiInfoKey); + String filterExecutionLogId = UUID.randomUUID().toString(); ValidationResult res = TestPlugin.validateFilter( @@ -159,7 +151,7 @@ private boolean validateFilterForRequest( } private void processRecord(ConsumerRecord record) { - System.out.println("Kafka record: " + record.value()); + System.out.println("Kafka record: found - "); HttpResponseParams responseParam = HttpCallParser.parseKafkaMessage(record.value()); Context.accountId.set(Integer.parseInt(responseParam.getAccountId())); Map filters = this.getFilters(); @@ -172,8 +164,17 @@ private void processRecord(ConsumerRecord record) { System.out.println("Total number of filters: " + filters.size()); + String message = responseParam.getOrig(); + RawApi rawApi = RawApi.buildFromMessageNew(message); + int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); + responseParam.requestParams.setApiCollectionId(apiCollectionId); + String url = responseParam.getRequestParams().getURL(); + URLMethods.Method method = + URLMethods.Method.fromString(responseParam.getRequestParams().getMethod()); + ApiInfo.ApiInfoKey apiInfoKey = new ApiInfo.ApiInfoKey(apiCollectionId, url, method); + for (FilterConfig apiFilter : apiFilters.values()) { - boolean hasPassedFilter = validateFilterForRequest(responseParam, apiFilter); + boolean hasPassedFilter = validateFilterForRequest(apiFilter, rawApi, apiInfoKey, message); // If a request passes any of the filter, then it's a malicious request, // and so we push it to kafka @@ -280,7 +281,7 @@ private void generateAndPushMaliciousEventRequest( .setDetectedAt(responseParam.getTime()) .build(); try { - System.out.println("Pushing malicious event to kafka: " + maliciousEvent); + System.out.println("Pushing malicious event to kafka: "); MessageEnvelope.generateEnvelope(responseParam.getAccountId(), actor, maliciousEvent) .marshal() .ifPresent( From 009f014235364080b3a4e5ffc572d2374bd6f908 Mon Sep 17 00:00:00 2001 From: ayushaga14 Date: Mon, 30 Dec 2024 21:05:04 +0530 Subject: [PATCH 71/73] [temp commit] remove unneceesary build from stagiong workflow --- .github/workflows/staging.yml | 23 +++-------------------- 1 file changed, 3 insertions(+), 20 deletions(-) diff --git a/.github/workflows/staging.yml b/.github/workflows/staging.yml index ff95a5d2bb..6bb9697195 100644 --- a/.github/workflows/staging.yml +++ b/.github/workflows/staging.yml @@ -41,9 +41,7 @@ jobs: wget -O general.json https://raw.githubusercontent.com/akto-api-security/pii-types/master/general.json wget -O fintech.json https://raw.githubusercontent.com/akto-api-security/akto/master/pii-types/fintech.json wget -O filetypes.json https://raw.githubusercontent.com/akto-api-security/akto/master/pii-types/filetypes.json - - name: Prepare Dashboard polaris UI - working-directory: ./apps/dashboard/web/polaris_web - run: npm install && export RELEASE_VERSION=${{steps.docker_tag.outputs.IMAGE_TAG}} && npm run build + - name: Configure AWS Credentials uses: aws-actions/configure-aws-credentials@v1 with: @@ -51,10 +49,7 @@ jobs: aws-secret-access-key: ${{secrets.AWS_SECRET_ACCESS_KEY}} aws-region: ap-south-1 - - name: Deploy polaris site to S3 bucket - run: aws s3 sync ./apps/dashboard/web/polaris_web/web/dist s3://dashboard-on-cdn/polaris_web/${{steps.docker_tag.outputs.IMAGE_TAG}}/dist --delete - - - run: mvn package -Dakto-image-tag=${{ github.event.inputs.Tag }} -Dakto-build-time=$(eval "date +%s") -Dakto-release-version=${{steps.docker_tag.outputs.IMAGE_TAG}} + - run: mvn package -Dakto-image-tag=${{ github.event.inputs.Tag }} -Dakto-build-time=$(eval "date +%s") -Dakto-release-version=${{steps.docker_tag.outputs.IMAGE_TAG}} -DskipTests - name: DockerHub login env: DOCKER_USERNAME: ${{secrets.DOCKER_USERNAME}} @@ -74,22 +69,10 @@ jobs: echo $IMAGE_TAG >> $GITHUB_STEP_SUMMARY docker buildx create --use # Build a docker container and push it to DockerHub - cd apps/dashboard - docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/$ECR_REPOSITORY-dashboard:$IMAGE_TAG $IMAGE_TAG_DASHBOARD . --push - cd ../testing - docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-api-testing:$IMAGE_TAG $IMAGE_TAG_TESTING . --push - cd ../testing-cli - docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-api-testing-cli:$IMAGE_TAG $IMAGE_TAG_TESTING_CLI . --push - cd ../billing - docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-billing:$IMAGE_TAG . --push - cd ../internal - docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-internal:$IMAGE_TAG . --push - cd ../threat-detection + cd apps/threat-detection docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-threat-detection:$IMAGE_TAG . --push cd ../threat-detection-backend docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/akto-threat-detection-backend:$IMAGE_TAG . --push - cd ../source-code-analyser - docker buildx build --platform linux/arm64/v8,linux/amd64 -t $ECR_REGISTRY/source-code-analyser:$IMAGE_TAG . --push - name: Set up JDK 11 uses: actions/setup-java@v1 From 14f676f6aee9c2fd2933503ecb8e1b535dc0c8df Mon Sep 17 00:00:00 2001 From: Ajinkya <109141486+ag060@users.noreply.github.com> Date: Wed, 1 Jan 2025 10:26:47 +0530 Subject: [PATCH 72/73] using proto message envelope for kafka transport in threat detection client (#1874) --- .gitignore | 1 + .../threat/backend/service/HealthService.java | 17 -- .../detection/kafka/KafkaProtoProducer.java | 48 ++++ .../tasks/AbstractKafkaConsumerTask.java | 26 +- .../detection/tasks/FlushSampleDataTask.java | 43 ++-- .../tasks/MaliciousTrafficDetectorTask.java | 61 +++-- .../tasks/SendMaliciousEventsToBackend.java | 226 +++++++++--------- protobuf/health/service/v1/service.proto | 24 -- .../message/malicious_event/v1/message.proto | 6 + .../message/sample_request/v1/message.proto | 6 + 10 files changed, 238 insertions(+), 220 deletions(-) delete mode 100644 apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/HealthService.java create mode 100644 apps/threat-detection/src/main/java/com/akto/threat/detection/kafka/KafkaProtoProducer.java delete mode 100644 protobuf/health/service/v1/service.proto diff --git a/.gitignore b/.gitignore index 685da2accc..d90e415a62 100644 --- a/.gitignore +++ b/.gitignore @@ -22,3 +22,4 @@ https: **/data-zoo-data **/data-zoo-logs **/bin +.factorypath diff --git a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/HealthService.java b/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/HealthService.java deleted file mode 100644 index 5469ed7d16..0000000000 --- a/apps/threat-detection-backend/src/main/java/com/akto/threat/backend/service/HealthService.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.akto.threat.backend.service; - -import com.akto.proto.generated.health.service.v1.CheckRequest; -import com.akto.proto.generated.health.service.v1.CheckResponse; -import com.akto.proto.generated.health.service.v1.CheckResponse.ServingStatus; -import com.akto.proto.generated.health.service.v1.HealthServiceGrpc; -import io.grpc.stub.StreamObserver; - -public class HealthService extends HealthServiceGrpc.HealthServiceImplBase { - - @Override - public void check(CheckRequest request, StreamObserver responseObserver) { - responseObserver.onNext( - CheckResponse.newBuilder().setStatus(ServingStatus.SERVING_STATUS_SERVING).build()); - responseObserver.onCompleted(); - } -} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/kafka/KafkaProtoProducer.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/kafka/KafkaProtoProducer.java new file mode 100644 index 0000000000..cfd51ace83 --- /dev/null +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/kafka/KafkaProtoProducer.java @@ -0,0 +1,48 @@ +package com.akto.threat.detection.kafka; + +import com.akto.kafka.KafkaConfig; +import com.google.protobuf.Message; +import java.time.Duration; +import java.util.Properties; +import org.apache.kafka.clients.producer.*; +import org.apache.kafka.common.serialization.StringSerializer; + +public class KafkaProtoProducer { + private KafkaProducer producer; + public boolean producerReady; + + public KafkaProtoProducer(KafkaConfig kafkaConfig) { + this.producer = generateProducer( + kafkaConfig.getBootstrapServers(), + kafkaConfig.getProducerConfig().getLingerMs(), + kafkaConfig.getProducerConfig().getBatchSize()); + } + + public void send(String topic, Message message) { + byte[] messageBytes = message.toByteArray(); + this.producer.send(new ProducerRecord<>(topic, messageBytes)); + } + + public void close() { + this.producerReady = false; + producer.close(Duration.ofMillis(0)); // close immediately + } + + private KafkaProducer generateProducer(String brokerIP, int lingerMS, int batchSize) { + if (producer != null) + close(); // close existing producer connection + + int requestTimeoutMs = 5000; + Properties kafkaProps = new Properties(); + kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerIP); + kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArraySerializer"); + kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + kafkaProps.put(ProducerConfig.BATCH_SIZE_CONFIG, batchSize); + kafkaProps.put(ProducerConfig.LINGER_MS_CONFIG, lingerMS); + kafkaProps.put(ProducerConfig.RETRIES_CONFIG, 0); + kafkaProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs); + kafkaProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, lingerMS + requestTimeoutMs); + return new KafkaProducer(kafkaProps); + } +} diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java index cfd9ccecd6..4a0a731523 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/AbstractKafkaConsumerTask.java @@ -10,10 +10,12 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.StringDeserializer; -public abstract class AbstractKafkaConsumerTask implements Task { +public abstract class AbstractKafkaConsumerTask implements Task { - protected Consumer kafkaConsumer; + protected Consumer kafkaConsumer; protected KafkaConfig kafkaConfig; protected String kafkaTopic; @@ -24,9 +26,16 @@ public AbstractKafkaConsumerTask(KafkaConfig kafkaConfig, String kafkaTopic) { String kafkaBrokerUrl = kafkaConfig.getBootstrapServers(); String groupId = kafkaConfig.getGroupId(); - Properties properties = - Utils.configProperties( - kafkaBrokerUrl, groupId, kafkaConfig.getConsumerConfig().getMaxPollRecords()); + Properties properties = new Properties(); + properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaBrokerUrl); + properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + properties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, kafkaConfig.getConsumerConfig().getMaxPollRecords()); + properties.put(ConsumerConfig.GROUP_ID_CONFIG, kafkaConfig.getGroupId()); + properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + this.kafkaConsumer = new KafkaConsumer<>(properties); } @@ -40,9 +49,8 @@ public void run() { () -> { // Poll data from Kafka topic while (true) { - ConsumerRecords records = - kafkaConsumer.poll( - Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); + ConsumerRecords records = kafkaConsumer.poll( + Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); if (records.isEmpty()) { continue; } @@ -60,5 +68,5 @@ public void run() { }); } - abstract void processRecords(ConsumerRecords records); + abstract void processRecords(ConsumerRecords records); } diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java index 6829d6ecad..4b857c389e 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/FlushSampleDataTask.java @@ -3,6 +3,7 @@ import com.akto.dto.type.URLMethods; import com.akto.kafka.KafkaConfig; import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleRequestKafkaEnvelope; import com.akto.threat.detection.db.entity.MaliciousEventEntity; import com.akto.threat.detection.dto.MessageEnvelope; import com.google.protobuf.InvalidProtocolBufferException; @@ -17,7 +18,7 @@ /* This will read sample malicious data from kafka topic and save it to DB. */ -public class FlushSampleDataTask extends AbstractKafkaConsumerTask { +public class FlushSampleDataTask extends AbstractKafkaConsumerTask { private final SessionFactory sessionFactory; @@ -27,37 +28,29 @@ public FlushSampleDataTask( this.sessionFactory = sessionFactory; } - protected void processRecords(ConsumerRecords records) { + protected void processRecords(ConsumerRecords records) { List events = new ArrayList<>(); records.forEach( r -> { - String message = r.value(); - SampleMaliciousRequest.Builder builder = SampleMaliciousRequest.newBuilder(); - MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); - if (m == null) { - return; - } - + SampleRequestKafkaEnvelope envelope; try { - JsonFormat.parser().merge(m.getData(), builder); + envelope = SampleRequestKafkaEnvelope.parseFrom(r.value()); + SampleMaliciousRequest evt = envelope.getMaliciousRequest(); + + events.add( + MaliciousEventEntity.newBuilder() + .setActor(envelope.getActor()) + .setFilterId(evt.getFilterId()) + .setUrl(evt.getUrl()) + .setMethod(URLMethods.Method.fromString(evt.getMethod())) + .setTimestamp(evt.getTimestamp()) + .setOrig(evt.getPayload()) + .setApiCollectionId(evt.getApiCollectionId()) + .setIp(evt.getIp()) + .build()); } catch (InvalidProtocolBufferException e) { e.printStackTrace(); - return; } - - SampleMaliciousRequest evt = builder.build(); - - events.add( - MaliciousEventEntity.newBuilder() - .setActor(m.getActor()) - .setFilterId(evt.getFilterId()) - .setUrl(evt.getUrl()) - .setMethod(URLMethods.Method.fromString(evt.getMethod())) - .setTimestamp(evt.getTimestamp()) - .setOrig(evt.getPayload()) - .setApiCollectionId(evt.getApiCollectionId()) - .setIp(evt.getIp()) - .build()); }); Session session = this.sessionFactory.openSession(); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index d92a40ba02..090630014d 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -14,11 +14,12 @@ import com.akto.dto.test_editor.YamlTemplate; import com.akto.dto.type.URLMethods; import com.akto.hybrid_parsers.HttpCallParser; -import com.akto.kafka.Kafka; import com.akto.kafka.KafkaConfig; import com.akto.proto.generated.threat_detection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventKafkaEnvelope; import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventMessage; import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; +import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleRequestKafkaEnvelope; import com.akto.rules.TestPlugin; import com.akto.runtime.utils.Utils; import com.akto.test_editor.execution.VariableResolver; @@ -27,6 +28,7 @@ import com.akto.threat.detection.cache.RedisBackedCounterCache; import com.akto.threat.detection.constants.KafkaTopic; import com.akto.threat.detection.dto.MessageEnvelope; +import com.akto.threat.detection.kafka.KafkaProtoProducer; import com.akto.threat.detection.smart_event_detector.window_based.WindowBasedThresholdNotifier; import com.google.protobuf.InvalidProtocolBufferException; import io.lettuce.core.RedisClient; @@ -54,7 +56,7 @@ public class MaliciousTrafficDetectorTask implements Task { private int filterLastUpdatedAt = 0; private int filterUpdateIntervalSec = 300; - private final Kafka internalKafka; + private final KafkaProtoProducer internalKafka; private static final DataActor dataActor = DataActorFactory.fetchInstance(); @@ -77,11 +79,7 @@ public MaliciousTrafficDetectorTask( new RedisBackedCounterCache(redisClient, "wbt"), new WindowBasedThresholdNotifier.Config(100, 10 * 60)); - this.internalKafka = - new Kafka( - internalConfig.getBootstrapServers(), - internalConfig.getProducerConfig().getLingerMs(), - internalConfig.getProducerConfig().getBatchSize()); + this.internalKafka = new KafkaProtoProducer(internalConfig); } public void run() { @@ -123,19 +121,20 @@ private Map getFilters() { return apiFilters; } - private boolean validateFilterForRequest(FilterConfig apiFilter, RawApi rawApi, ApiInfo.ApiInfoKey apiInfoKey, String message) { + private boolean validateFilterForRequest( + FilterConfig apiFilter, RawApi rawApi, ApiInfo.ApiInfoKey apiInfoKey, String message) { try { System.out.println("using buildFromMessageNew func"); Map varMap = apiFilter.resolveVarMap(); VariableResolver.resolveWordList( - varMap, - new HashMap>() { - { - put(apiInfoKey, Collections.singletonList(message)); - } - }, - apiInfoKey); + varMap, + new HashMap>() { + { + put(apiInfoKey, Collections.singletonList(message)); + } + }, + apiInfoKey); String filterExecutionLogId = UUID.randomUUID().toString(); ValidationResult res = @@ -160,7 +159,7 @@ private void processRecord(ConsumerRecord record) { return; } - List maliciousMessages = new ArrayList<>(); + List maliciousMessages = new ArrayList<>(); System.out.println("Total number of filters: " + filters.size()); @@ -212,13 +211,12 @@ private void processRecord(ConsumerRecord record) { .setFilterId(apiFilter.getId()) .build(); - try { - maliciousMessages.add( - MessageEnvelope.generateEnvelope( - responseParam.getAccountId(), actor, maliciousReq)); - } catch (InvalidProtocolBufferException e) { - return; - } + maliciousMessages.add( + SampleRequestKafkaEnvelope.newBuilder() + .setActor(actor) + .setAccountId(responseParam.getAccountId()) + .setMaliciousRequest(maliciousReq) + .build()); if (!isAggFilter) { generateAndPushMaliciousEventRequest( @@ -250,12 +248,7 @@ private void processRecord(ConsumerRecord record) { try { maliciousMessages.forEach( sample -> { - sample - .marshal() - .ifPresent( - data -> { - internalKafka.send(data, KafkaTopic.ThreatDetection.MALICIOUS_EVENTS); - }); + internalKafka.send(KafkaTopic.ThreatDetection.MALICIOUS_EVENTS, sample); }); } catch (Exception e) { e.printStackTrace(); @@ -281,12 +274,18 @@ private void generateAndPushMaliciousEventRequest( .setDetectedAt(responseParam.getTime()) .build(); try { - System.out.println("Pushing malicious event to kafka: "); + System.out.println("Pushing malicious event to kafka: " + maliciousEvent); + MaliciousEventKafkaEnvelope envelope = + MaliciousEventKafkaEnvelope.newBuilder() + .setActor(actor) + .setAccountId(responseParam.getAccountId()) + .setMaliciousEvent(maliciousEvent) + .build(); MessageEnvelope.generateEnvelope(responseParam.getAccountId(), actor, maliciousEvent) .marshal() .ifPresent( data -> { - internalKafka.send(data, KafkaTopic.ThreatDetection.ALERTS); + internalKafka.send(KafkaTopic.ThreatDetection.ALERTS, envelope); }); } catch (InvalidProtocolBufferException e) { e.printStackTrace(); diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java index c2a9e7cf54..2ee173f1a0 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/SendMaliciousEventsToBackend.java @@ -2,6 +2,7 @@ import com.akto.kafka.KafkaConfig; import com.akto.proto.generated.threat_detection.message.malicious_event.event_type.v1.EventType; +import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventKafkaEnvelope; import com.akto.proto.generated.threat_detection.message.malicious_event.v1.MaliciousEventMessage; import com.akto.proto.generated.threat_detection.message.sample_request.v1.SampleMaliciousRequest; import com.akto.proto.generated.threat_detection.service.malicious_alert_service.v1.RecordMaliciousEventRequest; @@ -28,133 +29,130 @@ /* This will send alerts to threat detection backend */ -public class SendMaliciousEventsToBackend extends AbstractKafkaConsumerTask { +public class SendMaliciousEventsToBackend extends AbstractKafkaConsumerTask { - private final SessionFactory sessionFactory; - private final CloseableHttpClient httpClient; + private final SessionFactory sessionFactory; + private final CloseableHttpClient httpClient; - public SendMaliciousEventsToBackend( - SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { - super(trafficConfig, topic); - this.sessionFactory = sessionFactory; - this.httpClient = HttpClients.createDefault(); - } + public SendMaliciousEventsToBackend( + SessionFactory sessionFactory, KafkaConfig trafficConfig, String topic) { + super(trafficConfig, topic); + this.sessionFactory = sessionFactory; + this.httpClient = HttpClients.createDefault(); + } - private void markSampleDataAsSent(List ids) { - Session session = this.sessionFactory.openSession(); - Transaction txn = session.beginTransaction(); - try { - session - .createQuery( - "update MaliciousEventEntity m set m.alertedToBackend = true where m.id in :ids") - .setParameterList("ids", ids) - .executeUpdate(); - } catch (Exception ex) { - ex.printStackTrace(); - txn.rollback(); - } finally { - txn.commit(); - session.close(); + private void markSampleDataAsSent(List ids) { + Session session = this.sessionFactory.openSession(); + Transaction txn = session.beginTransaction(); + try { + session + .createQuery( + "update MaliciousEventEntity m set m.alertedToBackend = true where m.id in :ids") + .setParameterList("ids", ids) + .executeUpdate(); + } catch (Exception ex) { + ex.printStackTrace(); + txn.rollback(); + } finally { + txn.commit(); + session.close(); + } } - } - private List getSampleMaliciousRequests(String actor, String filterId) { - Session session = this.sessionFactory.openSession(); - Transaction txn = session.beginTransaction(); - try { - return session - .createQuery( - "from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId and" - + " m.alertedToBackend = false order by m.createdAt desc", - MaliciousEventEntity.class) - .setParameter("actor", actor) - .setParameter("filterId", filterId) - .setMaxResults(50) - .getResultList(); - } catch (Exception ex) { - ex.printStackTrace(); - txn.rollback(); - } finally { - txn.commit(); - session.close(); + private List getSampleMaliciousRequests(String actor, String filterId) { + Session session = this.sessionFactory.openSession(); + Transaction txn = session.beginTransaction(); + try { + return session + .createQuery( + "from MaliciousEventEntity m where m.actor = :actor and m.filterId = :filterId and" + + " m.alertedToBackend = false order by m.createdAt desc", + MaliciousEventEntity.class) + .setParameter("actor", actor) + .setParameter("filterId", filterId) + .setMaxResults(50) + .getResultList(); + } catch (Exception ex) { + ex.printStackTrace(); + txn.rollback(); + } finally { + txn.commit(); + session.close(); + } + + return Collections.emptyList(); } - return Collections.emptyList(); - } + protected void processRecords(ConsumerRecords records) { + records.forEach( + r -> { + MaliciousEventKafkaEnvelope envelope; + try { + envelope = MaliciousEventKafkaEnvelope.parseFrom(r.value()); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + return; + } - protected void processRecords(ConsumerRecords records) { - records.forEach( - r -> { - String message = r.value(); - MaliciousEventMessage.Builder builder = MaliciousEventMessage.newBuilder(); - MessageEnvelope m = MessageEnvelope.unmarshal(message).orElse(null); - if (m == null) { - return; - } + if (envelope == null) { + return; + } - try { - JsonFormat.parser().merge(m.getData(), builder); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); - return; - } + try { + MaliciousEventMessage evt = envelope.getMaliciousEvent(); - MaliciousEventMessage evt = builder.build(); + // Get sample data from postgres for this alert + List sampleData = this.getSampleMaliciousRequests(evt.getActor(), + evt.getFilterId()); + RecordMaliciousEventRequest.Builder reqBuilder = RecordMaliciousEventRequest.newBuilder() + .setMaliciousEvent(evt); + if (EventType.EVENT_TYPE_AGGREGATED.equals(evt.getEventType())) { + sampleData = this.getSampleMaliciousRequests(evt.getActor(), evt.getFilterId()); - // Get sample data from postgres for this alert - List sampleData = - this.getSampleMaliciousRequests(evt.getActor(), evt.getFilterId()); - try { - RecordMaliciousEventRequest.Builder reqBuilder = - RecordMaliciousEventRequest.newBuilder().setMaliciousEvent(evt); - if (EventType.EVENT_TYPE_AGGREGATED.equals(evt.getEventType())) { - sampleData = this.getSampleMaliciousRequests(evt.getActor(), evt.getFilterId()); + reqBuilder.addAllSampleRequests( + sampleData.stream() + .map( + d -> SampleMaliciousRequest.newBuilder() + .setUrl(d.getUrl()) + .setMethod(d.getMethod().name()) + .setTimestamp(d.getTimestamp()) + .setPayload(d.getOrig()) + .setIp(d.getIp()) + .setApiCollectionId(d.getApiCollectionId()) + .build()) + .collect(Collectors.toList())); + } - reqBuilder.addAllSampleRequests( - sampleData.stream() - .map( - d -> - SampleMaliciousRequest.newBuilder() - .setUrl(d.getUrl()) - .setMethod(d.getMethod().name()) - .setTimestamp(d.getTimestamp()) - .setPayload(d.getOrig()) - .setIp(d.getIp()) - .setApiCollectionId(d.getApiCollectionId()) - .build()) - .collect(Collectors.toList())); - } + List sampleIds = sampleData.stream().map(MaliciousEventEntity::getId) + .collect(Collectors.toList()); - List sampleIds = - sampleData.stream().map(MaliciousEventEntity::getId).collect(Collectors.toList()); + RecordMaliciousEventRequest maliciousEventRequest = reqBuilder.build(); + String url = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); + String token = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"); + ProtoMessageUtils.toString(maliciousEventRequest) + .ifPresent( + msg -> { + StringEntity requestEntity = new StringEntity(msg, + ContentType.APPLICATION_JSON); + HttpPost req = new HttpPost( + String.format("%s/api/threat_detection/record_malicious_event", + url)); + req.addHeader("Authorization", "Bearer " + token); + req.setEntity(requestEntity); + try { + System.out.println("Sending request to backend: " + msg); + this.httpClient.execute(req); + } catch (IOException e) { + e.printStackTrace(); + } - RecordMaliciousEventRequest maliciousEventRequest = reqBuilder.build(); - String url = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_URL"); - String token = System.getenv("AKTO_THREAT_PROTECTION_BACKEND_TOKEN"); - ProtoMessageUtils.toString(maliciousEventRequest) - .ifPresent( - msg -> { - StringEntity requestEntity = - new StringEntity(msg, ContentType.APPLICATION_JSON); - HttpPost req = - new HttpPost( - String.format("%s/api/threat_detection/record_malicious_event", url)); - req.addHeader("Authorization", "Bearer " + token); - req.setEntity(requestEntity); - try { - System.out.println("Sending request to backend: " + msg); - this.httpClient.execute(req); - } catch (IOException e) { + if (!sampleIds.isEmpty()) { + markSampleDataAsSent(sampleIds); + } + }); + } catch (Exception e) { e.printStackTrace(); - } - - if (!sampleIds.isEmpty()) { - markSampleDataAsSent(sampleIds); - } - }); - } catch (Exception e) { - e.printStackTrace(); - } - }); - } + } + }); + } } diff --git a/protobuf/health/service/v1/service.proto b/protobuf/health/service/v1/service.proto deleted file mode 100644 index 755ed7d4c1..0000000000 --- a/protobuf/health/service/v1/service.proto +++ /dev/null @@ -1,24 +0,0 @@ -syntax = "proto3"; - -package health.service.v1; - -option java_outer_classname = "HealthServiceProto"; -option java_package = "health.service.v1"; - -message CheckRequest { - string service = 1; -} - -message CheckResponse { - enum ServingStatus { - SERVING_STATUS_UNSPECIFIED = 0; - SERVING_STATUS_SERVING = 1; - SERVING_STATUS_NOT_SERVING = 2; - SERVING_STATUS_SERVICE_UNKNOWN = 3; // Used only by the Watch method. - } - ServingStatus status = 1; -} - -service HealthService { - rpc Check(CheckRequest) returns (CheckResponse); -} \ No newline at end of file diff --git a/protobuf/threat_detection/message/malicious_event/v1/message.proto b/protobuf/threat_detection/message/malicious_event/v1/message.proto index 8c9fabff98..3850fc90ad 100644 --- a/protobuf/threat_detection/message/malicious_event/v1/message.proto +++ b/protobuf/threat_detection/message/malicious_event/v1/message.proto @@ -18,3 +18,9 @@ message MaliciousEventMessage { string latest_api_payload = 8; threat_detection.message.malicious_event.event_type.v1.EventType event_type = 9; } + +message MaliciousEventKafkaEnvelope { + string account_id = 1; + string actor = 2; + MaliciousEventMessage malicious_event = 3; +} diff --git a/protobuf/threat_detection/message/sample_request/v1/message.proto b/protobuf/threat_detection/message/sample_request/v1/message.proto index 1020e56636..4192572509 100644 --- a/protobuf/threat_detection/message/sample_request/v1/message.proto +++ b/protobuf/threat_detection/message/sample_request/v1/message.proto @@ -14,3 +14,9 @@ message SampleMaliciousRequest { string payload = 6; string filter_id = 7; } + +message SampleRequestKafkaEnvelope { + string account_id = 1; + string actor = 2; + SampleMaliciousRequest malicious_request = 3; +} From 018df017347bf8e6d71b2ea89af51f433f7e4a0d Mon Sep 17 00:00:00 2001 From: Ajinkya Date: Wed, 1 Jan 2025 10:32:31 +0530 Subject: [PATCH 73/73] not saving original request and response payload --- .../tasks/MaliciousTrafficDetectorTask.java | 510 ++++++++++-------- 1 file changed, 289 insertions(+), 221 deletions(-) diff --git a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java index 090630014d..6bd5e25ce3 100644 --- a/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java +++ b/apps/threat-detection/src/main/java/com/akto/threat/detection/tasks/MaliciousTrafficDetectorTask.java @@ -47,248 +47,316 @@ */ public class MaliciousTrafficDetectorTask implements Task { - private final Consumer kafkaConsumer; - private final KafkaConfig kafkaConfig; - private final HttpCallParser httpCallParser; - private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; + private final Consumer kafkaConsumer; + private final KafkaConfig kafkaConfig; + private final HttpCallParser httpCallParser; + private final WindowBasedThresholdNotifier windowBasedThresholdNotifier; - private Map apiFilters; - private int filterLastUpdatedAt = 0; - private int filterUpdateIntervalSec = 300; + private Map apiFilters; + private int filterLastUpdatedAt = 0; + private int filterUpdateIntervalSec = 300; - private final KafkaProtoProducer internalKafka; + private final KafkaProtoProducer internalKafka; - private static final DataActor dataActor = DataActorFactory.fetchInstance(); + private static final DataActor dataActor = DataActorFactory.fetchInstance(); - public MaliciousTrafficDetectorTask( - KafkaConfig trafficConfig, KafkaConfig internalConfig, RedisClient redisClient) { - this.kafkaConfig = trafficConfig; + public MaliciousTrafficDetectorTask( + KafkaConfig trafficConfig, + KafkaConfig internalConfig, + RedisClient redisClient + ) { + this.kafkaConfig = trafficConfig; - String kafkaBrokerUrl = trafficConfig.getBootstrapServers(); - String groupId = trafficConfig.getGroupId(); + String kafkaBrokerUrl = trafficConfig.getBootstrapServers(); + String groupId = trafficConfig.getGroupId(); - this.kafkaConsumer = - new KafkaConsumer<>( + this.kafkaConsumer = new KafkaConsumer<>( Utils.configProperties( - kafkaBrokerUrl, groupId, trafficConfig.getConsumerConfig().getMaxPollRecords())); + kafkaBrokerUrl, + groupId, + trafficConfig.getConsumerConfig().getMaxPollRecords() + ) + ); - this.httpCallParser = new HttpCallParser(120, 1000); + this.httpCallParser = new HttpCallParser(120, 1000); - this.windowBasedThresholdNotifier = - new WindowBasedThresholdNotifier( + this.windowBasedThresholdNotifier = new WindowBasedThresholdNotifier( new RedisBackedCounterCache(redisClient, "wbt"), - new WindowBasedThresholdNotifier.Config(100, 10 * 60)); - - this.internalKafka = new KafkaProtoProducer(internalConfig); - } - - public void run() { - this.kafkaConsumer.subscribe(Collections.singletonList("akto.api.logs")); - ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); - pollingExecutor.execute( - () -> { - // Poll data from Kafka topic - while (true) { - ConsumerRecords records = - kafkaConsumer.poll( - Duration.ofMillis(kafkaConfig.getConsumerConfig().getPollDurationMilli())); - - try { - for (ConsumerRecord record : records) { - processRecord(record); - } - - if (!records.isEmpty()) { - // Should we commit even if there are no records ? - kafkaConsumer.commitSync(); - } - } catch (Exception e) { - e.printStackTrace(); - } - } - }); - } + new WindowBasedThresholdNotifier.Config(100, 10 * 60) + ); - private Map getFilters() { - int now = (int) (System.currentTimeMillis() / 1000); - if (now - filterLastUpdatedAt < filterUpdateIntervalSec) { - return apiFilters; + this.internalKafka = new KafkaProtoProducer(internalConfig); } - List templates = dataActor.fetchFilterYamlTemplates(); - apiFilters = FilterYamlTemplateDao.fetchFilterConfig(false, templates, false); - this.filterLastUpdatedAt = now; - return apiFilters; - } - - private boolean validateFilterForRequest( - FilterConfig apiFilter, RawApi rawApi, ApiInfo.ApiInfoKey apiInfoKey, String message) { - try { - System.out.println("using buildFromMessageNew func"); - - Map varMap = apiFilter.resolveVarMap(); - VariableResolver.resolveWordList( - varMap, - new HashMap>() { - { - put(apiInfoKey, Collections.singletonList(message)); - } - }, - apiInfoKey); - - String filterExecutionLogId = UUID.randomUUID().toString(); - ValidationResult res = - TestPlugin.validateFilter( - apiFilter.getFilter().getNode(), rawApi, apiInfoKey, varMap, filterExecutionLogId); + public void run() { + this.kafkaConsumer.subscribe( + Collections.singletonList("akto.api.logs") + ); + ExecutorService pollingExecutor = Executors.newSingleThreadExecutor(); + pollingExecutor.execute(() -> { + // Poll data from Kafka topic + while (true) { + ConsumerRecords records = kafkaConsumer.poll( + Duration.ofMillis( + kafkaConfig.getConsumerConfig().getPollDurationMilli() + ) + ); + + try { + for (ConsumerRecord record : records) { + processRecord(record); + } - return res.getIsValid(); - } catch (Exception e) { - e.printStackTrace(); + if (!records.isEmpty()) { + // Should we commit even if there are no records ? + kafkaConsumer.commitSync(); + } + } catch (Exception e) { + e.printStackTrace(); + } + } + }); } - return false; - } - - private void processRecord(ConsumerRecord record) { - System.out.println("Kafka record: found - "); - HttpResponseParams responseParam = HttpCallParser.parseKafkaMessage(record.value()); - Context.accountId.set(Integer.parseInt(responseParam.getAccountId())); - Map filters = this.getFilters(); - if (filters.isEmpty()) { - System.out.println("No filters found"); - return; + private Map getFilters() { + int now = (int) (System.currentTimeMillis() / 1000); + if (now - filterLastUpdatedAt < filterUpdateIntervalSec) { + return apiFilters; + } + + List templates = dataActor.fetchFilterYamlTemplates(); + apiFilters = FilterYamlTemplateDao.fetchFilterConfig( + false, + templates, + false + ); + this.filterLastUpdatedAt = now; + return apiFilters; } - List maliciousMessages = new ArrayList<>(); - - System.out.println("Total number of filters: " + filters.size()); - - String message = responseParam.getOrig(); - RawApi rawApi = RawApi.buildFromMessageNew(message); - int apiCollectionId = httpCallParser.createApiCollectionId(responseParam); - responseParam.requestParams.setApiCollectionId(apiCollectionId); - String url = responseParam.getRequestParams().getURL(); - URLMethods.Method method = - URLMethods.Method.fromString(responseParam.getRequestParams().getMethod()); - ApiInfo.ApiInfoKey apiInfoKey = new ApiInfo.ApiInfoKey(apiCollectionId, url, method); - - for (FilterConfig apiFilter : apiFilters.values()) { - boolean hasPassedFilter = validateFilterForRequest(apiFilter, rawApi, apiInfoKey, message); - - // If a request passes any of the filter, then it's a malicious request, - // and so we push it to kafka - if (hasPassedFilter) { - // Later we will also add aggregation support - // Eg: 100 4xx requests in last 10 minutes. - // But regardless of whether request falls in aggregation or not, - // we still push malicious requests to kafka - - System.out.println("Filter passed: " + apiFilter.getId()); - - // todo: modify fetch yaml and read aggregate rules from it - List rules = new ArrayList<>(); - rules.add(new Rule("Lfi Rule 1", new Condition(10, 10))); - AggregationRules aggRules = new AggregationRules(); - aggRules.setRule(rules); - - boolean isAggFilter = aggRules != null && !aggRules.getRule().isEmpty(); - - SourceIPActorGenerator.instance - .generate(responseParam) - .ifPresent( - actor -> { - String groupKey = apiFilter.getId(); - String aggKey = actor + "|" + groupKey; - - SampleMaliciousRequest maliciousReq = - SampleMaliciousRequest.newBuilder() - .setUrl(responseParam.getRequestParams().getURL()) - .setMethod(responseParam.getRequestParams().getMethod()) - .setPayload(responseParam.getOrig()) - .setIp(actor) // For now using actor as IP - .setApiCollectionId(responseParam.getRequestParams().getApiCollectionId()) - .setTimestamp(responseParam.getTime()) - .setFilterId(apiFilter.getId()) - .build(); - - maliciousMessages.add( - SampleRequestKafkaEnvelope.newBuilder() - .setActor(actor) - .setAccountId(responseParam.getAccountId()) - .setMaliciousRequest(maliciousReq) - .build()); - - if (!isAggFilter) { - generateAndPushMaliciousEventRequest( - apiFilter, actor, responseParam, maliciousReq, EventType.EVENT_TYPE_SINGLE); - return; - } - - // Aggregation rules - for (Rule rule : aggRules.getRule()) { - WindowBasedThresholdNotifier.Result result = - this.windowBasedThresholdNotifier.shouldNotify(aggKey, maliciousReq, rule); - - if (result.shouldNotify()) { - System.out.print("Notifying for aggregation rule: " + rule); - generateAndPushMaliciousEventRequest( - apiFilter, - actor, - responseParam, - maliciousReq, - EventType.EVENT_TYPE_AGGREGATED); + private boolean validateFilterForRequest( + FilterConfig apiFilter, + RawApi rawApi, + ApiInfo.ApiInfoKey apiInfoKey, + String message + ) { + try { + System.out.println("using buildFromMessageNew func"); + + Map varMap = apiFilter.resolveVarMap(); + VariableResolver.resolveWordList( + varMap, + new HashMap>() { + { + put(apiInfoKey, Collections.singletonList(message)); } - } - }); - } + }, + apiInfoKey + ); + + String filterExecutionLogId = UUID.randomUUID().toString(); + ValidationResult res = TestPlugin.validateFilter( + apiFilter.getFilter().getNode(), + rawApi, + apiInfoKey, + varMap, + filterExecutionLogId + ); + + return res.getIsValid(); + } catch (Exception e) { + e.printStackTrace(); + } + + return false; } - // Should we push all the messages in one go - // or call kafka.send for each HttpRequestParams - try { - maliciousMessages.forEach( - sample -> { - internalKafka.send(KafkaTopic.ThreatDetection.MALICIOUS_EVENTS, sample); - }); - } catch (Exception e) { - e.printStackTrace(); + private void processRecord(ConsumerRecord record) { + System.out.println("Kafka record: found - "); + HttpResponseParams responseParam = HttpCallParser.parseKafkaMessage( + record.value() + ); + Context.accountId.set(Integer.parseInt(responseParam.getAccountId())); + Map filters = this.getFilters(); + if (filters.isEmpty()) { + System.out.println("No filters found"); + return; + } + + List maliciousMessages = new ArrayList<>(); + + System.out.println("Total number of filters: " + filters.size()); + + String message = responseParam.getOrig(); + RawApi rawApi = RawApi.buildFromMessageNew(message); + int apiCollectionId = httpCallParser.createApiCollectionId( + responseParam + ); + responseParam.requestParams.setApiCollectionId(apiCollectionId); + String url = responseParam.getRequestParams().getURL(); + URLMethods.Method method = URLMethods.Method.fromString( + responseParam.getRequestParams().getMethod() + ); + ApiInfo.ApiInfoKey apiInfoKey = new ApiInfo.ApiInfoKey( + apiCollectionId, + url, + method + ); + + for (FilterConfig apiFilter : apiFilters.values()) { + boolean hasPassedFilter = validateFilterForRequest( + apiFilter, + rawApi, + apiInfoKey, + message + ); + + // If a request passes any of the filter, then it's a malicious request, + // and so we push it to kafka + if (hasPassedFilter) { + // Later we will also add aggregation support + // Eg: 100 4xx requests in last 10 minutes. + // But regardless of whether request falls in aggregation or not, + // we still push malicious requests to kafka + + System.out.println("Filter passed: " + apiFilter.getId()); + + // todo: modify fetch yaml and read aggregate rules from it + List rules = new ArrayList<>(); + rules.add(new Rule("Lfi Rule 1", new Condition(10, 10))); + AggregationRules aggRules = new AggregationRules(); + aggRules.setRule(rules); + + boolean isAggFilter = + aggRules != null && !aggRules.getRule().isEmpty(); + + SourceIPActorGenerator.instance + .generate(responseParam) + .ifPresent(actor -> { + String groupKey = apiFilter.getId(); + String aggKey = actor + "|" + groupKey; + + SampleMaliciousRequest maliciousReq = + SampleMaliciousRequest.newBuilder() + .setUrl( + responseParam.getRequestParams().getURL() + ) + .setMethod( + responseParam.getRequestParams().getMethod() + ) + .setPayload(responseParam.getOrig()) + .setIp(actor) // For now using actor as IP + .setApiCollectionId( + responseParam + .getRequestParams() + .getApiCollectionId() + ) + .setTimestamp(responseParam.getTime()) + .setFilterId(apiFilter.getId()) + .build(); + + maliciousMessages.add( + SampleRequestKafkaEnvelope.newBuilder() + .setActor(actor) + .setAccountId(responseParam.getAccountId()) + .setMaliciousRequest(maliciousReq) + .build() + ); + + if (!isAggFilter) { + generateAndPushMaliciousEventRequest( + apiFilter, + actor, + responseParam, + maliciousReq, + EventType.EVENT_TYPE_SINGLE + ); + return; + } + + // Aggregation rules + for (Rule rule : aggRules.getRule()) { + WindowBasedThresholdNotifier.Result result = + this.windowBasedThresholdNotifier.shouldNotify( + aggKey, + maliciousReq, + rule + ); + + if (result.shouldNotify()) { + System.out.print( + "Notifying for aggregation rule: " + rule + ); + generateAndPushMaliciousEventRequest( + apiFilter, + actor, + responseParam, + maliciousReq, + EventType.EVENT_TYPE_AGGREGATED + ); + } + } + }); + } + } + + // Should we push all the messages in one go + // or call kafka.send for each HttpRequestParams + try { + maliciousMessages.forEach(sample -> { + internalKafka.send( + KafkaTopic.ThreatDetection.MALICIOUS_EVENTS, + sample + ); + }); + } catch (Exception e) { + e.printStackTrace(); + } } - } - - private void generateAndPushMaliciousEventRequest( - FilterConfig apiFilter, - String actor, - HttpResponseParams responseParam, - SampleMaliciousRequest maliciousReq, - EventType eventType) { - MaliciousEventMessage maliciousEvent = - MaliciousEventMessage.newBuilder() - .setFilterId(apiFilter.getId()) - .setActor(actor) - .setDetectedAt(responseParam.getTime()) - .setEventType(eventType) - .setLatestApiCollectionId(maliciousReq.getApiCollectionId()) - .setLatestApiIp(maliciousReq.getIp()) - .setLatestApiPayload(maliciousReq.getPayload()) - .setLatestApiMethod(maliciousReq.getMethod()) - .setDetectedAt(responseParam.getTime()) - .build(); - try { - System.out.println("Pushing malicious event to kafka: " + maliciousEvent); - MaliciousEventKafkaEnvelope envelope = - MaliciousEventKafkaEnvelope.newBuilder() - .setActor(actor) - .setAccountId(responseParam.getAccountId()) - .setMaliciousEvent(maliciousEvent) - .build(); - MessageEnvelope.generateEnvelope(responseParam.getAccountId(), actor, maliciousEvent) - .marshal() - .ifPresent( - data -> { - internalKafka.send(KafkaTopic.ThreatDetection.ALERTS, envelope); - }); - } catch (InvalidProtocolBufferException e) { - e.printStackTrace(); + + private void generateAndPushMaliciousEventRequest( + FilterConfig apiFilter, + String actor, + HttpResponseParams responseParam, + SampleMaliciousRequest maliciousReq, + EventType eventType + ) { + MaliciousEventMessage maliciousEvent = + MaliciousEventMessage.newBuilder() + .setFilterId(apiFilter.getId()) + .setActor(actor) + .setDetectedAt(responseParam.getTime()) + .setEventType(eventType) + .setLatestApiCollectionId(maliciousReq.getApiCollectionId()) + .setLatestApiIp(maliciousReq.getIp()) + .setLatestApiPayload("") // ignoring original payload for now + .setLatestApiMethod(maliciousReq.getMethod()) + .setDetectedAt(responseParam.getTime()) + .build(); + try { + System.out.println( + "Pushing malicious event to kafka: " + maliciousEvent + ); + MaliciousEventKafkaEnvelope envelope = + MaliciousEventKafkaEnvelope.newBuilder() + .setActor(actor) + .setAccountId(responseParam.getAccountId()) + .setMaliciousEvent(maliciousEvent) + .build(); + MessageEnvelope.generateEnvelope( + responseParam.getAccountId(), + actor, + maliciousEvent + ) + .marshal() + .ifPresent(data -> { + internalKafka.send( + KafkaTopic.ThreatDetection.ALERTS, + envelope + ); + }); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } } - } }