Skip to content

MINOR: Update usage of deprecated API#6146

Merged
mjsax merged 4 commits intoapache:trunkfrom
mjsax:minor-code-cleanup-streamsresetter
Jan 29, 2019
Merged

MINOR: Update usage of deprecated API#6146
mjsax merged 4 commits intoapache:trunkfrom
mjsax:minor-code-cleanup-streamsresetter

Conversation

@mjsax
Copy link
Copy Markdown
Member

@mjsax mjsax commented Jan 15, 2019

close(long, TimeUnit) was deprecated and replaced with close(Duration);

@mjsax mjsax added the streams label Jan 15, 2019
@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 15, 2019

Call for review @guozhangwang @bbejeck @vvcephei

} finally {
if (kafkaAdminClient != null) {
kafkaAdminClient.close(60, TimeUnit.SECONDS);
kafkaAdminClient.close(java.time.Duration.ofSeconds(60));
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We cannot import java.time.Duration because we already import javax.xml.datatype.Duration.

I was wondering if we could get rid of javax.xml.datatype.Duration though and replace it with java.time.Duration that was not available in Java7 when we added javax.xml.datatype.Duration.

Thoughts? \cc @jeqo who extended the reset tool via #3831

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The only place we need this duration is

duration.negate().addTo(now);

java.time.Duration has negated but its addTo is taking a Temporal, and all its extends like LocalTime etc do not have the right API of getTime. So I think we just keep it like this.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should remove the javax.xml.datatype.Duration usage since it's not part of the Java base module.

Copy link
Copy Markdown
Member Author

@mjsax mjsax Jan 22, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was locking into the corresponding KIPs

Both say:

--by-duration  PnDTnHnMnS

Duration must be specified in ISO8601 format.

Looking into JavaDocs of java.time.Duration#parse() it says:

     * Obtains a {@code Duration} from a text string such as {@code PnDTnHnMn.nS}.
     * <p>
     * This will parse a textual representation of a duration, including the
     * string produced by {@code toString()}. The formats accepted are based
     * on the ISO-8601 duration format {@code PnDTnHnMn.nS} with days
     * considered to be exactly 24 hours.

Thus, it seems save to replace

final javax.xml.datatype.Duration durationParsed = DatatypeFactory.newInstance().newDuration(duration);

with

final java.time.Duration durationParsed = java.time.Duration.parse(duration);

Btw: we should do the same for kafka.admin.ConsumerGroupCommand (I can add this change to this PR).

Thoughts?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mjsax LGTM

Both kafka/tools/StreamsResetter.java and kafka/admin/ConsumerGroupCommand.scala should be migrated to remove usage of javax.xml.datatype.*

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mjsax , I agree, it seems safe and desirable to swap them out.

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 23, 2019

Updated this. Can you please review again @guozhangwang

import java.util.stream.Collectors;

/**
* {@link StreamsResetter} resets the processing state of a Kafka Streams application so that, for example, you can reprocess its input from scratch.
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just shorting some too long lines here -- not content change.

final String duration = options.valueOf(byDurationOption);
final Duration durationParsed = DatatypeFactory.newInstance().newDuration(duration);
resetByDuration(client, inputTopicPartitions, durationParsed);
resetByDuration(client, inputTopicPartitions, Duration.parse(duration));
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the actual update to move to java.time.Duration

final long timestamp = now.getTime();
final Instant now = Instant.now();
duration.negated().addTo(now);
final long timestamp = now.toEpochMilli();
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Part of actual update.

val durationParsed = Duration.parse(duration)
val now = Instant.now()
durationParsed.negated().addTo(now)
val timestamp = now.toEpochMilli
Copy link
Copy Markdown
Contributor

@vvcephei vvcephei Jan 23, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this code covered by any tests? Duration and Instant are immutable, so I don't think this code would work.

What do you think about:

Suggested change
val timestamp = now.toEpochMilli
val timestamp = now.minus(durationParsed).toEpochMilli()

(also applies to the same change below)

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this code covered by any tests?

Yes. Jenkins failed (I did not run the tests locally before I pushed...) -- Same issue in StreamsResetter.

@guozhangwang
Copy link
Copy Markdown
Contributor

retest this

@guozhangwang
Copy link
Copy Markdown
Contributor

LGTM modulo @vvcephei 's comment. Please feel free to merge after comments addressed and jenkins passed

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 24, 2019

Updates this.

Copy link
Copy Markdown
Contributor

@vvcephei vvcephei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks!

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 29, 2019

Retest this please

@mjsax mjsax merged commit 1fa02d5 into apache:trunk Jan 29, 2019
@mjsax mjsax deleted the minor-code-cleanup-streamsresetter branch January 29, 2019 19:15
jarekr pushed a commit to confluentinc/kafka that referenced this pull request Apr 18, 2019
* ak/trunk:
  MINOR: Update usage of deprecated API (apache#6146)
  KAFKA-4217: Add KStream.flatTransform (apache#5273)
  MINOR: Update Gradle to 5.1.1 (apache#6160)
  KAFKA-3522: Generalize Segments (apache#6170)
  Added quotes around the class path (apache#4469)
  KAFKA-7837: Ensure offline partitions are picked up as soon as possible when shrinking ISR (apache#6202)
  MINOR: In the MetadataResponse schema, ignorable should be a boolean
  KAFKA-7838: Log leader and follower end offsets when shrinking ISR (apache#6168)
  KAFKA-5692: Change PreferredReplicaLeaderElectionCommand to use Admin… (apache#3848)
  MINOR: clarify why suppress can sometimes drop tombstones (apache#6195)
  MINOR: Upgrade ducktape to 0.7.5 (apache#6197)
  MINOR: Improve IntegrationTestUtils documentation (apache#5664)
  MINOR: upgrade to jdk8 8u202
  KAFKA-7693; Fix SequenceNumber overflow in producer (apache#5989)
  KAFKA-7692; Fix ProducerStateManager SequenceNumber overflow (apache#5990)
  MINOR: update copyright year in the NOTICE file. (apache#6196)
  KAFKA-7793: Improve the Trogdor command line. (apache#6133)
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
Reviewers: Guozhang Wang <guozhang@confluent.io>, Ismael Juma <ismael@confluent.io>, Jorge Quilcate Otoya <quilcate.jorge@gmail.com>, John Roesler <john@confluent.io>, Bill Bejeck <bill@confluent.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants