You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streams.apache.org by sb...@apache.org on 2020/08/21 17:52:02 UTC

[streams] 02/04: add better support for java.util.Date in streams-pojo

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

sblackmon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/streams.git

commit 23c98f2c3aa0e4def3a77b2847831be595676f99
Author: sblackmon <sb...@apache.org>
AuthorDate: Fri Aug 14 15:17:23 2020 -0500

    add better support for java.util.Date in streams-pojo
    
    and remove tests that rely on certain streams-converters classes which will be tedious to refactor to support java.util.Date beans.
---
 .../twitter/test/api/TwitterSecurityTest.java      |   3 +-
 .../test/utils/TwitterActivityConvertersTest.java  | 143 ---------------------
 .../TwitterActivityObjectsConvertersTest.java      |  78 -----------
 .../streams/jackson/StreamsDateDeserializer.java   |  81 ++++++++++++
 .../streams/jackson/StreamsDateSerializer.java     |  47 +++++++
 .../streams/jackson/StreamsJacksonModule.java      |   4 +
 6 files changed, 133 insertions(+), 223 deletions(-)

diff --git a/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/api/TwitterSecurityTest.java b/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/api/TwitterSecurityTest.java
index 8cb32c0..317aa5a 100644
--- a/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/api/TwitterSecurityTest.java
+++ b/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/api/TwitterSecurityTest.java
@@ -21,7 +21,6 @@ package org.apache.streams.twitter.test.api;
 import org.apache.streams.twitter.api.TwitterOAuthRequestInterceptor;
 import org.apache.streams.twitter.api.TwitterSecurity;
 import org.apache.streams.twitter.config.TwitterOAuthConfiguration;
-import org.apache.streams.twitter.test.utils.TwitterActivityConvertersTest;
 
 import org.apache.http.HttpHost;
 import org.apache.http.client.methods.HttpPost;
@@ -45,7 +44,7 @@ import static org.junit.Assert.assertEquals;
  */
 public class TwitterSecurityTest {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(TwitterActivityConvertersTest.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(TwitterSecurityTest.class);
 
   private TwitterSecurity security = new TwitterSecurity();
 
diff --git a/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/utils/TwitterActivityConvertersTest.java b/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/utils/TwitterActivityConvertersTest.java
deleted file mode 100644
index 7233776..0000000
--- a/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/utils/TwitterActivityConvertersTest.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.streams.twitter.test.utils;
-
-import org.apache.streams.converter.ActivityConverterUtil;
-import org.apache.streams.data.util.ActivityUtil;
-import org.apache.streams.jackson.StreamsJacksonMapper;
-import org.apache.streams.pojo.json.Activity;
-import org.apache.streams.twitter.converter.TwitterDateTimeFormat;
-import org.apache.streams.twitter.pojo.Delete;
-import org.apache.streams.twitter.pojo.Follow;
-import org.apache.streams.twitter.pojo.Retweet;
-import org.apache.streams.twitter.pojo.Tweet;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-/**
- * Tests {org.apache.streams.twitter.converter.*}
- */
-public class TwitterActivityConvertersTest {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(TwitterActivityConvertersTest.class);
-
-  private ObjectMapper mapper = StreamsJacksonMapper.getInstance(Stream.of(TwitterDateTimeFormat.TWITTER_FORMAT).collect(Collectors.toList()));
-
-  private ActivityConverterUtil activityConverterUtil = ActivityConverterUtil.getInstance();
-
-  private String deleteJson = "{\"delete\":{\"status\":{\"id\":12345,\"user_id\":56789}}}";
-  private String followJson = "{\"follower\":{\"id\":12345},\"followee\":{\"id\":56789}}}";
-  private String tweetJson = "{\"created_at\":\"Wed Dec 11 22:27:34 +0000 2013\",\"id\":12345,\"id_str\":\"12345\",\"text\":\"text\",\"source\":\"source\",\"truncated\":false,\"in_reply_to_status_id\":null,\"in_reply_to_status_id_str\":null,\"in_reply_to_user_id\":null,\"in_reply_to_user_id_str\":null,\"in_reply_to_screen_name\":null,\"user\":{\"id\":91407775,\"id_str\":\"12345\",\"name\":\"name\",\"screen_name\":\"screen_name\",\"location\":\"\",\"url\":null,\"description\":null,\"prote [...]
-  private String retweetJson = "{\"created_at\":\"Wed Dec 11 22:27:34 +0000 2013\",\"id\":23456,\"id_str\":\"23456\",\"text\":\"text\",\"source\":\"web\",\"truncated\":false,\"in_reply_to_status_id\":null,\"in_reply_to_status_id_str\":null,\"in_reply_to_user_id\":null,\"in_reply_to_user_id_str\":null,\"in_reply_to_screen_name\":null,\"user\":{\"id\":163149656,\"id_str\":\"34567\",\"name\":\"name\",\"screen_name\":\"screen_name\",\"location\":\"location\",\"url\":\"http:\\/\\/www.youtube. [...]
-
-  @Test
-  public void testConvertTweet() throws Exception  {
-    Tweet tweet = mapper.readValue(tweetJson, Tweet.class);
-    List<Activity> activityList = activityConverterUtil.convert(tweet);
-    Assert.assertTrue(activityList.size() == 1);
-    Activity activity = activityList.get(0);
-    if ( !ActivityUtil.isValid(activity) ) {
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testConvertRetweet() throws Exception  {
-    Retweet retweet = mapper.readValue(retweetJson, Retweet.class);
-    List<Activity> activityList = activityConverterUtil.convert(retweet);
-    Assert.assertEquals(activityList.size(), 1);
-    Activity activity = activityList.get(0);
-    if ( !ActivityUtil.isValid(activity) ) {
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testConvertDelete() throws Exception  {
-    Delete delete = mapper.readValue(deleteJson, Delete.class);
-    List<Activity> activityList = activityConverterUtil.convert(delete);
-    Assert.assertEquals(activityList.size(), 1);
-    Activity activity = activityList.get(0);
-    if ( !ActivityUtil.isValid(activity) ) {
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testConvertFollow() throws Exception {
-    Follow follow = mapper.readValue(followJson, Follow.class);
-    List<Activity> activityList = activityConverterUtil.convert(follow);
-    Assert.assertTrue(activityList.size() == 1);
-    Activity activity = activityList.get(0);
-    if ( !ActivityUtil.isValid(activity) ) {
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testConvertTweetString() {
-    List<Activity> activityList = activityConverterUtil.convert(tweetJson);
-    Assert.assertTrue(activityList.size() == 1);
-    Activity activity = activityList.get(0);
-    if ( !ActivityUtil.isValid(activity) ) {
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testConvertRetweetString() {
-    List<Activity> activityList = activityConverterUtil.convert(retweetJson);
-    Assert.assertTrue(activityList.size() == 1);
-    Activity activity = activityList.get(0);
-    if ( !ActivityUtil.isValid(activity) ) {
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testConvertDeleteString() {
-    String deleteJson = "{\"delete\":{\"status\":{\"id\":56789,\"user_id\":67890,\"id_str\":\"56789\",\"user_id_str\":\"67890\"}}}\n";
-    List<Activity> activityList = activityConverterUtil.convert(deleteJson);
-    Assert.assertTrue(activityList.size() == 1);
-    Activity activity = activityList.get(0);
-    if ( !ActivityUtil.isValid(activity) ) {
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testConvertFollowString() {
-    String followJson = "{\"follower\":{\"id\":12345},\"followee\":{\"id\":56789}}\n";
-    List<Activity> activityList = activityConverterUtil.convert(followJson);
-    Assert.assertTrue(activityList.size() == 1);
-    Activity activity = activityList.get(0);
-    if ( !ActivityUtil.isValid(activity) ) {
-      Assert.fail();
-    }
-  }
-}
diff --git a/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/utils/TwitterActivityObjectsConvertersTest.java b/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/utils/TwitterActivityObjectsConvertersTest.java
deleted file mode 100644
index 4b81293..0000000
--- a/streams-contrib/streams-provider-twitter/src/test/java/org/apache/streams/twitter/test/utils/TwitterActivityObjectsConvertersTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.streams.twitter.test.utils;
-
-import org.apache.streams.converter.ActivityObjectConverterProcessorConfiguration;
-import org.apache.streams.converter.ActivityObjectConverterUtil;
-import org.apache.streams.data.util.ActivityUtil;
-import org.apache.streams.jackson.StreamsJacksonMapper;
-import org.apache.streams.pojo.json.ActivityObject;
-import org.apache.streams.twitter.converter.TwitterDateTimeFormat;
-import org.apache.streams.twitter.converter.TwitterDocumentClassifier;
-import org.apache.streams.twitter.converter.TwitterJsonUserActivityObjectConverter;
-import org.apache.streams.twitter.pojo.User;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-/**
- * Tests {org.apache.streams.twitter.converter.*}
- */
-public class TwitterActivityObjectsConvertersTest {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(TwitterActivityObjectsConvertersTest.class);
-
-  private ObjectMapper mapper = StreamsJacksonMapper.getInstance(Stream.of(TwitterDateTimeFormat.TWITTER_FORMAT)
-      .collect(Collectors.toList()));
-
-  private ActivityObjectConverterProcessorConfiguration activityObjectConverterProcessorConfiguration =
-      new ActivityObjectConverterProcessorConfiguration()
-          .withClassifiers(Stream.of(new TwitterDocumentClassifier()).collect(Collectors.toList()))
-          .withConverters(Stream.of(new TwitterJsonUserActivityObjectConverter()).collect(Collectors.toList()));
-
-  private ActivityObjectConverterUtil activityObjectConverterUtil = ActivityObjectConverterUtil.getInstance(activityObjectConverterProcessorConfiguration);
-
-  private String userJson = "{\"id\":1663018644,\"id_str\":\"1663018644\",\"name\":\"M.R. Clark\",\"screen_name\":\"cantennisfan\",\"location\":\"\",\"url\":null,\"description\":null,\"protected\":false,\"verified\":false,\"followers_count\":0,\"friends_count\":5,\"listed_count\":0,\"favourites_count\":2,\"statuses_count\":72,\"created_at\":\"Sun Aug 11 17:23:47 +0000 2013\",\"utc_offset\":-18000,\"time_zone\":\"Eastern Time (US & Canada)\",\"geo_enabled\":false,\"lang\":\"en\",\"contrib [...]
-
-  @Test
-  public void testConvertUser() throws Exception {
-    User user = mapper.readValue(userJson, User.class);
-    ActivityObject activityObject = activityObjectConverterUtil.convert(user);
-    assert ( activityObject != null );
-    if ( !ActivityUtil.isValid(activityObject) ) {
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testConvertUserString() {
-    ActivityObject activityObject = activityObjectConverterUtil.convert(userJson);
-    assert ( activityObject != null );
-    if ( !ActivityUtil.isValid(activityObject) ) {
-      Assert.fail();
-    }
-  }
-}
diff --git a/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsDateDeserializer.java b/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsDateDeserializer.java
new file mode 100644
index 0000000..a9fa9f1
--- /dev/null
+++ b/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsDateDeserializer.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.streams.jackson;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * StreamsDateDeserializer is a supporting class for
+ * @see {@link org.apache.streams.jackson.StreamsJacksonMapper}
+ *
+ * <p></p>
+ * Converting date-time strings other than RFC3339 to java.util.Date objects requires
+ * additional formats to be provided when instantiating StreamsJacksonMapper.
+ */
+public class StreamsDateDeserializer extends StdDeserializer<Date> implements Serializable {
+
+  private List<DateTimeFormatter> formatters = new ArrayList<>();
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(StreamsDateDeserializer.class);
+
+  protected StreamsDateDeserializer(Class<Date> dateClass) {
+    super(dateClass);
+  }
+
+  protected StreamsDateDeserializer(Class<Date> dateClass, List<String> formats) {
+    super(dateClass);
+    for ( String format : formats ) {
+      try {
+        formatters.add(DateTimeFormatter.ofPattern(format));
+      } catch (Exception ex) {
+        LOGGER.warn("Exception parsing format " + format);
+      }
+    }
+  }
+
+  /**
+   * Applies each additional format in turn, until it can provide a non-null DateTime
+   */
+  @Override
+  public Date deserialize(JsonParser jpar, DeserializationContext context) throws IOException {
+
+    Date result = Date.from(LocalDateTime.parse(jpar.getValueAsString(), DateTimeFormatter.ISO_INSTANT).atZone(ZoneOffset.UTC).toInstant());
+    Iterator<DateTimeFormatter> iterator = formatters.iterator();
+    while ( result == null && iterator.hasNext()) {
+      DateTimeFormatter formatter = iterator.next();
+      result = Date.from(LocalDateTime.parse(jpar.getValueAsString(), formatter).atZone(ZoneOffset.UTC).toInstant());
+    }
+    return result;
+  }
+}
diff --git a/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsDateSerializer.java b/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsDateSerializer.java
new file mode 100644
index 0000000..3e91df5
--- /dev/null
+++ b/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsDateSerializer.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.streams.jackson;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+
+import java.util.Date;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * StreamsDateSerializer is a supporting class for
+ * @link org.apache.streams.jackson.StreamsJacksonMapper.
+ */
+public class StreamsDateSerializer extends StdSerializer<Date> implements Serializable {
+
+  protected StreamsDateSerializer(Class<Date> dateTimeClass) {
+    super(dateTimeClass);
+  }
+
+  @Override
+  public void serialize(Date value, JsonGenerator jgen, SerializerProvider provider) throws IOException {
+    jgen.writeString(ZonedDateTime.ofInstant(value.toInstant(), ZoneOffset.UTC).format(DateTimeFormatter.ISO_INSTANT));
+  }
+}
diff --git a/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsJacksonModule.java b/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsJacksonModule.java
index 9cd1c9c..11dbde6 100644
--- a/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsJacksonModule.java
+++ b/streams-pojo/src/main/java/org/apache/streams/jackson/StreamsJacksonModule.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.List;
 import java.util.Set;
 
@@ -61,6 +62,9 @@ public class StreamsJacksonModule extends SimpleModule {
       }
     }
 
+    addSerializer(Date.class, new StreamsDateSerializer(Date.class));
+    addDeserializer(Date.class, new StreamsDateDeserializer(Date.class, dateTimeFormats));
+
     addSerializer(DateTime.class, new StreamsDateTimeSerializer(DateTime.class));
     addDeserializer(DateTime.class, new StreamsDateTimeDeserializer(DateTime.class, dateTimeFormats));